Thanks to visit codestin.com
Credit goes to github.com

Skip to content

Commit f0ba6bf

Browse files
committed
[BEAM-9735] Adding Always trigger and using it in Reshuffle
1 parent 1a217db commit f0ba6bf

3 files changed

Lines changed: 79 additions & 5 deletions

File tree

sdks/python/apache_beam/transforms/trigger.py

Lines changed: 43 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -253,7 +253,7 @@ def from_runner_api(proto, context):
253253
'after_end_of_window': AfterWatermark,
254254
'after_processing_time': AfterProcessingTime,
255255
# after_processing_time, after_synchronized_processing_time
256-
# always
256+
'always': Always,
257257
'default': DefaultTrigger,
258258
'element_count': AfterCount,
259259
# never
@@ -367,6 +367,47 @@ def has_ontime_pane(self):
367367
return False
368368

369369

370+
class Always(TriggerFn):
371+
"""Repeatedly invoke the given trigger, never finishing."""
372+
def __init__(self):
373+
pass
374+
375+
def __repr__(self):
376+
return 'Always'
377+
378+
def __eq__(self, other):
379+
return type(self) == type(other)
380+
381+
def __hash__(self):
382+
return 1
383+
384+
def on_element(self, element, window, context):
385+
pass
386+
387+
def on_merge(self, to_be_merged, merge_result, context):
388+
pass
389+
390+
def has_ontime_pane(self):
391+
False
392+
393+
def reset(self, window, context):
394+
pass
395+
396+
def should_fire(self, time_domain, watermark, window, context):
397+
return True
398+
399+
def on_fire(self, watermark, window, context):
400+
return False
401+
402+
@staticmethod
403+
def from_runner_api(proto, context):
404+
return Always()
405+
406+
def to_runner_api(self, context):
407+
return beam_runner_api_pb2.Trigger(
408+
always=beam_runner_api_pb2.Trigger.Always())
409+
410+
370411
class AfterWatermark(TriggerFn):
371412
"""Fire exactly once when the watermark passes the end of the window.
372413
@@ -985,7 +1026,7 @@ def create_trigger_driver(
9851026
if windowing.is_default() and is_batch:
9861027
driver = BatchGlobalTriggerDriver()
9871028
elif (windowing.windowfn == GlobalWindows() and
988-
windowing.triggerfn == AfterCount(1) and is_batch):
1029+
(windowing.triggerfn in [AfterCount(1), Always()]) and is_batch):
9891030
# Here we also just pass through all the values exactly once.
9901031
driver = BatchGlobalTriggerDriver()
9911032
else:

sdks/python/apache_beam/transforms/trigger_test.py

Lines changed: 34 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -55,6 +55,7 @@
5555
from apache_beam.transforms.trigger import AfterEach
5656
from apache_beam.transforms.trigger import AfterProcessingTime
5757
from apache_beam.transforms.trigger import AfterWatermark
58+
from apache_beam.transforms.trigger import Always
5859
from apache_beam.transforms.trigger import DefaultTrigger
5960
from apache_beam.transforms.trigger import GeneralTriggerDriver
6061
from apache_beam.transforms.trigger import InMemoryUnmergedState
@@ -484,6 +485,39 @@ def format_result(k_v):
484485
'B-3': {10, 15, 16},
485486
}.items())))
486487

488+
def test_always(self):
489+
with TestPipeline() as p:
490+
491+
def construct_timestamped(k_t):
492+
return TimestampedValue((k_t[0], k_t[1]), k_t[1])
493+
494+
def format_result(k_v):
495+
return ('%s-%s' % (k_v[0], len(k_v[1])), set(k_v[1]))
496+
497+
result = (
498+
p
499+
| beam.Create([1, 1, 2, 3, 4, 5, 10, 11])
500+
| beam.FlatMap(lambda t: [('A', t), ('B', t + 5)])
501+
| beam.Map(construct_timestamped)
502+
| beam.WindowInto(
503+
FixedWindows(10),
504+
trigger=Always(),
505+
accumulation_mode=AccumulationMode.DISCARDING)
506+
| beam.GroupByKey()
507+
| beam.Map(format_result))
508+
assert_that(
509+
result,
510+
equal_to(
511+
list({
512+
'A-2': {10, 11},
513+
# Elements out of windows are also emitted.
514+
'A-6': {1, 2, 3, 4, 5},
515+
# A,1 is emitted twice.
516+
'B-5': {6, 7, 8, 9},
517+
# B,6 is emitted twice.
518+
'B-3': {10, 15, 16},
519+
}.items())))
520+
487521
def test_multiple_accumulating_firings(self):
488522
# PCollection will contain elements from 1 to 10.
489523
elements = [i for i in range(1, 11)]

sdks/python/apache_beam/transforms/util.py

Lines changed: 2 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -63,8 +63,7 @@
6363
from apache_beam.transforms.ptransform import ptransform_fn
6464
from apache_beam.transforms.timeutil import TimeDomain
6565
from apache_beam.transforms.trigger import AccumulationMode
66-
from apache_beam.transforms.trigger import AfterCount
67-
from apache_beam.transforms.trigger import Repeatedly
66+
from apache_beam.transforms.trigger import Always
6867
from apache_beam.transforms.userstate import BagStateSpec
6968
from apache_beam.transforms.userstate import CombiningValueStateSpec
7069
from apache_beam.transforms.userstate import TimerSpec
@@ -679,7 +678,7 @@ def restore_timestamps(element):
679678
# accept only standard coders.
680679
ungrouped._windowing = Windowing(
681680
window.GlobalWindows(),
682-
triggerfn=Repeatedly(AfterCount(1)),
681+
triggerfn=Always(),
683682
accumulation_mode=AccumulationMode.DISCARDING,
684683
timestamp_combiner=TimestampCombiner.OUTPUT_AT_EARLIEST)
685684
result = (

0 commit comments

Comments
 (0)