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

Skip to content

Commit a679bab

Browse files
author
Yichi Zhang
committed
[BEAM-9602] Add timer family support for python SDK
1 parent 54225bd commit a679bab

15 files changed

Lines changed: 385 additions & 112 deletions

File tree

sdks/python/apache_beam/runners/common.pxd

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -44,6 +44,7 @@ cdef class MethodWrapper(object):
4444
cdef object restriction_provider_arg_name
4545
cdef object watermark_estimator_provider
4646
cdef object watermark_estimator_provider_arg_name
47+
cdef object dynamic_timer_tag_arg_name
4748
cdef bint unbounded_per_element
4849

4950

sdks/python/apache_beam/runners/common.py

Lines changed: 23 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -198,6 +198,7 @@ def __init__(self, obj_to_invoke, method_name):
198198
self.restriction_provider_arg_name = None
199199
self.watermark_estimator_provider = None
200200
self.watermark_estimator_provider_arg_name = None
201+
self.dynamic_timer_tag_arg_name = None
201202

202203
if hasattr(self.method_value, 'unbounded_per_element'):
203204
self.unbounded_per_element = True
@@ -223,14 +224,22 @@ def __init__(self, obj_to_invoke, method_name):
223224
elif isinstance(v, core.DoFn.WatermarkEstimatorParam):
224225
self.watermark_estimator_provider = v.watermark_estimator_provider
225226
self.watermark_estimator_provider_arg_name = kw
227+
elif core.DoFn.DynamicTimerTagParam == v:
228+
self.dynamic_timer_tag_arg_name = kw
226229

227230
# Create NoOpWatermarkEstimatorProvider if there is no
228231
# WatermarkEstimatorParam provided.
229232
if self.watermark_estimator_provider is None:
230233
self.watermark_estimator_provider = NoOpWatermarkEstimatorProvider()
231234

232235
def invoke_timer_callback(
233-
self, user_state_context, key, window, timestamp, pane_info):
236+
self,
237+
user_state_context,
238+
key,
239+
window,
240+
timestamp,
241+
pane_info,
242+
dynamic_timer_tag):
234243
# TODO(ccy): support side inputs.
235244
kwargs = {}
236245
if self.has_userstate_arguments:
@@ -246,6 +255,8 @@ def invoke_timer_callback(
246255
kwargs[self.window_arg_name] = window
247256
if self.key_arg_name:
248257
kwargs[self.key_arg_name] = key
258+
if self.dynamic_timer_tag_arg_name:
259+
kwargs[self.dynamic_timer_tag_arg_name] = dynamic_timer_tag
249260

250261
if kwargs:
251262
return self.method_value(**kwargs)
@@ -526,12 +537,18 @@ def invoke_teardown(self):
526537
"""
527538
self.signature.teardown_lifecycle_method.method_value()
528539

529-
def invoke_user_timer(self, timer_spec, key, window, timestamp, pane_info):
540+
def invoke_user_timer(
541+
self, timer_spec, key, window, timestamp, pane_info, dynamic_timer_tag):
530542
# self.output_processor is Optional, but in practice it won't be None here
531543
self.output_processor.process_outputs(
532544
WindowedValue(None, timestamp, (window, )),
533545
self.signature.timer_methods[timer_spec].invoke_timer_callback(
534-
self.user_state_context, key, window, timestamp, pane_info))
546+
self.user_state_context,
547+
key,
548+
window,
549+
timestamp,
550+
pane_info,
551+
dynamic_timer_tag))
535552

536553
def invoke_create_watermark_estimator(self, estimator_state):
537554
return self.signature.create_watermark_estimator_method.method_value(
@@ -1233,10 +1250,11 @@ def current_element_progress(self):
12331250
assert isinstance(self.do_fn_invoker, PerWindowInvoker)
12341251
return self.do_fn_invoker.current_element_progress()
12351252

1236-
def process_user_timer(self, timer_spec, key, window, timestamp, pane_info):
1253+
def process_user_timer(
1254+
self, timer_spec, key, window, timestamp, pane_info, dynamic_timer_tag):
12371255
try:
12381256
self.do_fn_invoker.invoke_user_timer(
1239-
timer_spec, key, window, timestamp, pane_info)
1257+
timer_spec, key, window, timestamp, pane_info, dynamic_timer_tag)
12401258
except BaseException as exn:
12411259
self._reraise_augmented(exn)
12421260

sdks/python/apache_beam/runners/direct/direct_userstate.py

Lines changed: 21 additions & 14 deletions
Original file line numberDiff line numberDiff line change
@@ -206,8 +206,7 @@ def __init__(self, step_context, dofn, key_coder):
206206
self.dofn = dofn
207207
self.key_coder = key_coder
208208

209-
self.all_state_specs, self.all_timer_specs = (
210-
userstate.get_dofn_specs(dofn))
209+
self.all_state_specs, self.all_timer_specs = userstate.get_dofn_specs(dofn)
211210
self.state_tags = {}
212211
for state_spec in self.all_state_specs:
213212
state_key = 'user/%s' % state_spec.name
@@ -226,12 +225,14 @@ def __init__(self, step_context, dofn, key_coder):
226225
self.cached_states = {}
227226
self.cached_timers = {}
228227

229-
def get_timer(self, timer_spec, key, window, timestamp, pane):
228+
def get_timer(
229+
self, timer_spec: userstate.TimerSpec, key, window, timestamp,
230+
pane) -> userstate.RuntimeTimer:
230231
assert timer_spec in self.all_timer_specs
231232
encoded_key = self.key_coder.encode(key)
232233
cache_key = (encoded_key, window, timer_spec)
233234
if cache_key not in self.cached_timers:
234-
self.cached_timers[cache_key] = userstate.RuntimeTimer(timer_spec)
235+
self.cached_timers[cache_key] = userstate.RuntimeTimer()
235236
return self.cached_timers[cache_key]
236237

237238
def get_state(self, state_spec, key, window):
@@ -291,16 +292,22 @@ def commit(self):
291292
encoded_key, window, timer_spec = cache_key
292293
state = self.step_context.get_keyed_state(encoded_key)
293294
timer_name = 'user/%s' % timer_spec.name
294-
if runtime_timer._cleared:
295-
state.clear_timer(window, timer_name, timer_spec.time_domain)
296-
if runtime_timer._new_timestamp is not None:
297-
# TODO(ccy): add corresponding watermark holds after the DirectRunner
298-
# allows for keyed watermark holds.
299-
state.set_timer(
300-
window,
301-
timer_name,
302-
timer_spec.time_domain,
303-
runtime_timer._new_timestamp)
295+
for dynamic_timer_tag, timer in runtime_timer._timer_recordings.items():
296+
if timer.cleared:
297+
state.clear_timer(
298+
window,
299+
timer_name,
300+
timer_spec.time_domain,
301+
dynamic_timer_tag=dynamic_timer_tag)
302+
if timer.timestamp:
303+
# TODO(ccy): add corresponding watermark holds after the DirectRunner
304+
# allows for keyed watermark holds.
305+
state.set_timer(
306+
window,
307+
timer_name,
308+
timer_spec.time_domain,
309+
timer.timestamp,
310+
dynamic_timer_tag=dynamic_timer_tag)
304311

305312
def reset(self):
306313
for state in self.cached_states.values():

sdks/python/apache_beam/runners/direct/transform_evaluator.py

Lines changed: 6 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -311,7 +311,10 @@ def process_timer_wrapper(self, timer_firing):
311311
"""
312312
state = self._step_context.get_keyed_state(timer_firing.encoded_key)
313313
state.clear_timer(
314-
timer_firing.window, timer_firing.name, timer_firing.time_domain)
314+
timer_firing.window,
315+
timer_firing.name,
316+
timer_firing.time_domain,
317+
dynamic_timer_tag=timer_firing.dynamic_timer_tag)
315318
self.process_timer(timer_firing)
316319

317320
def process_timer(self, timer_firing):
@@ -875,7 +878,8 @@ def process_timer(self, timer_firing):
875878
timer_firing.window,
876879
timer_firing.timestamp,
877880
# TODO Add paneinfo to timer_firing in DirectRunner
878-
None)
881+
None,
882+
timer_firing.dynamic_timer_tag)
879883

880884
def process_element(self, element):
881885
self.runner.process(element)

sdks/python/apache_beam/runners/direct/util.py

Lines changed: 15 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -61,16 +61,28 @@ def __init__(
6161

6262
class TimerFiring(object):
6363
"""A single instance of a fired timer."""
64-
def __init__(self, encoded_key, window, name, time_domain, timestamp):
64+
def __init__(
65+
self,
66+
encoded_key,
67+
window,
68+
name,
69+
time_domain,
70+
timestamp,
71+
dynamic_timer_tag=''):
6572
self.encoded_key = encoded_key
6673
self.window = window
6774
self.name = name
6875
self.time_domain = time_domain
6976
self.timestamp = timestamp
77+
self.dynamic_timer_tag = dynamic_timer_tag
7078

7179
def __repr__(self):
72-
return 'TimerFiring({!r}, {!r}, {}, {})'.format(
73-
self.encoded_key, self.name, self.time_domain, self.timestamp)
80+
return 'TimerFiring({!r}, {!r}, {}, {}, {})'.format(
81+
self.encoded_key,
82+
self.name,
83+
self.time_domain,
84+
self.timestamp,
85+
self.dynamic_timer_tag)
7486

7587

7688
class KeyedWorkItem(object):

sdks/python/apache_beam/runners/direct/watermark_manager.py

Lines changed: 8 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -325,8 +325,14 @@ def extract_transform_timers(self):
325325
if had_realtime_timer:
326326
has_realtime_timer = True
327327
for expired in timers:
328-
window, (name, time_domain, timestamp) = expired
328+
window, (name, time_domain, timestamp, dynamic_timer_tag) = expired
329329
fired_timers.append(
330-
TimerFiring(encoded_key, window, name, time_domain, timestamp))
330+
TimerFiring(
331+
encoded_key,
332+
window,
333+
name,
334+
time_domain,
335+
timestamp,
336+
dynamic_timer_tag=dynamic_timer_tag))
331337
self._fired_timers.update(fired_timers)
332338
return fired_timers, has_realtime_timer

sdks/python/apache_beam/runners/portability/flink_runner_test.py

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -395,6 +395,9 @@ def test_callbacks_with_exception(self):
395395
def test_register_finalizations(self):
396396
raise unittest.SkipTest("BEAM-11021")
397397

398+
def test_pardo_dynamic_timer(self):
399+
raise unittest.SkipTest("BEAM-10120")
400+
398401
# Inherits all other tests.
399402

400403

sdks/python/apache_beam/runners/portability/fn_api_runner/fn_runner_test.py

Lines changed: 24 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -493,6 +493,30 @@ def is_buffered_correctly(actual):
493493

494494
assert_that(actual, is_buffered_correctly)
495495

496+
def test_pardo_dynamic_timer(self):
497+
class DynamicTimerDoFn(beam.DoFn):
498+
dynamic_timer_spec = userstate.TimerSpec(
499+
'dynamic_timer', userstate.TimeDomain.WATERMARK)
500+
501+
def process(
502+
self, element,
503+
dynamic_timer=beam.DoFn.TimerParam(dynamic_timer_spec)):
504+
dynamic_timer.set(element[1], dynamic_timer_tag=element[0])
505+
506+
@userstate.on_timer(dynamic_timer_spec)
507+
def dynamic_timer_callback(
508+
self,
509+
tag=beam.DoFn.DynamicTimerTagParam,
510+
timestamp=beam.DoFn.TimestampParam):
511+
yield (tag, timestamp)
512+
513+
with self.create_pipeline() as p:
514+
actual = (
515+
p
516+
| beam.Create([('key1', 10), ('key2', 20), ('key3', 30)])
517+
| beam.ParDo(DynamicTimerDoFn()))
518+
assert_that(actual, equal_to([('key1', 10), ('key2', 20), ('key3', 30)]))
519+
496520
def test_sdf(self):
497521
class ExpandingStringsDoFn(beam.DoFn):
498522
def process(

sdks/python/apache_beam/runners/worker/bundle_processor.py

Lines changed: 17 additions & 24 deletions
Original file line numberDiff line numberDiff line change
@@ -88,8 +88,8 @@
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):
644644
class 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,

sdks/python/apache_beam/runners/worker/operations.py

Lines changed: 4 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -73,6 +73,7 @@
7373
from apache_beam.runners.sdf_utils import SplitResultResidual
7474
from apache_beam.runners.worker.bundle_processor import ExecutionContext
7575
from apache_beam.runners.worker.statesampler import StateSampler
76+
from apache_beam.transforms.userstate import TimerSpec
7677

7778
# Allow some "pure mode" declarations.
7879
try:
@@ -685,7 +686,7 @@ def setup(self):
685686
self.timer_specs = {
686687
spec.name: spec
687688
for spec in userstate.get_dofn_specs(fn)[1]
688-
}
689+
} # type: Dict[str, TimerSpec]
689690

690691
if self.side_input_maps is None:
691692
if tags_and_types:
@@ -741,7 +742,8 @@ def process_timer(self, tag, timer_data):
741742
timer_data.user_key,
742743
timer_data.windows[0],
743744
timer_data.fire_timestamp,
744-
timer_data.paneinfo)
745+
timer_data.paneinfo,
746+
timer_data.dynamic_timer_tag)
745747

746748
def finish(self):
747749
# type: () -> None

0 commit comments

Comments
 (0)