Skip to content
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
merged 4 commits into from
Sep 12, 2022
Merged
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
11 changes: 8 additions & 3 deletions sdks/python/apache_beam/transforms/trigger.py
Original file line number Diff line number Diff line change
Expand Up @@ -377,6 +377,9 @@ class AfterProcessingTime(TriggerFn):

AfterProcessingTime is experimental. No backwards compatibility guarantees.
"""

COUNT_TAG = _CombiningValueStateTag('count', combiners.CountCombineFn())

def __init__(self, delay=0):
"""Initialize a processing time trigger with a delay in seconds."""
self.delay = delay
Expand All @@ -385,8 +388,10 @@ def __repr__(self):
return 'AfterProcessingTime(delay=%d)' % self.delay

def on_element(self, element, window, context):
context.set_timer(
'', TimeDomain.REAL_TIME, context.get_current_time() + self.delay)
context.add_state(self.COUNT_TAG, 1)
if context.get_state(self.COUNT_TAG) == 1:
Copy link
Member

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?

Suggested change
if context.get_state(self.COUNT_TAG) == 1:
if context.get_state(self.COUNT_TAG) >= 1:

Copy link
Contributor Author

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.

Copy link
Contributor Author

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

context.set_timer(
'', TimeDomain.REAL_TIME, context.get_current_time() + self.delay)

def on_merge(self, to_be_merged, merge_result, context):
# timers will be kept through merging
Expand All @@ -400,7 +405,7 @@ def on_fire(self, timestamp, window, context):
return True

def reset(self, window, context):
pass
context.clear_state(self.COUNT_TAG)

def may_lose_data(self, unused_windowing):
"""AfterProcessingTime may finish."""
Expand Down
82 changes: 82 additions & 0 deletions sdks/python/apache_beam/transforms/trigger_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -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]
Copy link
Contributor Author

Choose a reason for hiding this comment

The 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:
Expand Down