-
Notifications
You must be signed in to change notification settings - Fork 4.3k
/
core.py
3644 lines (3011 loc) · 131 KB
/
core.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.
#
"""Core PTransform subclasses, such as FlatMap, GroupByKey, and Map."""
# pytype: skip-file
import concurrent.futures
import copy
import inspect
import logging
import random
import sys
import traceback
import types
import typing
from itertools import dropwhile
from apache_beam import coders
from apache_beam import pvalue
from apache_beam import typehints
from apache_beam.coders import typecoders
from apache_beam.internal import pickler
from apache_beam.internal import util
from apache_beam.options.pipeline_options import TypeOptions
from apache_beam.portability import common_urns
from apache_beam.portability import python_urns
from apache_beam.portability.api import beam_runner_api_pb2
from apache_beam.transforms import ptransform
from apache_beam.transforms import userstate
from apache_beam.transforms.display import DisplayDataItem
from apache_beam.transforms.display import HasDisplayData
from apache_beam.transforms.ptransform import PTransform
from apache_beam.transforms.ptransform import PTransformWithSideInputs
from apache_beam.transforms.sideinputs import SIDE_INPUT_PREFIX
from apache_beam.transforms.sideinputs import get_sideinput_index
from apache_beam.transforms.userstate import StateSpec
from apache_beam.transforms.userstate import TimerSpec
from apache_beam.transforms.window import GlobalWindows
from apache_beam.transforms.window import SlidingWindows
from apache_beam.transforms.window import TimestampCombiner
from apache_beam.transforms.window import TimestampedValue
from apache_beam.transforms.window import WindowedValue
from apache_beam.transforms.window import WindowFn
from apache_beam.typehints import row_type
from apache_beam.typehints import trivial_inference
from apache_beam.typehints.batch import BatchConverter
from apache_beam.typehints.decorators import TypeCheckError
from apache_beam.typehints.decorators import WithTypeHints
from apache_beam.typehints.decorators import get_signature
from apache_beam.typehints.decorators import get_type_hints
from apache_beam.typehints.decorators import with_input_types
from apache_beam.typehints.decorators import with_output_types
from apache_beam.typehints.trivial_inference import element_type
from apache_beam.typehints.typehints import TypeConstraint
from apache_beam.typehints.typehints import is_consistent_with
from apache_beam.typehints.typehints import visit_inner_types
from apache_beam.utils import urns
from apache_beam.utils.timestamp import Duration
if typing.TYPE_CHECKING:
from google.protobuf import message # pylint: disable=ungrouped-imports
from apache_beam.io import iobase
from apache_beam.pipeline import Pipeline
from apache_beam.runners.pipeline_context import PipelineContext
from apache_beam.transforms import create_source
from apache_beam.transforms.trigger import AccumulationMode
from apache_beam.transforms.trigger import DefaultTrigger
from apache_beam.transforms.trigger import TriggerFn
__all__ = [
'DoFn',
'CombineFn',
'PartitionFn',
'ParDo',
'FlatMap',
'FlatMapTuple',
'Map',
'MapTuple',
'Filter',
'CombineGlobally',
'CombinePerKey',
'CombineValues',
'GroupBy',
'GroupByKey',
'Select',
'Partition',
'Windowing',
'WindowInto',
'Flatten',
'Create',
'Impulse',
'RestrictionProvider',
'WatermarkEstimatorProvider',
]
# Type variables
T = typing.TypeVar('T')
K = typing.TypeVar('K')
V = typing.TypeVar('V')
_LOGGER = logging.getLogger(__name__)
class DoFnContext(object):
"""A context available to all methods of DoFn instance."""
pass
class DoFnProcessContext(DoFnContext):
"""A processing context passed to DoFn process() during execution.
Experimental; no backwards-compatibility guarantees.
Most importantly, a DoFn.process method will access context.element
to get the element it is supposed to process.
Attributes:
label: label of the ParDo whose element is being processed.
element: element being processed
(in process method only; always None in start_bundle and finish_bundle)
timestamp: timestamp of the element
(in process method only; always None in start_bundle and finish_bundle)
windows: windows of the element
(in process method only; always None in start_bundle and finish_bundle)
state: a DoFnState object, which holds the runner's internal state
for this element.
Not used by the pipeline code.
"""
def __init__(self, label, element=None, state=None):
"""Initialize a processing context object with an element and state.
The element represents one value from a PCollection that will be accessed
by a DoFn object during pipeline execution, and state is an arbitrary object
where counters and other pipeline state information can be passed in.
DoFnProcessContext objects are also used as inputs to PartitionFn instances.
Args:
label: label of the PCollection whose element is being processed.
element: element of a PCollection being processed using this context.
state: a DoFnState object with state to be passed in to the DoFn object.
"""
self.label = label
self.state = state
if element is not None:
self.set_element(element)
def set_element(self, windowed_value):
if windowed_value is None:
# Not currently processing an element.
if hasattr(self, 'element'):
del self.element
del self.timestamp
del self.windows
else:
self.element = windowed_value.value
self.timestamp = windowed_value.timestamp
self.windows = windowed_value.windows
class ProcessContinuation(object):
"""An object that may be produced as the last element of a process method
invocation.
Experimental; no backwards-compatibility guarantees.
If produced, indicates that there is more work to be done for the current
input element.
"""
def __init__(self, resume_delay=0):
"""Initializes a ProcessContinuation object.
Args:
resume_delay: indicates the minimum time, in seconds, that should elapse
before re-invoking process() method for resuming the invocation of the
current element.
"""
self.resume_delay = resume_delay
@staticmethod
def resume(resume_delay=0):
"""A convenient method that produces a ``ProcessContinuation``.
Args:
resume_delay: delay after which processing current element should be
resumed.
Returns: a ``ProcessContinuation`` for signalling the runner that current
input element has not been fully processed and should be resumed later.
"""
return ProcessContinuation(resume_delay=resume_delay)
class RestrictionProvider(object):
"""Provides methods for generating and manipulating restrictions.
This class should be implemented to support Splittable ``DoFn`` in Python
SDK. See https://s.apache.org/splittable-do-fn for more details about
Splittable ``DoFn``.
To denote a ``DoFn`` class to be Splittable ``DoFn``, ``DoFn.process()``
method of that class should have exactly one parameter whose default value is
an instance of ``RestrictionParam``. This ``RestrictionParam`` can either be
constructed with an explicit ``RestrictionProvider``, or, if no
``RestrictionProvider`` is provided, the ``DoFn`` itself must be a
``RestrictionProvider``.
The provided ``RestrictionProvider`` instance must provide suitable overrides
for the following methods:
* create_tracker()
* initial_restriction()
* restriction_size()
Optionally, ``RestrictionProvider`` may override default implementations of
following methods:
* restriction_coder()
* split()
* split_and_size()
* truncate()
** Pausing and resuming processing of an element **
As the last element produced by the iterator returned by the
``DoFn.process()`` method, a Splittable ``DoFn`` may return an object of type
``ProcessContinuation``.
If restriction_tracker.defer_remander is called in the ```DoFn.process()``, it
means that runner should later re-invoke ``DoFn.process()`` method to resume
processing the current element and the manner in which the re-invocation
should be performed.
** Updating output watermark **
``DoFn.process()`` method of Splittable ``DoFn``s could contain a parameter
with default value ``DoFn.WatermarkReporterParam``. If specified this asks the
runner to provide a function that can be used to give the runner a
(best-effort) lower bound about the timestamps of future output associated
with the current element processed by the ``DoFn``. If the ``DoFn`` has
multiple outputs, the watermark applies to all of them. Provided function must
be invoked with a single parameter of type ``Timestamp`` or as an integer that
gives the watermark in number of seconds.
"""
def create_tracker(self, restriction):
# type: (...) -> iobase.RestrictionTracker
"""Produces a new ``RestrictionTracker`` for the given restriction.
This API is required to be implemented.
Args:
restriction: an object that defines a restriction as identified by a
Splittable ``DoFn`` that utilizes the current ``RestrictionProvider``.
For example, a tuple that gives a range of positions for a Splittable
``DoFn`` that reads files based on byte positions.
Returns: an object of type ``RestrictionTracker``.
"""
raise NotImplementedError
def initial_restriction(self, element):
"""Produces an initial restriction for the given element.
This API is required to be implemented.
"""
raise NotImplementedError
def split(self, element, restriction):
"""Splits the given element and restriction initially.
This method enables runners to perform bulk splitting initially allowing for
a rapid increase in parallelism. Note that initial split is a different
concept from the split during element processing time. Please refer to
``iobase.RestrictionTracker.try_split`` for details about splitting when the
current element and restriction are actively being processed.
Returns an iterator of restrictions. The total set of elements produced by
reading input element for each of the returned restrictions should be the
same as the total set of elements produced by reading the input element for
the input restriction.
This API is optional if ``split_and_size`` has been implemented.
If this method is not override, there is no initial splitting happening on
each restriction.
"""
yield restriction
def restriction_coder(self):
"""Returns a ``Coder`` for restrictions.
Returned``Coder`` will be used for the restrictions produced by the current
``RestrictionProvider``.
Returns:
an object of type ``Coder``.
"""
return coders.registry.get_coder(object)
def restriction_size(self, element, restriction):
"""Returns the size of a restriction with respect to the given element.
By default, asks a newly-created restriction tracker for the default size
of the restriction.
The return value must be non-negative.
Must be thread safe. Will be invoked concurrently during bundle processing
due to runner initiated splitting and progress estimation.
This API is required to be implemented.
"""
raise NotImplementedError
def split_and_size(self, element, restriction):
"""Like split, but also does sizing, returning (restriction, size) pairs.
For each pair, size must be non-negative.
This API is optional if ``split`` and ``restriction_size`` have been
implemented.
"""
for part in self.split(element, restriction):
yield part, self.restriction_size(element, part)
def truncate(self, element, restriction):
"""Truncates the provided restriction into a restriction representing a
finite amount of work when the pipeline is
`draining <https://docs.google.com/document/d/1NExwHlj-2q2WUGhSO4jTu8XGhDPmm3cllSN8IMmWci8/edit#> for additional details about drain.>_`. # pylint: disable=line-too-long
By default, if the restriction is bounded then the restriction will be
returned otherwise None will be returned.
This API is optional and should only be implemented if more granularity is
required.
Return a truncated finite restriction if further processing is required
otherwise return None to represent that no further processing of this
restriction is required.
The default behavior when a pipeline is being drained is that bounded
restrictions process entirely while unbounded restrictions process till a
checkpoint is possible.
"""
restriction_tracker = self.create_tracker(restriction)
if restriction_tracker.is_bounded():
return restriction
def get_function_arguments(obj, func):
# type: (...) -> typing.Tuple[typing.List[str], typing.List[typing.Any]]
"""Return the function arguments based on the name provided. If they have
a _inspect_function attached to the class then use that otherwise default
to the modified version of python inspect library.
Returns:
Same as get_function_args_defaults.
"""
func_name = '_inspect_%s' % func
if hasattr(obj, func_name):
f = getattr(obj, func_name)
return f()
f = getattr(obj, func)
return get_function_args_defaults(f)
def get_function_args_defaults(f):
# type: (...) -> typing.Tuple[typing.List[str], typing.List[typing.Any]]
"""Returns the function arguments of a given function.
Returns:
(args: List[str], defaults: List[Any]). The first list names the
arguments of the method and the second one has the values of the default
arguments. This is similar to ``inspect.getfullargspec()``'s results, except
it doesn't include bound arguments and may follow function wrappers.
"""
signature = get_signature(f)
parameter = inspect.Parameter
# TODO(BEAM-5878) support kwonlyargs on Python 3.
_SUPPORTED_ARG_TYPES = [
parameter.POSITIONAL_ONLY, parameter.POSITIONAL_OR_KEYWORD
]
args = [
name for name,
p in signature.parameters.items() if p.kind in _SUPPORTED_ARG_TYPES
]
defaults = [
p.default for p in signature.parameters.values()
if p.kind in _SUPPORTED_ARG_TYPES and p.default is not p.empty
]
return args, defaults
class WatermarkEstimatorProvider(object):
"""Provides methods for generating WatermarkEstimator.
This class should be implemented if wanting to providing output_watermark
information within an SDF.
In order to make an SDF.process() access to the typical WatermarkEstimator,
the SDF author should have an argument whose default value is a
DoFn.WatermarkEstimatorParam instance. This DoFn.WatermarkEstimatorParam
can either be constructed with an explicit WatermarkEstimatorProvider,
or, if no WatermarkEstimatorProvider is provided, the DoFn itself must
be a WatermarkEstimatorProvider.
"""
def initial_estimator_state(self, element, restriction):
"""Returns the initial state of the WatermarkEstimator with given element
and restriction.
This function is called by the system.
"""
raise NotImplementedError
def create_watermark_estimator(self, estimator_state):
"""Create a new WatermarkEstimator based on the state. The state is
typically useful when resuming processing an element.
"""
raise NotImplementedError
def estimator_state_coder(self):
return coders.registry.get_coder(object)
class _DoFnParam(object):
"""DoFn parameter."""
def __init__(self, param_id):
self.param_id = param_id
def __eq__(self, other):
if type(self) == type(other):
return self.param_id == other.param_id
return False
def __hash__(self):
return hash(self.param_id)
def __repr__(self):
return self.param_id
class _RestrictionDoFnParam(_DoFnParam):
"""Restriction Provider DoFn parameter."""
def __init__(self, restriction_provider=None):
# type: (typing.Optional[RestrictionProvider]) -> None
if (restriction_provider is not None and
not isinstance(restriction_provider, RestrictionProvider)):
raise ValueError(
'DoFn.RestrictionParam expected RestrictionProvider object.')
self.restriction_provider = restriction_provider
self.param_id = (
'RestrictionParam(%s)' % restriction_provider.__class__.__name__)
class _StateDoFnParam(_DoFnParam):
"""State DoFn parameter."""
def __init__(self, state_spec):
# type: (StateSpec) -> None
if not isinstance(state_spec, StateSpec):
raise ValueError("DoFn.StateParam expected StateSpec object.")
self.state_spec = state_spec
self.param_id = 'StateParam(%s)' % state_spec.name
class _TimerDoFnParam(_DoFnParam):
"""Timer DoFn parameter."""
def __init__(self, timer_spec):
# type: (TimerSpec) -> None
if not isinstance(timer_spec, TimerSpec):
raise ValueError("DoFn.TimerParam expected TimerSpec object.")
self.timer_spec = timer_spec
self.param_id = 'TimerParam(%s)' % timer_spec.name
class _BundleFinalizerParam(_DoFnParam):
"""Bundle Finalization DoFn parameter."""
def __init__(self):
self._callbacks = []
self.param_id = "FinalizeBundle"
def register(self, callback):
self._callbacks.append(callback)
# Log errors when calling callback to make sure all callbacks get called
# though there are errors. And errors should not fail pipeline.
def finalize_bundle(self):
for callback in self._callbacks:
try:
callback()
except Exception as e:
_LOGGER.warning("Got exception from finalization call: %s", e)
def has_callbacks(self):
# type: () -> bool
return len(self._callbacks) > 0
def reset(self):
# type: () -> None
del self._callbacks[:]
class _WatermarkEstimatorParam(_DoFnParam):
"""WatermarkEstimator DoFn parameter."""
def __init__(
self,
watermark_estimator_provider: typing.
Optional[WatermarkEstimatorProvider] = None):
if (watermark_estimator_provider is not None and not isinstance(
watermark_estimator_provider, WatermarkEstimatorProvider)):
raise ValueError(
'DoFn.WatermarkEstimatorParam expected'
'WatermarkEstimatorProvider object.')
self.watermark_estimator_provider = watermark_estimator_provider
self.param_id = 'WatermarkEstimatorProvider'
class DoFn(WithTypeHints, HasDisplayData, urns.RunnerApiFn):
"""A function object used by a transform with custom processing.
The ParDo transform is such a transform. The ParDo.apply
method will take an object of type DoFn and apply it to all elements of a
PCollection object.
In order to have concrete DoFn objects one has to subclass from DoFn and
define the desired behavior (start_bundle/finish_bundle and process) or wrap a
callable object using the CallableWrapperDoFn class.
"""
# Parameters that can be used in the .process() method.
ElementParam = _DoFnParam('ElementParam')
SideInputParam = _DoFnParam('SideInputParam')
TimestampParam = _DoFnParam('TimestampParam')
WindowParam = _DoFnParam('WindowParam')
PaneInfoParam = _DoFnParam('PaneInfoParam')
WatermarkEstimatorParam = _WatermarkEstimatorParam
BundleFinalizerParam = _BundleFinalizerParam
KeyParam = _DoFnParam('KeyParam')
# Parameters to access state and timers. Not restricted to use only in the
# .process() method. Usage: DoFn.StateParam(state_spec),
# DoFn.TimerParam(timer_spec), DoFn.TimestampParam, DoFn.WindowParam,
# DoFn.KeyParam
StateParam = _StateDoFnParam
TimerParam = _TimerDoFnParam
DynamicTimerTagParam = _DoFnParam('DynamicTimerTagParam')
DoFnProcessParams = [
ElementParam,
SideInputParam,
TimestampParam,
WindowParam,
WatermarkEstimatorParam,
PaneInfoParam,
BundleFinalizerParam,
KeyParam,
StateParam,
TimerParam,
]
RestrictionParam = _RestrictionDoFnParam
@staticmethod
def from_callable(fn):
return CallableWrapperDoFn(fn)
@staticmethod
def unbounded_per_element():
"""A decorator on process fn specifying that the fn performs an unbounded
amount of work per input element."""
def wrapper(process_fn):
process_fn.unbounded_per_element = True
return process_fn
return wrapper
@staticmethod
def yields_elements(fn):
"""A decorator to apply to ``process_batch`` indicating it yields elements.
By default ``process_batch`` is assumed to both consume and produce
"batches", which are collections of multiple logical Beam elements. This
decorator indicates that ``process_batch`` **produces** individual elements
at a time. ``process_batch`` is always expected to consume batches.
"""
if not fn.__name__ in ('process', 'process_batch'):
raise TypeError(
"@yields_elements must be applied to a process or "
f"process_batch method, got {fn!r}.")
fn._beam_yields_elements = True
return fn
@staticmethod
def yields_batches(fn):
"""A decorator to apply to ``process`` indicating it yields batches.
By default ``process`` is assumed to both consume and produce
individual elements at a time. This decorator indicates that ``process``
**produces** "batches", which are collections of multiple logical Beam
elements.
"""
if not fn.__name__ in ('process', 'process_batch'):
raise TypeError(
"@yields_elements must be applied to a process or "
f"process_batch method, got {fn!r}.")
fn._beam_yields_batches = True
return fn
def default_label(self):
return self.__class__.__name__
def process(self, element, *args, **kwargs):
"""Method to use for processing elements.
This is invoked by ``DoFnRunner`` for each element of a input
``PCollection``.
The following parameters can be used as default values on ``process``
arguments to indicate that a DoFn accepts the corresponding parameters. For
example, a DoFn might accept the element and its timestamp with the
following signature::
def process(element=DoFn.ElementParam, timestamp=DoFn.TimestampParam):
...
The full set of parameters is:
- ``DoFn.ElementParam``: element to be processed, should not be mutated.
- ``DoFn.SideInputParam``: a side input that may be used when processing.
- ``DoFn.TimestampParam``: timestamp of the input element.
- ``DoFn.WindowParam``: ``Window`` the input element belongs to.
- ``DoFn.TimerParam``: a ``userstate.RuntimeTimer`` object defined by the
spec of the parameter.
- ``DoFn.StateParam``: a ``userstate.RuntimeState`` object defined by the
spec of the parameter.
- ``DoFn.KeyParam``: key associated with the element.
- ``DoFn.RestrictionParam``: an ``iobase.RestrictionTracker`` will be
provided here to allow treatment as a Splittable ``DoFn``. The restriction
tracker will be derived from the restriction provider in the parameter.
- ``DoFn.WatermarkEstimatorParam``: a function that can be used to track
output watermark of Splittable ``DoFn`` implementations.
Args:
element: The element to be processed
*args: side inputs
**kwargs: other keyword arguments.
Returns:
An Iterable of output elements or None.
"""
raise NotImplementedError
def process_batch(self, batch, *args, **kwargs):
raise NotImplementedError
def setup(self):
"""Called to prepare an instance for processing bundles of elements.
This is a good place to initialize transient in-memory resources, such as
network connections. The resources can then be disposed in
``DoFn.teardown``.
"""
pass
def start_bundle(self):
"""Called before a bundle of elements is processed on a worker.
Elements to be processed are split into bundles and distributed
to workers. Before a worker calls process() on the first element
of its bundle, it calls this method.
"""
pass
def finish_bundle(self):
"""Called after a bundle of elements is processed on a worker.
"""
pass
def teardown(self):
"""Called to use to clean up this instance before it is discarded.
A runner will do its best to call this method on any given instance to
prevent leaks of transient resources, however, there may be situations where
this is impossible (e.g. process crash, hardware failure, etc.) or
unnecessary (e.g. the pipeline is shutting down and the process is about to
be killed anyway, so all transient resources will be released automatically
by the OS). In these cases, the call may not happen. It will also not be
retried, because in such situations the DoFn instance no longer exists, so
there's no instance to retry it on.
Thus, all work that depends on input elements, and all externally important
side effects, must be performed in ``DoFn.process`` or
``DoFn.finish_bundle``.
"""
pass
def get_function_arguments(self, func):
return get_function_arguments(self, func)
def default_type_hints(self):
process_type_hints = typehints.decorators.IOTypeHints.from_callable(
self.process) or typehints.decorators.IOTypeHints.empty()
if self._process_yields_batches:
# process() produces batches, don't use it's output typehint
process_type_hints = process_type_hints.with_output_types_from(
typehints.decorators.IOTypeHints.empty())
if self._process_batch_yields_elements:
# process_batch() produces elements, *do* use it's output typehint
# First access the typehint
process_batch_type_hints = typehints.decorators.IOTypeHints.from_callable(
self.process_batch) or typehints.decorators.IOTypeHints.empty()
# Then we deconflict with the typehint from process, if it exists
if (process_batch_type_hints.output_types !=
typehints.decorators.IOTypeHints.empty().output_types):
if (process_type_hints.output_types !=
typehints.decorators.IOTypeHints.empty().output_types and
process_batch_type_hints.output_types !=
process_type_hints.output_types):
raise TypeError(
f"DoFn {self!r} yields element from both process and "
"process_batch, but they have mismatched output typehints:\n"
f" process: {process_type_hints.output_types}\n"
f" process_batch: {process_batch_type_hints.output_types}")
process_type_hints = process_type_hints.with_output_types_from(
process_batch_type_hints)
try:
process_type_hints = process_type_hints.strip_iterable()
except ValueError as e:
raise ValueError('Return value not iterable: %s: %s' % (self, e))
# Prefer class decorator type hints for backwards compatibility.
return get_type_hints(self.__class__).with_defaults(process_type_hints)
# TODO(sourabhbajaj): Do we want to remove the responsibility of these from
# the DoFn or maybe the runner
def infer_output_type(self, input_type):
# TODO(https://github.com/apache/beam/issues/19824): Side inputs types.
return trivial_inference.element_type(
_strip_output_annotations(
trivial_inference.infer_return_type(self.process, [input_type])))
@property
def _process_defined(self) -> bool:
# Check if this DoFn's process method has been overridden
# Note that we retrieve the __func__ attribute, if it exists, to get the
# underlying function from the bound method.
# If __func__ doesn't exist, self.process was likely overridden with a free
# function, as in CallableWrapperDoFn.
return getattr(self.process, '__func__', self.process) != DoFn.process
@property
def _process_batch_defined(self) -> bool:
# Check if this DoFn's process_batch method has been overridden
# Note that we retrieve the __func__ attribute, if it exists, to get the
# underlying function from the bound method.
# If __func__ doesn't exist, self.process_batch was likely overridden with
# a free function.
return getattr(
self.process_batch, '__func__',
self.process_batch) != DoFn.process_batch
@property
def _can_yield_batches(self) -> bool:
return ((self._process_defined and self._process_yields_batches) or (
self._process_batch_defined and
not self._process_batch_yields_elements))
@property
def _process_yields_batches(self) -> bool:
return getattr(self.process, '_beam_yields_batches', False)
@property
def _process_batch_yields_elements(self) -> bool:
return getattr(self.process_batch, '_beam_yields_elements', False)
def get_input_batch_type(
self, input_element_type
) -> typing.Optional[typing.Union[TypeConstraint, type]]:
"""Determine the batch type expected as input to process_batch.
The default implementation of ``get_input_batch_type`` simply observes the
input typehint for the first parameter of ``process_batch``. A Batched DoFn
may override this method if a dynamic approach is required.
Args:
input_element_type: The **element type** of the input PCollection this
DoFn is being applied to.
Returns:
``None`` if this DoFn cannot accept batches, else a Beam typehint or
a native Python typehint.
"""
if not self._process_batch_defined:
return None
input_type = list(
inspect.signature(self.process_batch).parameters.values())[0].annotation
if input_type == inspect.Signature.empty:
# TODO(https://github.com/apache/beam/issues/21652): Consider supporting
# an alternative (dynamic?) approach for declaring input type
raise TypeError(
f"Either {self.__class__.__name__}.process_batch() must have a type "
f"annotation on its first parameter, or {self.__class__.__name__} "
"must override get_input_batch_type.")
return input_type
def _get_input_batch_type_normalized(self, input_element_type):
return typehints.native_type_compatibility.convert_to_beam_type(
self.get_input_batch_type(input_element_type))
def _get_output_batch_type_normalized(self, input_element_type):
return typehints.native_type_compatibility.convert_to_beam_type(
self.get_output_batch_type(input_element_type))
@staticmethod
def _get_element_type_from_return_annotation(method, input_type):
return_type = inspect.signature(method).return_annotation
if return_type == inspect.Signature.empty:
# output type not annotated, try to infer it
return_type = trivial_inference.infer_return_type(method, [input_type])
return_type = typehints.native_type_compatibility.convert_to_beam_type(
return_type)
if isinstance(return_type, typehints.typehints.IterableTypeConstraint):
return return_type.inner_type
elif isinstance(return_type, typehints.typehints.IteratorTypeConstraint):
return return_type.yielded_type
else:
raise TypeError(
"Expected Iterator in return type annotation for "
f"{method!r}, did you mean Iterator[{return_type}]? Note Beam DoFn "
"process and process_batch methods are expected to produce "
"generators - they should 'yield' rather than 'return'.")
def get_output_batch_type(
self, input_element_type
) -> typing.Optional[typing.Union[TypeConstraint, type]]:
"""Determine the batch type produced by this DoFn's ``process_batch``
implementation and/or its ``process`` implementation with
``@yields_batch``.
The default implementation of this method observes the return type
annotations on ``process_batch`` and/or ``process``. A Batched DoFn may
override this method if a dynamic approach is required.
Args:
input_element_type: The **element type** of the input PCollection this
DoFn is being applied to.
Returns:
``None`` if this DoFn will never yield batches, else a Beam typehint or
a native Python typehint.
"""
output_batch_type = None
if self._process_defined and self._process_yields_batches:
output_batch_type = self._get_element_type_from_return_annotation(
self.process, input_element_type)
if self._process_batch_defined and not self._process_batch_yields_elements:
process_batch_type = self._get_element_type_from_return_annotation(
self.process_batch,
self._get_input_batch_type_normalized(input_element_type))
# TODO: Consider requiring an inheritance relationship rather than
# equality
if (output_batch_type is not None and
(not process_batch_type == output_batch_type)):
raise TypeError(
f"DoFn {self!r} yields batches from both process and "
"process_batch, but they produce different types:\n"
f" process: {output_batch_type}\n"
f" process_batch: {process_batch_type!r}")
output_batch_type = process_batch_type
return output_batch_type
def _process_argspec_fn(self):
"""Returns the Python callable that will eventually be invoked.
This should ideally be the user-level function that is called with
the main and (if any) side inputs, and is used to relate the type
hint parameters with the input parameters (e.g., by argument name).
"""
return self.process
urns.RunnerApiFn.register_pickle_urn(python_urns.PICKLED_DOFN)
class CallableWrapperDoFn(DoFn):
"""For internal use only; no backwards-compatibility guarantees.
A DoFn (function) object wrapping a callable object.
The purpose of this class is to conveniently wrap simple functions and use
them in transforms.
"""
def __init__(self, fn, fullargspec=None):
"""Initializes a CallableWrapperDoFn object wrapping a callable.
Args:
fn: A callable object.
Raises:
TypeError: if fn parameter is not a callable type.
"""
if not callable(fn):
raise TypeError('Expected a callable object instead of: %r' % fn)
self._fn = fn
self._fullargspec = fullargspec
if isinstance(
fn, (types.BuiltinFunctionType, types.MethodType, types.FunctionType)):
self.process = fn
else:
# For cases such as set / list where fn is callable but not a function
self.process = lambda element: fn(element)
super().__init__()
def display_data(self):
# If the callable has a name, then it's likely a function, and
# we show its name.
# Otherwise, it might be an instance of a callable class. We
# show its class.
display_data_value = (
self._fn.__name__
if hasattr(self._fn, '__name__') else self._fn.__class__)
return {
'fn': DisplayDataItem(display_data_value, label='Transform Function')
}
def __repr__(self):
return 'CallableWrapperDoFn(%s)' % self._fn
def default_type_hints(self):
fn_type_hints = typehints.decorators.IOTypeHints.from_callable(self._fn)
type_hints = get_type_hints(self._fn).with_defaults(fn_type_hints)
# The fn's output type should be iterable. Strip off the outer
# container type due to the 'flatten' portion of FlatMap/ParDo.
try:
type_hints = type_hints.strip_iterable()
except ValueError as e:
raise TypeCheckError(
'Return value not iterable: %s: %s' %
(self.display_data()['fn'].value, e))
return type_hints
def infer_output_type(self, input_type):
return trivial_inference.element_type(
_strip_output_annotations(
trivial_inference.infer_return_type(self._fn, [input_type])))
def _process_argspec_fn(self):
return getattr(self._fn, '_argspec_fn', self._fn)
def _inspect_process(self):
if self._fullargspec:
return self._fullargspec
else:
return get_function_args_defaults(self._process_argspec_fn())
class CombineFn(WithTypeHints, HasDisplayData, urns.RunnerApiFn):
"""A function object used by a Combine transform with custom processing.
A CombineFn specifies how multiple values in all or part of a PCollection can
be merged into a single value---essentially providing the same kind of
information as the arguments to the Python "reduce" builtin (except for the
input argument, which is an instance of CombineFnProcessContext). The
combining process proceeds as follows:
1. Input values are partitioned into one or more batches.
2. For each batch, the setup method is invoked.
3. For each batch, the create_accumulator method is invoked to create a fresh
initial "accumulator" value representing the combination of zero values.
4. For each input value in the batch, the add_input method is invoked to
combine more values with the accumulator for that batch.
5. The merge_accumulators method is invoked to combine accumulators from
separate batches into a single combined output accumulator value, once all
of the accumulators have had all the input value in their batches added to