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

Skip to content

Commit 6eefa9c

Browse files
committed
Never trigger for Python.
1 parent a7866b5 commit 6eefa9c

2 files changed

Lines changed: 66 additions & 1 deletion

File tree

sdks/python/apache_beam/transforms/trigger.py

Lines changed: 43 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -256,7 +256,7 @@ def from_runner_api(proto, context):
256256
'always': Always,
257257
'default': DefaultTrigger,
258258
'element_count': AfterCount,
259-
# never
259+
'never': Never,
260260
'or_finally': OrFinally,
261261
'repeat': Repeatedly,
262262
}[proto.WhichOneof('trigger')].from_runner_api(proto, context)
@@ -408,6 +408,48 @@ def to_runner_api(self, context):
408408
always=beam_runner_api_pb2.Trigger.Always())
409409

410410

411+
class Never(TriggerFn):
412+
"""A trigger that never fires.
413+
"""
414+
def __init__(self):
415+
pass
416+
417+
def __repr__(self):
418+
return 'Never'
419+
420+
def __eq__(self, other):
421+
return type(self) == type(other)
422+
423+
def __hash__(self):
424+
return hash(type(self))
425+
426+
def on_element(self, element, window, context):
427+
pass
428+
429+
def on_merge(self, to_be_merged, merge_result, context):
430+
pass
431+
432+
def has_ontime_pane(self):
433+
False
434+
435+
def reset(self, window, context):
436+
pass
437+
438+
def should_fire(self, time_domain, watermark, window, context):
439+
return False
440+
441+
def on_fire(self, watermark, window, context):
442+
return True
443+
444+
@staticmethod
445+
def from_runner_api(proto, context):
446+
return Never()
447+
448+
def to_runner_api(self, context):
449+
return beam_runner_api_pb2.Trigger(
450+
never=beam_runner_api_pb2.Trigger.Never())
451+
452+
411453
class AfterWatermark(TriggerFn):
412454
"""Fire exactly once when the watermark passes the end of the window.
413455

sdks/python/apache_beam/transforms/trigger_test.py

Lines changed: 23 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -59,6 +59,7 @@
5959
from apache_beam.transforms.trigger import DefaultTrigger
6060
from apache_beam.transforms.trigger import GeneralTriggerDriver
6161
from apache_beam.transforms.trigger import InMemoryUnmergedState
62+
from apache_beam.transforms.trigger import Never
6263
from apache_beam.transforms.trigger import Repeatedly
6364
from apache_beam.transforms.trigger import TriggerFn
6465
from apache_beam.transforms.window import FixedWindows
@@ -518,6 +519,28 @@ def format_result(k_v):
518519
'B-3': {10, 15, 16},
519520
}.items())))
520521

522+
def test_never(self):
523+
with TestPipeline() as p:
524+
525+
def construct_timestamped(k_t):
526+
return TimestampedValue((k_t[0], k_t[1]), k_t[1])
527+
528+
def format_result(k_v):
529+
return ('%s-%s' % (k_v[0], len(k_v[1])), set(k_v[1]))
530+
531+
result = (
532+
p
533+
| beam.Create([1, 1, 2, 3, 4, 5, 10, 11])
534+
| beam.FlatMap(lambda t: [('A', t), ('B', t + 5)])
535+
| beam.Map(construct_timestamped)
536+
| beam.WindowInto(
537+
FixedWindows(10),
538+
trigger=Never(),
539+
accumulation_mode=AccumulationMode.DISCARDING)
540+
| beam.GroupByKey()
541+
| beam.Map(format_result))
542+
assert_that(result, equal_to([]))
543+
521544
def test_multiple_accumulating_firings(self):
522545
# PCollection will contain elements from 1 to 10.
523546
elements = [i for i in range(1, 11)]

0 commit comments

Comments
 (0)