8888 from apache_beam .runners .sdf_utils import SplitResultResidual
8989 from apache_beam .runners .worker import data_plane
9090 from apache_beam .runners .worker import sdk_worker
91- from apache_beam .transforms import window
9291 from apache_beam .transforms .core import Windowing
92+ from apache_beam .transforms .window import BoundedWindow
9393 from apache_beam .utils import windowed_value
9494
9595# This module is experimental. No backwards-compatibility guarantees.
@@ -381,7 +381,7 @@ def __init__(self,
381381 self ._element_coder = coder .wrapped_value_coder
382382 self ._target_window_coder = coder .window_coder
383383 # TODO(robertwb): Limit the cache size.
384- self ._cache = {} # type: Dict[window. BoundedWindow, Any]
384+ self ._cache = {} # type: Dict[BoundedWindow, Any]
385385
386386 def __getitem__ (self , window ):
387387 target_window = self ._side_input_data .window_mapping_fn (window )
@@ -644,14 +644,14 @@ def commit(self):
644644class OutputTimer (userstate .BaseTimer ):
645645 def __init__ (self ,
646646 key ,
647- window , # type: windowed_value. BoundedWindow
647+ window , # type: BoundedWindow
648648 timestamp , # type: timestamp.Timestamp
649649 paneinfo , # type: windowed_value.PaneInfo
650650 time_domain , # type: str
651651 timer_family_id , # type: str
652652 timer_coder_impl , # type: coder_impl.TimerCoderImpl
653653 output_stream # type: data_plane.ClosableOutputStream
654- ):
654+ ):
655655 self ._key = key
656656 self ._window = window
657657 self ._input_timestamp = timestamp
@@ -661,12 +661,11 @@ def __init__(self,
661661 self ._output_stream = output_stream
662662 self ._timer_coder_impl = timer_coder_impl
663663
664- def set (self , ts ):
665- # type: (timestamp.TimestampTypes) -> None
664+ def set (self , ts : timestamp .TimestampTypes , dynamic_timer_tag = '' ) -> None :
666665 ts = timestamp .Timestamp .of (ts )
667666 timer = userstate .Timer (
668667 user_key = self ._key ,
669- dynamic_timer_tag = '' ,
668+ dynamic_timer_tag = dynamic_timer_tag ,
670669 windows = (self ._window , ),
671670 clear_bit = False ,
672671 fire_timestamp = ts ,
@@ -676,11 +675,10 @@ def set(self, ts):
676675 self ._timer_coder_impl .encode_to_stream (timer , self ._output_stream , True )
677676 self ._output_stream .maybe_flush ()
678677
679- def clear (self ):
680- # type: () -> None
678+ def clear (self , dynamic_timer_tag = '' ) -> None :
681679 timer = userstate .Timer (
682680 user_key = self ._key ,
683- dynamic_timer_tag = '' ,
681+ dynamic_timer_tag = dynamic_timer_tag ,
684682 windows = (self ._window , ),
685683 clear_bit = True ,
686684 fire_timestamp = None ,
@@ -713,10 +711,8 @@ def __init__(self,
713711 Args:
714712 state_handler: A StateServicer object.
715713 transform_id: The name of the PTransform that this context is associated.
716- key_coder:
717- window_coder:
718- timer_family_specs: A list of ``userstate.TimerSpec`` objects specifying
719- the timers associated with this operation.
714+ key_coder: Coder for the key type.
715+ window_coder: Coder for the window type.
720716 """
721717 self ._state_handler = state_handler
722718 self ._transform_id = transform_id
@@ -731,23 +727,20 @@ def add_timer_info(self, timer_family_id, timer_info):
731727 self ._timers_info [timer_family_id ] = timer_info
732728
733729 def get_timer (
734- self ,
735- timer_spec ,
736- key ,
737- window , # type: windowed_value.BoundedWindow
738- timestamp ,
739- pane ):
740- # type: (...) -> OutputTimer
730+ self , timer_spec : userstate .TimerSpec , key , window , timestamp ,
731+ pane ) -> OutputTimer :
741732 assert self ._timers_info [timer_spec .name ].output_stream is not None
733+ timer_coder_impl = self ._timers_info [timer_spec .name ].timer_coder_impl
734+ output_stream = self ._timers_info [timer_spec .name ].output_stream
742735 return OutputTimer (
743736 key ,
744737 window ,
745738 timestamp ,
746739 pane ,
747740 timer_spec .time_domain ,
748741 timer_spec .name ,
749- self . _timers_info [ timer_spec . name ]. timer_coder_impl ,
750- self . _timers_info [ timer_spec . name ]. output_stream )
742+ timer_coder_impl ,
743+ output_stream )
751744
752745 def get_state (self , * args ):
753746 # type: (*Any) -> FnApiUserRuntimeStateTypes
@@ -759,7 +752,7 @@ def get_state(self, *args):
759752 def _create_state (self ,
760753 state_spec , # type: userstate.StateSpec
761754 key ,
762- window # type: windowed_value. BoundedWindow
755+ window # type: BoundedWindow
763756 ):
764757 # type: (...) -> FnApiUserRuntimeStateTypes
765758 if isinstance (state_spec ,
0 commit comments