forked from apache/beam
/
common.py
801 lines (657 loc) · 29.3 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
#
# 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.
#
# cython: language_level=3
# cython: profile=True
"""Worker operations executor.
For internal use only; no backwards-compatibility guarantees.
"""
from __future__ import absolute_import
import sys
import traceback
from builtins import next
from builtins import object
from builtins import zip
from future.utils import raise_
from past.builtins import basestring
from past.builtins import unicode
from apache_beam.internal import util
from apache_beam.options.value_provider import RuntimeValueProvider
from apache_beam.pvalue import TaggedOutput
from apache_beam.transforms import DoFn
from apache_beam.transforms import core
from apache_beam.transforms.core import RestrictionProvider
from apache_beam.transforms.window import GlobalWindow
from apache_beam.transforms.window import TimestampedValue
from apache_beam.transforms.window import WindowFn
from apache_beam.utils.counters import Counter
from apache_beam.utils.counters import CounterName
from apache_beam.utils.windowed_value import WindowedValue
class NameContext(object):
"""Holds the name information for a step."""
def __init__(self, step_name):
"""Creates a new step NameContext.
Args:
step_name: The name of the step.
"""
self.step_name = step_name
def __eq__(self, other):
return self.step_name == other.step_name
def __ne__(self, other):
return not self == other
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
# TODO(BEAM-4028): Move DataflowNameContext to Dataflow internal code.
class DataflowNameContext(NameContext):
"""Holds the name information for a step in Dataflow.
This includes a step_name (e.g. s2), a user_name (e.g. Foo/Bar/ParDo(Fab)),
and a system_name (e.g. s2-shuffle-read34)."""
def __init__(self, step_name, user_name, system_name):
"""Creates a new step NameContext.
Args:
step_name: The internal name of the step (e.g. s2).
user_name: The full user-given name of the step (e.g. Foo/Bar/ParDo(Far)).
system_name: The step name in the optimized graph (e.g. s2-1).
"""
super(DataflowNameContext, self).__init__(step_name)
self.user_name = user_name
self.system_name = system_name
def __eq__(self, other):
return (self.step_name == other.step_name and
self.user_name == other.user_name and
self.system_name == other.system_name)
def __ne__(self, other):
return not self == other
def __hash__(self):
return hash((self.step_name, self.user_name, self.system_name))
def __repr__(self):
return 'DataflowNameContext(%s)' % self.__dict__
def logging_name(self):
"""Stackdriver logging relies on user-given step names (e.g. Foo/Bar)."""
return self.user_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):
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)):
raise ValueError('\'obj_to_invoke\' has to be either a \'DoFn\' or '
'a \'RestrictionProvider\'. Received %r instead.'
% obj_to_invoke)
args, _, _, defaults = core.get_function_arguments(
obj_to_invoke, method_name)
defaults = defaults if defaults else []
method_value = getattr(obj_to_invoke, method_name)
self.method_value = method_value
self.args = args
self.defaults = defaults
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):
# 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.start_bundle_method = MethodWrapper(do_fn, 'start_bundle')
self.finish_bundle_method = MethodWrapper(do_fn, 'finish_bundle')
restriction_provider = self._get_restriction_provider(do_fn)
self.initial_restriction_method = (
MethodWrapper(restriction_provider, 'initial_restriction')
if restriction_provider else None)
self.restriction_coder_method = (
MethodWrapper(restriction_provider, 'restriction_coder')
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()
def _get_restriction_provider(self, do_fn):
result = _find_param_with_default(self.process_method,
default_as_type=RestrictionProvider)
return result[1] if result else None
def _validate(self):
self._validate_process()
self._validate_bundle_method(self.start_bundle_method)
self._validate_bundle_method(self.finish_bundle_method)
def _validate_process(self):
"""Validate that none of the DoFnParameters are repeated in the function
"""
for param in core.DoFn.DoFnParams:
assert self.process_method.defaults.count(param) <= 1
def _validate_bundle_method(self, method_wrapper):
"""Validate that none of the DoFnParameters are used in the function
"""
for param in core.DoFn.DoFnParams:
assert param not in method_wrapper.defaults
def is_splittable_dofn(self):
return any([isinstance(default, RestrictionProvider) for default in
self.process_method.defaults])
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_processor, signature):
self.output_processor = output_processor
self.signature = signature
@staticmethod
def create_invoker(
signature,
output_processor=None,
context=None, side_inputs=None, input_args=None, input_kwargs=None,
process_invocation=True):
""" Creates a new DoFnInvoker based on given arguments.
Args:
output_processor: an OutputProcessor 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.
"""
side_inputs = side_inputs or []
default_arg_values = signature.process_method.defaults
use_simple_invoker = not process_invocation or (
not side_inputs and not input_args and not input_kwargs and
not default_arg_values)
if use_simple_invoker:
return SimpleInvoker(output_processor, signature)
else:
return PerWindowInvoker(
output_processor,
signature, context, side_inputs, input_args, input_kwargs)
def invoke_process(self, windowed_value, restriction_tracker=None,
output_processor=None,
additional_args=None, additional_kwargs=None):
"""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.
output_procesor: if provided given OutputProcessor will be used.
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_start_bundle(self):
"""Invokes the DoFn.start_bundle() method.
"""
self.output_processor.start_bundle_outputs(
self.signature.start_bundle_method.method_value())
def invoke_finish_bundle(self):
"""Invokes the DoFn.finish_bundle() method.
"""
self.output_processor.finish_bundle_outputs(
self.signature.finish_bundle_method.method_value())
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_restriction_coder(self):
return self.signature.restriction_coder_method.method_value()
def invoke_create_tracker(self, restriction):
return self.signature.create_tracker_method.method_value(restriction)
def _find_param_with_default(
method, default_as_value=None, default_as_type=None):
if ((default_as_value and default_as_type) or
not (default_as_value or default_as_type)):
raise ValueError(
'Exactly one of \'default_as_value\' and \'default_as_type\' should be '
'provided. Received %r and %r.' % (default_as_value, default_as_type))
defaults = method.defaults
default_as_value = default_as_value
default_as_type = default_as_type
ret = None
for i, value in enumerate(defaults):
if default_as_value and value == default_as_value:
ret = (method.args[len(method.args) - len(defaults) + i], value)
elif default_as_type and isinstance(value, default_as_type):
index = len(method.args) - len(defaults) + i
ret = (method.args[index], value)
return ret
class SimpleInvoker(DoFnInvoker):
"""An invoker that processes elements ignoring windowing information."""
def __init__(self, output_processor, signature):
super(SimpleInvoker, self).__init__(output_processor, signature)
self.process_method = signature.process_method.method_value
def invoke_process(self, windowed_value, restriction_tracker=None,
output_processor=None,
additional_args=None, additional_kwargs=None):
if not output_processor:
output_processor = self.output_processor
output_processor.process_outputs(
windowed_value, self.process_method(windowed_value.value))
class PerWindowInvoker(DoFnInvoker):
"""An invoker that processes elements considering windowing information."""
def __init__(self, output_processor, signature, context,
side_inputs, input_args, input_kwargs):
super(PerWindowInvoker, self).__init__(output_processor, 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
(core.DoFn.WindowParam in default_arg_values))
# 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.
# Flag to cache additional arguments on the first element if all
# inputs are within the global window.
self.cache_globally_windowed_args = not self.has_windowed_inputs
input_args = input_args if input_args else []
input_kwargs = input_kwargs if input_kwargs else {}
arguments = signature.process_method.args
defaults = signature.process_method.defaults
# Create placeholder for element parameter of DoFn.process() method.
self_in_args = int(signature.do_fn.is_process_bounded())
class ArgPlaceholder(object):
def __init__(self, placeholder):
self.placeholder = placeholder
if core.DoFn.ElementParam not in default_arg_values:
args_to_pick = len(arguments) - len(default_arg_values) - 1 - self_in_args
args_with_placeholders = (
[ArgPlaceholder(core.DoFn.ElementParam)] + input_args[:args_to_pick])
else:
args_to_pick = len(arguments) - len(defaults) - self_in_args
args_with_placeholders = input_args[:args_to_pick]
# Fill the OtherPlaceholders for context, window or timestamp
remaining_args_iter = iter(input_args[args_to_pick:])
for a, d in zip(arguments[-len(defaults):], defaults):
if d == core.DoFn.ElementParam:
args_with_placeholders.append(ArgPlaceholder(d))
elif d == core.DoFn.WindowParam:
args_with_placeholders.append(ArgPlaceholder(d))
elif d == core.DoFn.TimestampParam:
args_with_placeholders.append(ArgPlaceholder(d))
elif d == core.DoFn.SideInputParam:
# 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)
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
self.placeholders = [(i, x.placeholder) for (i, x) in enumerate(
args_with_placeholders)
if isinstance(x, ArgPlaceholder)]
self.args_for_process = args_with_placeholders
self.kwargs_for_process = input_kwargs
def invoke_process(self, windowed_value, restriction_tracker=None,
output_processor=None,
additional_args=None, additional_kwargs=None):
if not additional_args:
additional_args = []
if not additional_kwargs:
additional_kwargs = {}
if not output_processor:
output_processor = self.output_processor
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
if restriction_tracker:
restriction_tracker_param = _find_param_with_default(
self.signature.process_method,
default_as_type=core.RestrictionProvider)[0]
if not restriction_tracker_param:
raise ValueError(
'A RestrictionTracker %r was provided but DoFn does not have a '
'RestrictionTrackerParam defined' % restriction_tracker)
additional_kwargs[restriction_tracker_param] = restriction_tracker
if self.has_windowed_inputs and len(windowed_value.windows) != 1:
for w in windowed_value.windows:
self._invoke_per_window(
WindowedValue(windowed_value.value, windowed_value.timestamp, (w,)),
additional_args, additional_kwargs, output_processor)
else:
self._invoke_per_window(
windowed_value, additional_args, additional_kwargs, output_processor)
def _invoke_per_window(
self, windowed_value, additional_args,
additional_kwargs, output_processor):
if self.has_windowed_inputs:
window, = windowed_value.windows
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)
elif self.cache_globally_windowed_args:
# Attempt to cache additional args if all inputs are globally
# windowed inputs when processing the first element.
self.cache_globally_windowed_args = False
# Fill in sideInputs if they are globally windowed
global_window = GlobalWindow()
self.args_for_process, self.kwargs_for_process = (
util.insert_values_in_args(
self.args_for_process, self.kwargs_for_process,
[si[global_window] for si in self.side_inputs]))
args_for_process, kwargs_for_process = (
self.args_for_process, self.kwargs_for_process)
else:
args_for_process, kwargs_for_process = (
self.args_for_process, self.kwargs_for_process)
# TODO(sourabhbajaj): Investigate why we can't use `is` instead of ==
for i, p in self.placeholders:
if p == core.DoFn.ElementParam:
args_for_process[i] = windowed_value.value
elif p == core.DoFn.WindowParam:
args_for_process[i] = window
elif p == core.DoFn.TimestampParam:
args_for_process[i] = windowed_value.timestamp
if additional_kwargs:
if kwargs_for_process is None:
kwargs_for_process = additional_kwargs
else:
for key in additional_kwargs:
kwargs_for_process[key] = additional_kwargs[key]
if kwargs_for_process:
output_processor.process_outputs(
windowed_value,
self.process_method(*args_for_process, **kwargs_for_process))
else:
output_processor.process_outputs(
windowed_value, self.process_method(*args_for_process))
class DoFnRunner(Receiver):
"""For internal use only; no backwards-compatibility guarantees.
A helper class for executing ParDo operations.
"""
def __init__(self,
fn,
args,
kwargs,
side_inputs,
windowing,
tagged_receivers=None,
step_name=None,
logging_context=None,
state=None,
scoped_metrics_container=None,
operation_name=None):
"""Initializes a DoFnRunner.
Args:
fn: user DoFn to invoke
args: positional side input arguments (static and placeholder), if any
kwargs: keyword side input arguments (static and placeholder), if any
side_inputs: list of sideinput.SideInputMaps for deferred side inputs
windowing: windowing properties of the output PCollection(s)
tagged_receivers: a dict of tag name to Receiver objects
step_name: the name of this step
logging_context: DEPRECATED [BEAM-4728]
state: handle for accessing DoFn state
scoped_metrics_container: DEPRECATED
operation_name: The system name assigned by the runner for this operation.
"""
# Need to support multiple iterations.
side_inputs = list(side_inputs)
self.step_name = step_name
self.context = DoFnContext(step_name, state=state)
do_fn_signature = DoFnSignature(fn)
# Optimize for the common case.
main_receivers = tagged_receivers[None]
# TODO(BEAM-3937): Remove if block after output counter released.
if 'outputs_per_element_counter' in RuntimeValueProvider.experiments:
# TODO(BEAM-3955): Make step_name and operation_name less confused.
output_counter_name = (CounterName('per-element-output-count',
step_name=operation_name))
per_element_output_counter = state._counter_factory.get_counter(
output_counter_name, Counter.DATAFLOW_DISTRIBUTION).accumulator
else:
per_element_output_counter = None
output_processor = _OutputProcessor(
windowing.windowfn, main_receivers, tagged_receivers,
per_element_output_counter)
self.do_fn_invoker = DoFnInvoker.create_invoker(
do_fn_signature, output_processor, self.context, side_inputs, args,
kwargs)
def receive(self, windowed_value):
self.process(windowed_value)
def process(self, windowed_value):
try:
self.do_fn_invoker.invoke_process(windowed_value)
except BaseException as exn:
self._reraise_augmented(exn)
def _invoke_bundle_method(self, bundle_method):
try:
self.context.set_element(None)
bundle_method()
except BaseException as exn:
self._reraise_augmented(exn)
def start(self):
self._invoke_bundle_method(self.do_fn_invoker.invoke_start_bundle)
def finish(self):
self._invoke_bundle_method(self.do_fn_invoker.invoke_finish_bundle)
def _reraise_augmented(self, exn):
if getattr(exn, '_tagged_with_step', False) or not self.step_name:
raise
step_annotation = " [while running '%s']" % self.step_name
# To emulate exception chaining (not available in Python 2).
original_traceback = sys.exc_info()[2]
try:
# Attempt to construct the same kind of exception
# with an augmented message.
new_exn = type(exn)(exn.args[0] + step_annotation, *exn.args[1:])
new_exn._tagged_with_step = True # Could raise attribute error.
except: # pylint: disable=bare-except
# If anything goes wrong, construct a RuntimeError whose message
# records the original exception's type and message.
new_exn = RuntimeError(
traceback.format_exception_only(type(exn), exn)[-1].strip()
+ step_annotation)
new_exn._tagged_with_step = True
raise_(type(new_exn), new_exn, original_traceback)
class OutputProcessor(object):
def process_outputs(self, windowed_input_element, results):
raise NotImplementedError
class _OutputProcessor(OutputProcessor):
"""Processes output produced by DoFn method invocations."""
def __init__(self,
window_fn,
main_receivers,
tagged_receivers,
per_element_output_counter):
"""Initializes ``_OutputProcessor``.
Args:
window_fn: a windowing function (WindowFn).
main_receivers: a dict of tag name to Receiver objects.
tagged_receivers: main receiver object.
per_element_output_counter: per_element_output_counter of one work_item.
could be none if experimental flag turn off
"""
self.window_fn = window_fn
self.main_receivers = main_receivers
self.tagged_receivers = tagged_receivers
self.per_element_output_counter = per_element_output_counter
def process_outputs(self, windowed_input_element, results):
"""Dispatch the result of process computation to the appropriate receivers.
A value wrapped in a TaggedOutput object will be unwrapped and
then dispatched to the appropriate indexed output.
"""
if results is None:
# TODO(BEAM-3937): Remove if block after output counter released.
# Only enable per_element_output_counter when counter cythonized.
if (self.per_element_output_counter is not None and
self.per_element_output_counter.is_cythonized):
self.per_element_output_counter.add_input(0)
return
output_element_count = 0
for result in results:
# results here may be a generator, which cannot call len on it.
output_element_count += 1
tag = None
if isinstance(result, TaggedOutput):
tag = result.tag
if not isinstance(tag, basestring):
raise TypeError('In %s, tag %s is not a string' % (self, tag))
result = result.value
if isinstance(result, WindowedValue):
windowed_value = result
if (windowed_input_element is not None
and len(windowed_input_element.windows) != 1):
windowed_value.windows *= len(windowed_input_element.windows)
elif isinstance(result, TimestampedValue):
assign_context = WindowFn.AssignContext(result.timestamp, result.value)
windowed_value = WindowedValue(
result.value, result.timestamp,
self.window_fn.assign(assign_context))
if len(windowed_input_element.windows) != 1:
windowed_value.windows *= len(windowed_input_element.windows)
else:
windowed_value = windowed_input_element.with_value(result)
if tag is None:
self.main_receivers.receive(windowed_value)
else:
self.tagged_receivers[tag].receive(windowed_value)
# TODO(BEAM-3937): Remove if block after output counter released.
# Only enable per_element_output_counter when counter cythonized
if (self.per_element_output_counter is not None and
self.per_element_output_counter.is_cythonized):
self.per_element_output_counter.add_input(output_element_count)
def start_bundle_outputs(self, results):
"""Validate that start_bundle does not output any elements"""
if results is None:
return
raise RuntimeError(
'Start Bundle should not output any elements but got %s' % results)
def finish_bundle_outputs(self, results):
"""Dispatch the result of finish_bundle to the appropriate receivers.
A value wrapped in a TaggedOutput object will be unwrapped and
then dispatched to the appropriate indexed output.
"""
if results is None:
return
for result in results:
tag = None
if isinstance(result, TaggedOutput):
tag = result.tag
if not isinstance(tag, (str, unicode)):
raise TypeError('In %s, tag %s is not a string' % (self, tag))
result = result.value
if isinstance(result, WindowedValue):
windowed_value = result
else:
raise RuntimeError('Finish Bundle should only output WindowedValue ' +\
'type but got %s' % type(result))
if tag is None:
self.main_receivers.receive(windowed_value)
else:
self.tagged_receivers[tag].receive(windowed_value)
class _NoContext(WindowFn.AssignContext):
"""An uninspectable WindowFn.AssignContext."""
NO_VALUE = object()
def __init__(self, value, timestamp=NO_VALUE):
self.value = value
self._timestamp = timestamp
@property
def timestamp(self):
if self._timestamp is self.NO_VALUE:
raise ValueError('No timestamp in this context.')
else:
return self._timestamp
@property
def existing_windows(self):
raise ValueError('No existing_windows in this context.')
class DoFnState(object):
"""For internal use only; no backwards-compatibility guarantees.
Keeps track of state that DoFns want, currently, user counters.
"""
def __init__(self, counter_factory):
self.step_name = ''
self._counter_factory = counter_factory
def counter_for(self, aggregator):
"""Looks up the counter for this aggregator, creating one if necessary."""
return self._counter_factory.get_aggregator_counter(
self.step_name, aggregator)
# TODO(robertwb): Replace core.DoFnContext with this.
class DoFnContext(object):
"""For internal use only; no backwards-compatibility guarantees."""
def __init__(self, label, element=None, state=None):
self.label = label
self.state = state
if element is not None:
self.set_element(element)
def set_element(self, windowed_value):
self.windowed_value = windowed_value
@property
def element(self):
if self.windowed_value is None:
raise AttributeError('element not accessible in this context')
else:
return self.windowed_value.value
@property
def timestamp(self):
if self.windowed_value is None:
raise AttributeError('timestamp not accessible in this context')
else:
return self.windowed_value.timestamp
@property
def windows(self):
if self.windowed_value is None:
raise AttributeError('windows not accessible in this context')
else:
return self.windowed_value.windows