@@ -304,7 +304,7 @@ def from_runner_api(proto, context):
304304 'after_each' : AfterEach ,
305305 'after_end_of_window' : AfterWatermark ,
306306 'after_processing_time' : AfterProcessingTime ,
307- # after_processing_time, after_synchronized_processing_time
307+ 'after_synchronized_processing_time' : _AfterSynchronizedProcessingTime ,
308308 'always' : Always ,
309309 'default' : DefaultTrigger ,
310310 'element_count' : AfterCount ,
@@ -317,6 +317,17 @@ def from_runner_api(proto, context):
317317 def to_runner_api (self , unused_context ):
318318 pass
319319
320+ @abstractmethod
321+ def get_continuation_trigger (self ):
322+ """Returns:
323+ Trigger to use after a GroupBy to preserve the intention of this
324+ trigger. Specifically, triggers that are time based and intended
325+ to provide speculative results should continue providing speculative
326+ results. Triggers that fire once (or multiple times) should
327+ continue firing once (or multiple times).
328+ """
329+ pass
330+
320331
321332class DefaultTrigger (TriggerFn ):
322333 """Semantically Repeatedly(AfterWatermark()), but more optimized."""
@@ -366,6 +377,9 @@ def to_runner_api(self, unused_context):
366377 def has_ontime_pane (self ):
367378 return True
368379
380+ def get_continuation_trigger (self ):
381+ return self
382+
369383
370384class AfterProcessingTime (TriggerFn ):
371385 """Fire exactly once after a specified delay from processing time."""
@@ -421,6 +435,11 @@ def to_runner_api(self, context):
421435 def has_ontime_pane (self ):
422436 return False
423437
438+ def get_continuation_trigger (self ):
439+ # The continuation of an AfterProcessingTime trigger is an
440+ # _AfterSynchronizedProcessingTime trigger.
441+ return _AfterSynchronizedProcessingTime ()
442+
424443
425444class Always (TriggerFn ):
426445 """Repeatedly invoke the given trigger, never finishing."""
@@ -466,6 +485,9 @@ def to_runner_api(self, context):
466485 return beam_runner_api_pb2 .Trigger (
467486 always = beam_runner_api_pb2 .Trigger .Always ())
468487
488+ def get_continuation_trigger (self ):
489+ return self
490+
469491
470492class _Never (TriggerFn ):
471493 """A trigger that never fires.
@@ -518,6 +540,9 @@ def to_runner_api(self, context):
518540 return beam_runner_api_pb2 .Trigger (
519541 never = beam_runner_api_pb2 .Trigger .Never ())
520542
543+ def get_continuation_trigger (self ):
544+ return self
545+
521546
522547class AfterWatermark (TriggerFn ):
523548 """Fire exactly once when the watermark passes the end of the window.
@@ -531,9 +556,19 @@ class AfterWatermark(TriggerFn):
531556 LATE_TAG = _CombiningValueStateTag ('is_late' , any )
532557
533558 def __init__ (self , early = None , late = None ):
534- # TODO(zhoufek): Maybe don't wrap early/late if they are already Repeatedly
535- self .early = Repeatedly (early ) if early else None
536- self .late = Repeatedly (late ) if late else None
559+ self .early = self ._wrap_if_not_repeatedly (early )
560+ self .late = self ._wrap_if_not_repeatedly (late )
561+
562+ @staticmethod
563+ def _wrap_if_not_repeatedly (trigger ):
564+ if trigger and not isinstance (trigger , Repeatedly ):
565+ return Repeatedly (trigger )
566+ return trigger
567+
568+ def get_continuation_trigger (self ):
569+ return AfterWatermark (
570+ self .early .get_continuation_trigger () if self .early else None ,
571+ self .late .get_continuation_trigger () if self .late else None )
537572
538573 def __repr__ (self ):
539574 qualifiers = []
@@ -692,6 +727,9 @@ def to_runner_api(self, unused_context):
692727 def has_ontime_pane (self ):
693728 return False
694729
730+ def get_continuation_trigger (self ):
731+ return AfterCount (1 )
732+
695733
696734class Repeatedly (TriggerFn ):
697735 """Repeatedly invoke the given trigger, never finishing."""
@@ -741,6 +779,9 @@ def to_runner_api(self, context):
741779 def has_ontime_pane (self ):
742780 return self .underlying .has_ontime_pane ()
743781
782+ def get_continuation_trigger (self ):
783+ return Repeatedly (self .underlying .get_continuation_trigger ())
784+
744785
745786class _ParallelTriggerFn (TriggerFn , metaclass = ABCMeta ):
746787 def __init__ (self , * triggers ):
@@ -831,6 +872,12 @@ def to_runner_api(self, context):
831872 def has_ontime_pane (self ):
832873 return any (t .has_ontime_pane () for t in self .triggers )
833874
875+ def get_continuation_trigger (self ):
876+ return self .__class__ (
877+ * (
878+ subtrigger .get_continuation_trigger ()
879+ for subtrigger in self .triggers ))
880+
834881
835882class AfterAny (_ParallelTriggerFn ):
836883 """Fires when any subtrigger fires.
@@ -933,6 +980,13 @@ def to_runner_api(self, context):
933980 def has_ontime_pane (self ):
934981 return any (t .has_ontime_pane () for t in self .triggers )
935982
983+ def get_continuation_trigger (self ):
984+ return Repeatedly (
985+ AfterAny (
986+ * (
987+ subtrigger .get_continuation_trigger ()
988+ for subtrigger in self .triggers )))
989+
936990
937991class OrFinally (AfterAny ):
938992 @staticmethod
@@ -1643,3 +1697,60 @@ def __repr__(self):
16431697 state_str = '\n ' .join (
16441698 '%s: %s' % (key , dict (state )) for key , state in self .state .items ())
16451699 return 'timers: %s\n state: %s' % (dict (self .timers ), state_str )
1700+
1701+
1702+ class _AfterSynchronizedProcessingTime (TriggerFn ):
1703+ """A "runner's-discretion" trigger downstream of a GroupByKey
1704+ with AfterProcessingTime trigger.
1705+
1706+ In runners that directly execute this
1707+ Python code, the trigger currently always fires,
1708+ but this behavior is neither guaranteed nor
1709+ required by runners, regardless of whether they
1710+ execute triggers via Python.
1711+
1712+ _AfterSynchronizedProcessingTime is experimental
1713+ and internal-only. No backwards compatibility
1714+ guarantees.
1715+ """
1716+ def __init__ (self ):
1717+ pass
1718+
1719+ def __repr__ (self ):
1720+ return '_AfterSynchronizedProcessingTime()'
1721+
1722+ def __eq__ (self , other ):
1723+ return type (self ) == type (other )
1724+
1725+ def __hash__ (self ):
1726+ return hash (type (self ))
1727+
1728+ def on_element (self , _element , _window , _context ):
1729+ pass
1730+
1731+ def on_merge (self , _to_be_merged , _merge_result , _context ):
1732+ pass
1733+
1734+ def should_fire (self , _time_domain , _timestamp , _window , _context ):
1735+ return True
1736+
1737+ def on_fire (self , _timestamp , _window , _context ):
1738+ return False
1739+
1740+ def reset (self , _window , _context ):
1741+ pass
1742+
1743+ @staticmethod
1744+ def from_runner_api (_proto , _context ):
1745+ return _AfterSynchronizedProcessingTime ()
1746+
1747+ def to_runner_api (self , _context ):
1748+ return beam_runner_api_pb2 .Trigger (
1749+ after_synchronized_processing_time = beam_runner_api_pb2 .Trigger .
1750+ AfterSynchronizedProcessingTime ())
1751+
1752+ def has_ontime_pane (self ):
1753+ return False
1754+
1755+ def get_continuation_trigger (self ):
1756+ return self
0 commit comments