@@ -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+
411453class AfterWatermark (TriggerFn ):
412454 """Fire exactly once when the watermark passes the end of the window.
413455
0 commit comments