forked from apache/beam
-
Notifications
You must be signed in to change notification settings - Fork 2
/
Copy pathcommon.py
1916 lines (1658 loc) · 74.2 KB
/
common.py
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
#
# Licensed to the Apache Software Foundation (ASF) under one or more
# contributor license agreements. See the NOTICE file distributed with
# this work for additional information regarding copyright ownership.
# The ASF licenses this file to You under the Apache License, Version 2.0
# (the "License"); you may not use this file except in compliance with
# the License. You may obtain a copy of the License at
#
# http://www.apache.org/licenses/LICENSE-2.0
#
# Unless required by applicable law or agreed to in writing, software
# distributed under the License is distributed on an "AS IS" BASIS,
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
# See the License for the specific language governing permissions and
# limitations under the License.
#
"""Worker operations executor.
For internal use only; no backwards-compatibility guarantees.
"""
# pytype: skip-file
import logging
import sys
import threading
import traceback
from enum import Enum
from typing import TYPE_CHECKING
from typing import Any
from typing import Dict
from typing import Iterable
from typing import List
from typing import Mapping
from typing import Optional
from typing import Tuple
from apache_beam.coders import TupleCoder
from apache_beam.coders import coders
from apache_beam.internal import util
from apache_beam.options.value_provider import RuntimeValueProvider
from apache_beam.pvalue import TaggedOutput
from apache_beam.runners.sdf_utils import NoOpWatermarkEstimatorProvider
from apache_beam.runners.sdf_utils import RestrictionTrackerView
from apache_beam.runners.sdf_utils import SplitResultPrimary
from apache_beam.runners.sdf_utils import SplitResultResidual
from apache_beam.runners.sdf_utils import ThreadsafeRestrictionTracker
from apache_beam.runners.sdf_utils import ThreadsafeWatermarkEstimator
from apache_beam.transforms import DoFn
from apache_beam.transforms import core
from apache_beam.transforms import userstate
from apache_beam.transforms.core import RestrictionProvider
from apache_beam.transforms.core import WatermarkEstimatorProvider
from apache_beam.transforms.window import GlobalWindow
from apache_beam.transforms.window import GlobalWindows
from apache_beam.transforms.window import TimestampedValue
from apache_beam.transforms.window import WindowFn
from apache_beam.typehints.batch import BatchConverter
from apache_beam.utils.counters import Counter
from apache_beam.utils.counters import CounterName
from apache_beam.utils.timestamp import Timestamp
from apache_beam.utils.windowed_value import HomogeneousWindowedBatch
from apache_beam.utils.windowed_value import WindowedBatch
from apache_beam.utils.windowed_value import WindowedValue
if TYPE_CHECKING:
from apache_beam.runners.worker.bundle_processor import ExecutionContext
from apache_beam.transforms import sideinputs
from apache_beam.transforms.core import TimerSpec
from apache_beam.io.iobase import RestrictionProgress
from apache_beam.iobase import RestrictionTracker
from apache_beam.iobase import WatermarkEstimator
IMPULSE_VALUE_CODER_IMPL = coders.WindowedValueCoder(
coders.BytesCoder(), coders.GlobalWindowCoder()).get_impl()
ENCODED_IMPULSE_VALUE = IMPULSE_VALUE_CODER_IMPL.encode_nested(
GlobalWindows.windowed_value(b''))
_LOGGER = logging.getLogger(__name__)
class NameContext(object):
"""Holds the name information for a step."""
def __init__(self, step_name, transform_id=None):
# type: (str, Optional[str]) -> None
"""Creates a new step NameContext.
Args:
step_name: The name of the step.
"""
self.step_name = step_name
self.transform_id = transform_id
def __eq__(self, other):
return self.step_name == other.step_name
def __repr__(self):
return 'NameContext(%s)' % self.__dict__
def __hash__(self):
return hash(self.step_name)
def metrics_name(self):
"""Returns the step name used for metrics reporting."""
return self.step_name
def logging_name(self):
"""Returns the step name used for logging."""
return self.step_name
class Receiver(object):
"""For internal use only; no backwards-compatibility guarantees.
An object that consumes a WindowedValue.
This class can be efficiently used to pass values between the
sdk and worker harnesses.
"""
def receive(self, windowed_value):
# type: (WindowedValue) -> None
raise NotImplementedError
def receive_batch(self, windowed_batch):
# type: (WindowedBatch) -> None
raise NotImplementedError
def flush(self):
raise NotImplementedError
class MethodWrapper(object):
"""For internal use only; no backwards-compatibility guarantees.
Represents a method that can be invoked by `DoFnInvoker`."""
def __init__(self, obj_to_invoke, method_name):
"""
Initiates a ``MethodWrapper``.
Args:
obj_to_invoke: the object that contains the method. Has to either be a
`DoFn` object or a `RestrictionProvider` object.
method_name: name of the method as a string.
"""
if not isinstance(obj_to_invoke,
(DoFn, RestrictionProvider, WatermarkEstimatorProvider)):
raise ValueError(
'\'obj_to_invoke\' has to be either a \'DoFn\' or '
'a \'RestrictionProvider\'. Received %r instead.' % obj_to_invoke)
self.args, self.defaults = core.get_function_arguments(obj_to_invoke,
method_name)
# TODO(BEAM-5878) support kwonlyargs on Python 3.
self.method_value = getattr(obj_to_invoke, method_name)
self.method_name = method_name
self.has_userstate_arguments = False
self.state_args_to_replace = {} # type: Dict[str, core.StateSpec]
self.timer_args_to_replace = {} # type: Dict[str, core.TimerSpec]
self.timestamp_arg_name = None # type: Optional[str]
self.window_arg_name = None # type: Optional[str]
self.key_arg_name = None # type: Optional[str]
self.restriction_provider = None
self.restriction_provider_arg_name = None
self.watermark_estimator_provider = None
self.watermark_estimator_provider_arg_name = None
self.dynamic_timer_tag_arg_name = None
if hasattr(self.method_value, 'unbounded_per_element'):
self.unbounded_per_element = True
else:
self.unbounded_per_element = False
for kw, v in zip(self.args[-len(self.defaults):], self.defaults):
if isinstance(v, core.DoFn.StateParam):
self.state_args_to_replace[kw] = v.state_spec
self.has_userstate_arguments = True
elif isinstance(v, core.DoFn.TimerParam):
self.timer_args_to_replace[kw] = v.timer_spec
self.has_userstate_arguments = True
elif core.DoFn.TimestampParam == v:
self.timestamp_arg_name = kw
elif core.DoFn.WindowParam == v:
self.window_arg_name = kw
elif core.DoFn.WindowedValueParam == v:
self.window_arg_name = kw
elif core.DoFn.KeyParam == v:
self.key_arg_name = kw
elif isinstance(v, core.DoFn.RestrictionParam):
self.restriction_provider = v.restriction_provider or obj_to_invoke
self.restriction_provider_arg_name = kw
elif isinstance(v, core.DoFn.WatermarkEstimatorParam):
self.watermark_estimator_provider = (
v.watermark_estimator_provider or obj_to_invoke)
self.watermark_estimator_provider_arg_name = kw
elif core.DoFn.DynamicTimerTagParam == v:
self.dynamic_timer_tag_arg_name = kw
# Create NoOpWatermarkEstimatorProvider if there is no
# WatermarkEstimatorParam provided.
if self.watermark_estimator_provider is None:
self.watermark_estimator_provider = NoOpWatermarkEstimatorProvider()
def invoke_timer_callback(
self,
user_state_context,
key,
window,
timestamp,
pane_info,
dynamic_timer_tag):
# TODO(ccy): support side inputs.
kwargs = {}
if self.has_userstate_arguments:
for kw, state_spec in self.state_args_to_replace.items():
kwargs[kw] = user_state_context.get_state(state_spec, key, window)
for kw, timer_spec in self.timer_args_to_replace.items():
kwargs[kw] = user_state_context.get_timer(
timer_spec, key, window, timestamp, pane_info)
if self.timestamp_arg_name:
kwargs[self.timestamp_arg_name] = Timestamp.of(timestamp)
if self.window_arg_name:
kwargs[self.window_arg_name] = window
if self.key_arg_name:
kwargs[self.key_arg_name] = key
if self.dynamic_timer_tag_arg_name:
kwargs[self.dynamic_timer_tag_arg_name] = dynamic_timer_tag
if kwargs:
return self.method_value(**kwargs)
else:
return self.method_value()
class BatchingPreference(Enum):
DO_NOT_CARE = 1 # This operation can operate on batches or element-at-a-time
# TODO: Should we also store batching parameters here? (time/size preferences)
BATCH_REQUIRED = 2 # This operation can only operate on batches
BATCH_FORBIDDEN = 3 # This operation can only work element-at-a-time
# Other possibilities: BATCH_PREFERRED (with min batch size specified)
@property
def supports_batches(self) -> bool:
return self in (self.BATCH_REQUIRED, self.DO_NOT_CARE)
@property
def supports_elements(self) -> bool:
return self in (self.BATCH_FORBIDDEN, self.DO_NOT_CARE)
@property
def requires_batches(self) -> bool:
return self == self.BATCH_REQUIRED
class DoFnSignature(object):
"""Represents the signature of a given ``DoFn`` object.
Signature of a ``DoFn`` provides a view of the properties of a given ``DoFn``.
Among other things, this will give an extensible way for for (1) accessing the
structure of the ``DoFn`` including methods and method parameters
(2) identifying features that a given ``DoFn`` support, for example, whether
a given ``DoFn`` is a Splittable ``DoFn`` (
https://s.apache.org/splittable-do-fn) (3) validating a ``DoFn`` based on the
feature set offered by it.
"""
def __init__(self, do_fn):
# type: (core.DoFn) -> None
# We add a property here for all methods defined by Beam DoFn features.
assert isinstance(do_fn, core.DoFn)
self.do_fn = do_fn
self.process_method = MethodWrapper(do_fn, 'process')
self.process_batch_method = MethodWrapper(do_fn, 'process_batch')
self.start_bundle_method = MethodWrapper(do_fn, 'start_bundle')
self.finish_bundle_method = MethodWrapper(do_fn, 'finish_bundle')
self.setup_lifecycle_method = MethodWrapper(do_fn, 'setup')
self.teardown_lifecycle_method = MethodWrapper(do_fn, 'teardown')
restriction_provider = self.get_restriction_provider()
watermark_estimator_provider = self.get_watermark_estimator_provider()
self.create_watermark_estimator_method = (
MethodWrapper(
watermark_estimator_provider, 'create_watermark_estimator'))
self.initial_restriction_method = (
MethodWrapper(restriction_provider, 'initial_restriction')
if restriction_provider else None)
self.create_tracker_method = (
MethodWrapper(restriction_provider, 'create_tracker')
if restriction_provider else None)
self.split_method = (
MethodWrapper(restriction_provider, 'split')
if restriction_provider else None)
self._validate()
# Handle stateful DoFns.
self._is_stateful_dofn = userstate.is_stateful_dofn(do_fn)
self.timer_methods = {} # type: Dict[TimerSpec, MethodWrapper]
if self._is_stateful_dofn:
# Populate timer firing methods, keyed by TimerSpec.
_, all_timer_specs = userstate.get_dofn_specs(do_fn)
for timer_spec in all_timer_specs:
method = timer_spec._attached_callback
self.timer_methods[timer_spec] = MethodWrapper(do_fn, method.__name__)
def get_restriction_provider(self):
# type: () -> RestrictionProvider
return self.process_method.restriction_provider
def get_watermark_estimator_provider(self):
# type: () -> WatermarkEstimatorProvider
return self.process_method.watermark_estimator_provider
def is_unbounded_per_element(self):
return self.process_method.unbounded_per_element
def _validate(self):
# type: () -> None
self._validate_process()
self._validate_process_batch()
self._validate_bundle_method(self.start_bundle_method)
self._validate_bundle_method(self.finish_bundle_method)
self._validate_stateful_dofn()
def _check_duplicate_dofn_params(self, method: MethodWrapper):
param_ids = [
d.param_id for d in method.defaults if isinstance(d, core._DoFnParam)
]
if len(param_ids) != len(set(param_ids)):
raise ValueError(
'DoFn %r has duplicate %s method parameters: %s.' %
(self.do_fn, method.method_name, param_ids))
def _validate_process(self):
# type: () -> None
"""Validate that none of the DoFnParameters are repeated in the function
"""
self._check_duplicate_dofn_params(self.process_method)
def _validate_process_batch(self):
# type: () -> None
self._check_duplicate_dofn_params(self.process_batch_method)
for d in self.process_batch_method.defaults:
if not isinstance(d, core._DoFnParam):
continue
# Helpful errors for params which will be supported in the future
if d == (core.DoFn.ElementParam):
# We currently assume we can just get the typehint from the first
# parameter. ElementParam breaks this assumption
raise NotImplementedError(
f"DoFn {self.do_fn!r} uses unsupported DoFn param ElementParam.")
if d in (core.DoFn.KeyParam, core.DoFn.StateParam, core.DoFn.TimerParam):
raise NotImplementedError(
f"DoFn {self.do_fn!r} has unsupported per-key DoFn param {d}. "
"Per-key DoFn params are not yet supported for process_batch "
"(https://github.com/apache/beam/issues/21653).")
# Fallback to catch anything not explicitly supported
if not d in (core.DoFn.WindowParam,
core.DoFn.TimestampParam,
core.DoFn.PaneInfoParam):
raise ValueError(
f"DoFn {self.do_fn!r} has unsupported process_batch "
f"method parameter {d}")
def _validate_bundle_method(self, method_wrapper):
"""Validate that none of the DoFnParameters are used in the function
"""
for param in core.DoFn.DoFnProcessParams:
if param in method_wrapper.defaults:
raise ValueError(
'DoFn.process() method-only parameter %s cannot be used in %s.' %
(param, method_wrapper))
def _validate_stateful_dofn(self):
# type: () -> None
userstate.validate_stateful_dofn(self.do_fn)
def is_splittable_dofn(self):
# type: () -> bool
return self.get_restriction_provider() is not None
def get_restriction_coder(self):
# type: () -> Optional[TupleCoder]
"""Get coder for a restriction when processing an SDF. """
if self.is_splittable_dofn():
return TupleCoder([
(self.get_restriction_provider().restriction_coder()),
(self.get_watermark_estimator_provider().estimator_state_coder())
])
else:
return None
def is_stateful_dofn(self):
# type: () -> bool
return self._is_stateful_dofn
def has_timers(self):
# type: () -> bool
_, all_timer_specs = userstate.get_dofn_specs(self.do_fn)
return bool(all_timer_specs)
def has_bundle_finalization(self):
for sig in (self.start_bundle_method,
self.process_method,
self.finish_bundle_method):
for d in sig.defaults:
try:
if d == DoFn.BundleFinalizerParam:
return True
except Exception: # pylint: disable=broad-except
# Default value might be incomparable.
pass
return False
def get_bundle_contexts(self):
seen = set()
for sig in (self.setup_lifecycle_method,
self.start_bundle_method,
self.process_method,
self.process_batch_method,
self.finish_bundle_method,
self.teardown_lifecycle_method):
for d in sig.defaults:
try:
if isinstance(d, DoFn.BundleContextParam):
if d not in seen:
seen.add(d)
yield d
except Exception: # pylint: disable=broad-except
# Default value might be incomparable.
pass
def get_setup_contexts(self):
seen = set()
for sig in (self.setup_lifecycle_method,
self.start_bundle_method,
self.process_method,
self.process_batch_method,
self.finish_bundle_method,
self.teardown_lifecycle_method):
for d in sig.defaults:
try:
if isinstance(d, DoFn.SetupContextParam):
if d not in seen:
seen.add(d)
yield d
except Exception: # pylint: disable=broad-except
# Default value might be incomparable.
pass
class DoFnInvoker(object):
"""An abstraction that can be used to execute DoFn methods.
A DoFnInvoker describes a particular way for invoking methods of a DoFn
represented by a given DoFnSignature."""
def __init__(self,
output_handler, # type: _OutputHandler
signature # type: DoFnSignature
):
# type: (...) -> None
"""
Initializes `DoFnInvoker`
:param output_handler: an OutputHandler for receiving elements produced
by invoking functions of the DoFn.
:param signature: a DoFnSignature for the DoFn being invoked
"""
self.output_handler = output_handler
self.signature = signature
self.user_state_context = None # type: Optional[userstate.UserStateContext]
self.bundle_finalizer_param = None # type: Optional[core._BundleFinalizerParam]
@staticmethod
def create_invoker(
signature, # type: DoFnSignature
output_handler, # type: OutputHandler
context=None, # type: Optional[DoFnContext]
side_inputs=None, # type: Optional[List[sideinputs.SideInputMap]]
input_args=None, input_kwargs=None,
process_invocation=True,
user_state_context=None, # type: Optional[userstate.UserStateContext]
bundle_finalizer_param=None # type: Optional[core._BundleFinalizerParam]
):
# type: (...) -> DoFnInvoker
""" Creates a new DoFnInvoker based on given arguments.
Args:
output_handler: an OutputHandler for receiving elements produced by
invoking functions of the DoFn.
signature: a DoFnSignature for the DoFn being invoked.
context: Context to be used when invoking the DoFn (deprecated).
side_inputs: side inputs to be used when invoking th process method.
input_args: arguments to be used when invoking the process method. Some
of the arguments given here might be placeholders (for
example for side inputs) that get filled before invoking the
process method.
input_kwargs: keyword arguments to be used when invoking the process
method. Some of the keyword arguments given here might be
placeholders (for example for side inputs) that get filled
before invoking the process method.
process_invocation: If True, this function may return an invoker that
performs extra optimizations for invoking process()
method efficiently.
user_state_context: The UserStateContext instance for the current
Stateful DoFn.
bundle_finalizer_param: The param that passed to a process method, which
allows a callback to be registered.
"""
side_inputs = side_inputs or []
use_per_window_invoker = process_invocation and (
side_inputs or input_args or input_kwargs or
signature.process_method.defaults or
signature.process_batch_method.defaults or signature.is_stateful_dofn())
if not use_per_window_invoker:
return SimpleInvoker(output_handler, signature)
else:
if context is None:
raise TypeError("Must provide context when not using SimpleInvoker")
return PerWindowInvoker(
output_handler,
signature,
context,
side_inputs,
input_args,
input_kwargs,
user_state_context,
bundle_finalizer_param)
def invoke_process(self,
windowed_value, # type: WindowedValue
restriction=None,
watermark_estimator_state=None,
additional_args=None,
additional_kwargs=None
):
# type: (...) -> Iterable[SplitResultResidual]
"""Invokes the DoFn.process() function.
Args:
windowed_value: a WindowedValue object that gives the element for which
process() method should be invoked along with the window
the element belongs to.
restriction: The restriction to use when executing this splittable DoFn.
Should only be specified for splittable DoFns.
watermark_estimator_state: The watermark estimator state to use when
executing this splittable DoFn. Should only
be specified for splittable DoFns.
additional_args: additional arguments to be passed to the current
`DoFn.process()` invocation, usually as side inputs.
additional_kwargs: additional keyword arguments to be passed to the
current `DoFn.process()` invocation.
"""
raise NotImplementedError
def invoke_process_batch(self,
windowed_batch, # type: WindowedBatch
additional_args=None,
additional_kwargs=None
):
# type: (...) -> None
"""Invokes the DoFn.process() function.
Args:
windowed_batch: a WindowedBatch object that gives a batch of elements for
which process_batch() method should be invoked, along with
the window each element belongs to.
additional_args: additional arguments to be passed to the current
`DoFn.process()` invocation, usually as side inputs.
additional_kwargs: additional keyword arguments to be passed to the
current `DoFn.process()` invocation.
"""
raise NotImplementedError
def invoke_setup(self):
# type: () -> None
"""Invokes the DoFn.setup() method
"""
self._setup_context_values = {
c: c.create_and_enter()
for c in self.signature.get_setup_contexts()
}
self.signature.setup_lifecycle_method.method_value()
def invoke_start_bundle(self):
# type: () -> None
"""Invokes the DoFn.start_bundle() method.
"""
self._bundle_context_values = {
c: c.create_and_enter()
for c in self.signature.get_bundle_contexts()
}
self.output_handler.start_bundle_outputs(
self.signature.start_bundle_method.method_value())
def invoke_finish_bundle(self):
# type: () -> None
"""Invokes the DoFn.finish_bundle() method.
"""
self.output_handler.finish_bundle_outputs(
self.signature.finish_bundle_method.method_value())
for c in self._bundle_context_values.values():
c[0].__exit__(None, None, None)
self._bundle_context_values = None
def invoke_teardown(self):
# type: () -> None
"""Invokes the DoFn.teardown() method
"""
self.signature.teardown_lifecycle_method.method_value()
for c in self._setup_context_values.values():
c[0].__exit__(None, None, None)
self._setup_context_values = None
def invoke_user_timer(
self, timer_spec, key, window, timestamp, pane_info, dynamic_timer_tag):
# self.output_handler is Optional, but in practice it won't be None here
self.output_handler.handle_process_outputs(
WindowedValue(None, timestamp, (window, )),
self.signature.timer_methods[timer_spec].invoke_timer_callback(
self.user_state_context,
key,
window,
timestamp,
pane_info,
dynamic_timer_tag))
def invoke_create_watermark_estimator(self, estimator_state):
return self.signature.create_watermark_estimator_method.method_value(
estimator_state)
def invoke_split(self, element, restriction):
return self.signature.split_method.method_value(element, restriction)
def invoke_initial_restriction(self, element):
return self.signature.initial_restriction_method.method_value(element)
def invoke_create_tracker(self, restriction):
return self.signature.create_tracker_method.method_value(restriction)
class SimpleInvoker(DoFnInvoker):
"""An invoker that processes elements ignoring windowing information."""
def __init__(self,
output_handler, # type: OutputHandler
signature # type: DoFnSignature
):
# type: (...) -> None
super().__init__(output_handler, signature)
self.process_method = signature.process_method.method_value
self.process_batch_method = signature.process_batch_method.method_value
def invoke_process(self,
windowed_value, # type: WindowedValue
restriction=None,
watermark_estimator_state=None,
additional_args=None,
additional_kwargs=None
):
# type: (...) -> Iterable[SplitResultResidual]
self.output_handler.handle_process_outputs(
windowed_value, self.process_method(windowed_value.value))
return []
def invoke_process_batch(self,
windowed_batch, # type: WindowedBatch
restriction=None,
watermark_estimator_state=None,
additional_args=None,
additional_kwargs=None
):
# type: (...) -> None
self.output_handler.handle_process_batch_outputs(
windowed_batch, self.process_batch_method(windowed_batch.values))
def _get_arg_placeholders(
method: MethodWrapper,
input_args: Optional[List[Any]],
input_kwargs: Optional[Dict[str, any]]):
input_args = input_args if input_args else []
input_kwargs = input_kwargs if input_kwargs else {}
arg_names = method.args
default_arg_values = method.defaults
# Create placeholder for element parameter of DoFn.process() method.
# Not to be confused with ArgumentPlaceHolder, which may be passed in
# input_args and is a placeholder for side-inputs.
class ArgPlaceholder(object):
def __init__(self, placeholder):
self.placeholder = placeholder
if all(core.DoFn.ElementParam != arg for arg in default_arg_values):
# TODO(https://github.com/apache/beam/issues/19631): Handle cases in which
# len(arg_names) == len(default_arg_values).
args_to_pick = len(arg_names) - len(default_arg_values) - 1
# Positional argument values for process(), with placeholders for special
# values such as the element, timestamp, etc.
args_with_placeholders = ([ArgPlaceholder(core.DoFn.ElementParam)] +
input_args[:args_to_pick])
else:
args_to_pick = len(arg_names) - len(default_arg_values)
args_with_placeholders = input_args[:args_to_pick]
# Fill the OtherPlaceholders for context, key, window or timestamp
remaining_args_iter = iter(input_args[args_to_pick:])
for a, d in zip(arg_names[-len(default_arg_values):], default_arg_values):
if core.DoFn.ElementParam == d:
args_with_placeholders.append(ArgPlaceholder(d))
elif core.DoFn.KeyParam == d:
args_with_placeholders.append(ArgPlaceholder(d))
elif core.DoFn.WindowParam == d:
args_with_placeholders.append(ArgPlaceholder(d))
elif core.DoFn.WindowedValueParam == d:
args_with_placeholders.append(ArgPlaceholder(d))
elif core.DoFn.TimestampParam == d:
args_with_placeholders.append(ArgPlaceholder(d))
elif core.DoFn.PaneInfoParam == d:
args_with_placeholders.append(ArgPlaceholder(d))
elif core.DoFn.SideInputParam == d:
# If no more args are present then the value must be passed via kwarg
try:
args_with_placeholders.append(next(remaining_args_iter))
except StopIteration:
if a not in input_kwargs:
raise ValueError("Value for sideinput %s not provided" % a)
elif isinstance(d, core.DoFn.StateParam):
args_with_placeholders.append(ArgPlaceholder(d))
elif isinstance(d, core.DoFn.TimerParam):
args_with_placeholders.append(ArgPlaceholder(d))
elif isinstance(d, type) and core.DoFn.BundleFinalizerParam == d:
args_with_placeholders.append(ArgPlaceholder(d))
elif isinstance(d, core.DoFn.BundleContextParam):
args_with_placeholders.append(ArgPlaceholder(d))
elif isinstance(d, core.DoFn.SetupContextParam):
args_with_placeholders.append(ArgPlaceholder(d))
else:
# If no more args are present then the value must be passed via kwarg
try:
args_with_placeholders.append(next(remaining_args_iter))
except StopIteration:
pass
args_with_placeholders.extend(list(remaining_args_iter))
# Stash the list of placeholder positions for performance
placeholders = [(i, x.placeholder)
for (i, x) in enumerate(args_with_placeholders)
if isinstance(x, ArgPlaceholder)]
return placeholders, args_with_placeholders, input_kwargs
class PerWindowInvoker(DoFnInvoker):
"""An invoker that processes elements considering windowing information."""
def __init__(self,
output_handler, # type: OutputHandler
signature, # type: DoFnSignature
context, # type: DoFnContext
side_inputs, # type: Iterable[sideinputs.SideInputMap]
input_args,
input_kwargs,
user_state_context, # type: Optional[userstate.UserStateContext]
bundle_finalizer_param # type: Optional[core._BundleFinalizerParam]
):
super().__init__(output_handler, signature)
self.side_inputs = side_inputs
self.context = context
self.process_method = signature.process_method.method_value
default_arg_values = signature.process_method.defaults
self.has_windowed_inputs = (
not all(si.is_globally_windowed() for si in side_inputs) or any(
core.DoFn.WindowParam == arg
for arg in signature.process_method.defaults) or any(
core.DoFn.WindowParam == arg
for arg in signature.process_batch_method.defaults) or
signature.is_stateful_dofn())
self.user_state_context = user_state_context
self.is_splittable = signature.is_splittable_dofn()
self.is_key_param_required = any(
core.DoFn.KeyParam == arg for arg in default_arg_values)
self.threadsafe_restriction_tracker = None # type: Optional[ThreadsafeRestrictionTracker]
self.threadsafe_watermark_estimator = None # type: Optional[ThreadsafeWatermarkEstimator]
self.current_windowed_value = None # type: Optional[WindowedValue]
self.bundle_finalizer_param = bundle_finalizer_param
if self.is_splittable:
self.splitting_lock = threading.Lock()
self.current_window_index = None
self.stop_window_index = None
# TODO(https://github.com/apache/beam/issues/28776): Remove caching after
# fully rolling out.
# If true, always recalculate window args. If false, has_cached_window_args
# and has_cached_window_batch_args will be set to true if the corresponding
# self.args_for_process,have been updated and should be reused directly.
self.recalculate_window_args = (
self.has_windowed_inputs or 'disable_global_windowed_args_caching' in
RuntimeValueProvider.experiments)
self.has_cached_window_args = False
self.has_cached_window_batch_args = False
# Try to prepare all the arguments that can just be filled in
# without any additional work. in the process function.
# Also cache all the placeholders needed in the process function.
input_args = list(input_args)
(
self.placeholders_for_process,
self.args_for_process,
self.kwargs_for_process) = _get_arg_placeholders(
signature.process_method, input_args, input_kwargs)
self.process_batch_method = signature.process_batch_method.method_value
(
self.placeholders_for_process_batch,
self.args_for_process_batch,
self.kwargs_for_process_batch) = _get_arg_placeholders(
signature.process_batch_method, input_args, input_kwargs)
def invoke_process(self,
windowed_value, # type: WindowedValue
restriction=None,
watermark_estimator_state=None,
additional_args=None,
additional_kwargs=None
):
# type: (...) -> Iterable[SplitResultResidual]
if not additional_args:
additional_args = []
if not additional_kwargs:
additional_kwargs = {}
self.context.set_element(windowed_value)
# Call for the process function for each window if has windowed side inputs
# or if the process accesses the window parameter. We can just call it once
# otherwise as none of the arguments are changing
residuals = []
if self.is_splittable:
if restriction is None:
# This may be a SDF invoked as an ordinary DoFn on runners that don't
# understand SDF. See, e.g. BEAM-11472.
# In this case, processing the element is simply processing it against
# the entire initial restriction.
restriction = self.signature.initial_restriction_method.method_value(
windowed_value.value)
with self.splitting_lock:
self.current_windowed_value = windowed_value
self.restriction = restriction
self.watermark_estimator_state = watermark_estimator_state
try:
if self.has_windowed_inputs and len(windowed_value.windows) > 1:
for i, w in enumerate(windowed_value.windows):
if not self._should_process_window_for_sdf(
windowed_value, additional_kwargs, i):
break
residual = self._invoke_process_per_window(
WindowedValue(
windowed_value.value, windowed_value.timestamp, (w, )),
additional_args,
additional_kwargs)
if residual:
residuals.append(residual)
else:
if self._should_process_window_for_sdf(windowed_value,
additional_kwargs):
residual = self._invoke_process_per_window(
windowed_value, additional_args, additional_kwargs)
if residual:
residuals.append(residual)
finally:
with self.splitting_lock:
self.current_windowed_value = None
self.restriction = None
self.watermark_estimator_state = None
self.current_window_index = None
self.threadsafe_restriction_tracker = None
self.threadsafe_watermark_estimator = None
elif self.has_windowed_inputs and len(windowed_value.windows) != 1:
for w in windowed_value.windows:
self._invoke_process_per_window(
WindowedValue(
windowed_value.value, windowed_value.timestamp, (w, )),
additional_args,
additional_kwargs)
else:
self._invoke_process_per_window(
windowed_value, additional_args, additional_kwargs)
return residuals
def invoke_process_batch(self,
windowed_batch, # type: WindowedBatch
additional_args=None,
additional_kwargs=None
):
# type: (...) -> None
if not additional_args:
additional_args = []
if not additional_kwargs:
additional_kwargs = {}
assert isinstance(windowed_batch, HomogeneousWindowedBatch)
if self.has_windowed_inputs and len(windowed_batch.windows) != 1:
for w in windowed_batch.windows:
self._invoke_process_batch_per_window(
HomogeneousWindowedBatch.of(
windowed_batch.values,
windowed_batch.timestamp, (w, ),
windowed_batch.pane_info),
additional_args,
additional_kwargs)
else:
self._invoke_process_batch_per_window(
windowed_batch, additional_args, additional_kwargs)
def _should_process_window_for_sdf(
self,
windowed_value, # type: WindowedValue
additional_kwargs,
window_index=None, # type: Optional[int]
):
restriction_tracker = self.invoke_create_tracker(self.restriction)
watermark_estimator = self.invoke_create_watermark_estimator(
self.watermark_estimator_state)
with self.splitting_lock:
if window_index:
self.current_window_index = window_index
if window_index == 0:
self.stop_window_index = len(windowed_value.windows)
if window_index == self.stop_window_index:
return False
self.threadsafe_restriction_tracker = ThreadsafeRestrictionTracker(
restriction_tracker)
self.threadsafe_watermark_estimator = (
ThreadsafeWatermarkEstimator(watermark_estimator))
restriction_tracker_param = (
self.signature.process_method.restriction_provider_arg_name)
if not restriction_tracker_param:
raise ValueError(
'DoFn is splittable but DoFn does not have a '
'RestrictionTrackerParam defined')
additional_kwargs[restriction_tracker_param] = (
RestrictionTrackerView(self.threadsafe_restriction_tracker))
watermark_param = (
self.signature.process_method.watermark_estimator_provider_arg_name)
# When the watermark_estimator is a NoOpWatermarkEstimator, the system
# will not add watermark_param into the DoFn param list.
if watermark_param is not None:
additional_kwargs[watermark_param] = self.threadsafe_watermark_estimator
return True
def _invoke_process_per_window(self,
windowed_value, # type: WindowedValue
additional_args,
additional_kwargs,
):
# type: (...) -> Optional[SplitResultResidual]
if self.has_cached_window_args:
args_for_process, kwargs_for_process = (
self.args_for_process, self.kwargs_for_process)
else:
if self.has_windowed_inputs:
assert len(windowed_value.windows) <= 1
window, = windowed_value.windows
else:
window = GlobalWindow()
side_inputs = [si[window] for si in self.side_inputs]
side_inputs.extend(additional_args)
args_for_process, kwargs_for_process = util.insert_values_in_args(
self.args_for_process, self.kwargs_for_process, side_inputs)
if not self.recalculate_window_args:
self.args_for_process, self.kwargs_for_process = (
args_for_process, kwargs_for_process)