-
Notifications
You must be signed in to change notification settings - Fork 4.3k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[Issue#23071] Fix AfterProcessingTime for Python to behave like Java #23100
Merged
+91
−3
Merged
Changes from 1 commit
Commits
Show all changes
4 commits
Select commit
Hold shift + click to select a range
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -540,6 +540,88 @@ def test_trigger_encoding(self): | |
|
||
|
||
class TriggerPipelineTest(unittest.TestCase): | ||
def test_after_processing_time(self): | ||
test_options = PipelineOptions( | ||
flags=['--allow_unsafe_triggers', '--streaming']) | ||
with TestPipeline(options=test_options) as p: | ||
|
||
total_elements_in_trigger = 4 | ||
processing_time_delay = 2 | ||
window_size = 10 | ||
|
||
# yapf: disable | ||
test_stream = TestStream() | ||
for i in range(total_elements_in_trigger): | ||
(test_stream | ||
.advance_processing_time(processing_time_delay / total_elements_in_trigger ) | ||
.add_elements([('key', i)]) | ||
) | ||
|
||
test_stream.advance_processing_time(processing_time_delay) | ||
|
||
# Add dropped elements | ||
(test_stream | ||
.advance_processing_time(0.1) | ||
.add_elements([('key', "dropped-1")]) | ||
.advance_processing_time(0.1) | ||
.add_elements([('key', "dropped-2")]) | ||
) | ||
|
||
(test_stream | ||
.advance_processing_time(processing_time_delay) | ||
.advance_watermark_to_infinity() | ||
) | ||
# yapf: enable | ||
|
||
results = ( | ||
p | ||
| test_stream | ||
| beam.WindowInto( | ||
FixedWindows(window_size), | ||
trigger=AfterProcessingTime(processing_time_delay), | ||
accumulation_mode=AccumulationMode.DISCARDING) | ||
| beam.GroupByKey() | ||
| beam.Map(lambda x: x[1])) | ||
|
||
assert_that(results, equal_to([list(range(total_elements_in_trigger))])) | ||
|
||
def test_repeatedly_after_processing_time(self): | ||
test_options = PipelineOptions(flags=['--streaming']) | ||
with TestPipeline(options=test_options) as p: | ||
total_elements = 8 | ||
processing_time_delay = 2 | ||
window_size = 10 | ||
# yapf: disable | ||
test_stream = TestStream() | ||
for i in range(total_elements): | ||
(test_stream | ||
.advance_processing_time(processing_time_delay - 0.01) | ||
.add_elements([('key', i)]) | ||
) | ||
|
||
(test_stream | ||
.advance_processing_time(processing_time_delay) | ||
.advance_watermark_to_infinity() | ||
) | ||
# yapf: enable | ||
|
||
results = ( | ||
p | ||
| test_stream | ||
| beam.WindowInto( | ||
FixedWindows(window_size), | ||
trigger=Repeatedly(AfterProcessingTime(processing_time_delay)), | ||
accumulation_mode=AccumulationMode.DISCARDING) | ||
| beam.GroupByKey() | ||
| beam.Map(lambda x: x[1])) | ||
|
||
expected = [[i, i + 1] | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Maybe this is overly complicated? and we can just assume a fixed value |
||
for i in range(total_elements - total_elements % 2) | ||
if i % 2 == 0] | ||
expected += [] if total_elements % 2 == 0 else [[total_elements - 1]] | ||
|
||
assert_that(results, equal_to(expected)) | ||
|
||
def test_after_count(self): | ||
test_options = PipelineOptions(flags=['--allow_unsafe_triggers']) | ||
with TestPipeline(options=test_options) as p: | ||
|
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Would this be the correct check?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think that would make it the same as before, it would increase the timer for every element, we only want to increase if it's the first we see.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I have modified the logic from a Counter to a State, we don't need to actually count the elements, just modify the timer with the first element in the pane