-
Notifications
You must be signed in to change notification settings - Fork 4.3k
/
pipeline.py
1553 lines (1323 loc) · 60.8 KB
/
pipeline.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.
#
"""Pipeline, the top-level Beam object.
A pipeline holds a DAG of data transforms. Conceptually the nodes of the DAG
are transforms (:class:`~apache_beam.transforms.ptransform.PTransform` objects)
and the edges are values (mostly :class:`~apache_beam.pvalue.PCollection`
objects). The transforms take as inputs one or more PValues and output one or
more :class:`~apache_beam.pvalue.PValue` s.
The pipeline offers functionality to traverse the graph. The actual operation
to be executed for each node visited is specified through a runner object.
Typical usage::
# Create a pipeline object using a local runner for execution.
with beam.Pipeline('DirectRunner') as p:
# Add to the pipeline a "Create" transform. When executed this
# transform will produce a PCollection object with the specified values.
pcoll = p | 'Create' >> beam.Create([1, 2, 3])
# Another transform could be applied to pcoll, e.g., writing to a text file.
# For other transforms, refer to transforms/ directory.
pcoll | 'Write' >> beam.io.WriteToText('./output')
# run() will execute the DAG stored in the pipeline. The execution of the
# nodes visited is done using the specified local runner.
"""
# pytype: skip-file
# mypy: disallow-untyped-defs
import abc
import logging
import os
import re
import shutil
import tempfile
import unicodedata
from collections import defaultdict
from typing import TYPE_CHECKING
from typing import Any
from typing import Dict
from typing import FrozenSet
from typing import Iterable
from typing import List
from typing import Mapping
from typing import Optional
from typing import Sequence
from typing import Set
from typing import Tuple
from typing import Type
from typing import Union
from google.protobuf import message
from apache_beam import pvalue
from apache_beam.internal import pickler
from apache_beam.io.filesystems import FileSystems
from apache_beam.options.pipeline_options import CrossLanguageOptions
from apache_beam.options.pipeline_options import DebugOptions
from apache_beam.options.pipeline_options import PipelineOptions
from apache_beam.options.pipeline_options import SetupOptions
from apache_beam.options.pipeline_options import StandardOptions
from apache_beam.options.pipeline_options import TypeOptions
from apache_beam.options.pipeline_options_validator import PipelineOptionsValidator
from apache_beam.portability import common_urns
from apache_beam.portability.api import beam_runner_api_pb2
from apache_beam.runners import PipelineRunner
from apache_beam.runners import create_runner
from apache_beam.transforms import ParDo
from apache_beam.transforms import ptransform
from apache_beam.transforms.display import DisplayData
from apache_beam.transforms.resources import merge_resource_hints
from apache_beam.transforms.resources import resource_hints_from_options
from apache_beam.transforms.sideinputs import get_sideinput_index
from apache_beam.typehints import TypeCheckError
from apache_beam.typehints import typehints
from apache_beam.utils import proto_utils
from apache_beam.utils import subprocess_server
from apache_beam.utils.annotations import deprecated
from apache_beam.utils.interactive_utils import alter_label_if_ipython
from apache_beam.utils.interactive_utils import is_in_ipython
if TYPE_CHECKING:
from types import TracebackType
from apache_beam.runners.pipeline_context import PipelineContext
from apache_beam.runners.runner import PipelineResult
from apache_beam.transforms import environments
__all__ = ['Pipeline', 'PTransformOverride']
class Pipeline(object):
"""A pipeline object that manages a DAG of
:class:`~apache_beam.pvalue.PValue` s and their
:class:`~apache_beam.transforms.ptransform.PTransform` s.
Conceptually the :class:`~apache_beam.pvalue.PValue` s are the DAG's nodes and
the :class:`~apache_beam.transforms.ptransform.PTransform` s computing
the :class:`~apache_beam.pvalue.PValue` s are the edges.
All the transforms applied to the pipeline must have distinct full labels.
If same transform instance needs to be applied then the right shift operator
should be used to designate new names
(e.g. ``input | "label" >> my_transform``).
"""
@classmethod
def runner_implemented_transforms(cls):
# type: () -> FrozenSet[str]
# This set should only contain transforms which are required to be
# implemented by a runner.
return frozenset([
common_urns.primitives.GROUP_BY_KEY.urn,
common_urns.primitives.IMPULSE.urn,
])
def __init__(self, runner=None, options=None, argv=None):
# type: (Optional[Union[str, PipelineRunner]], Optional[PipelineOptions], Optional[List[str]]) -> None
"""Initialize a pipeline object.
Args:
runner (~apache_beam.runners.runner.PipelineRunner): An object of
type :class:`~apache_beam.runners.runner.PipelineRunner` that will be
used to execute the pipeline. For registered runners, the runner name
can be specified, otherwise a runner object must be supplied.
options (~apache_beam.options.pipeline_options.PipelineOptions):
A configured
:class:`~apache_beam.options.pipeline_options.PipelineOptions` object
containing arguments that should be used for running the Beam job.
argv (List[str]): a list of arguments (such as :data:`sys.argv`)
to be used for building a
:class:`~apache_beam.options.pipeline_options.PipelineOptions` object.
This will only be used if argument **options** is :data:`None`.
Raises:
ValueError: if either the runner or options argument is not
of the expected type.
"""
# Initializing logging configuration in case the user did not set it up.
logging.basicConfig()
if options is not None:
if isinstance(options, PipelineOptions):
self._options = options
else:
raise ValueError(
'Parameter options, if specified, must be of type PipelineOptions. '
'Received : %r' % options)
elif argv is not None:
if isinstance(argv, list):
self._options = PipelineOptions(argv)
else:
raise ValueError(
'Parameter argv, if specified, must be a list. Received : %r' %
argv)
else:
self._options = PipelineOptions([])
FileSystems.set_options(self._options)
pickle_library = self._options.view_as(SetupOptions).pickle_library
pickler.set_library(pickle_library)
if runner is None:
runner = self._options.view_as(StandardOptions).runner
if runner is None:
runner = StandardOptions.DEFAULT_RUNNER
logging.info((
'Missing pipeline option (runner). Executing pipeline '
'using the default runner: %s.'),
runner)
if isinstance(runner, str):
runner = create_runner(runner)
elif not isinstance(runner, PipelineRunner):
raise TypeError(
'Runner %s is not a PipelineRunner object or the '
'name of a registered runner.' % runner)
# Validate pipeline options
errors = PipelineOptionsValidator(self._options, runner).validate()
if errors:
raise ValueError(
'Pipeline has validations errors: \n' + '\n'.join(errors))
# set default experiments for portable runners
# (needs to occur prior to pipeline construction)
if runner.is_fnapi_compatible():
experiments = (self._options.view_as(DebugOptions).experiments or [])
if not 'beam_fn_api' in experiments:
experiments.append('beam_fn_api')
self._options.view_as(DebugOptions).experiments = experiments
self.local_tempdir = tempfile.mkdtemp(prefix='beam-pipeline-temp')
# Default runner to be used.
self.runner = runner
# Stack of transforms generated by nested apply() calls. The stack will
# contain a root node as an enclosing (parent) node for top transforms.
self.transforms_stack = [AppliedPTransform(None, None, '', None)]
# Set of transform labels (full labels) applied to the pipeline.
# If a transform is applied and the full label is already in the set
# then the transform will have to be cloned with a new label.
self.applied_labels = set() # type: Set[str]
# Hints supplied via pipeline options are considered the outermost hints.
self._root_transform().resource_hints = resource_hints_from_options(options)
# Create a ComponentIdMap for assigning IDs to components. Ensures that any
# components that receive an ID during pipeline construction (for example in
# ExternalTransform), will receive the same component ID when generating the
# full pipeline proto.
self.component_id_map = ComponentIdMap()
# Records whether this pipeline contains any external transforms.
self.contains_external_transforms = False
@property # type: ignore[misc] # decorated property not supported
@deprecated(
since='First stable release',
extra_message='References to <pipeline>.options'
' will not be supported')
def options(self):
# type: () -> PipelineOptions
return self._options
@property
def allow_unsafe_triggers(self):
# type: () -> bool
return self._options.view_as(TypeOptions).allow_unsafe_triggers
def _current_transform(self):
# type: () -> AppliedPTransform
"""Returns the transform currently on the top of the stack."""
return self.transforms_stack[-1]
def _root_transform(self):
# type: () -> AppliedPTransform
"""Returns the root transform of the transform stack."""
return self.transforms_stack[0]
def _remove_labels_recursively(self, applied_transform):
# type: (AppliedPTransform) -> None
for part in applied_transform.parts:
if part.full_label in self.applied_labels:
self.applied_labels.remove(part.full_label)
self._remove_labels_recursively(part)
def _replace(self, override):
# type: (PTransformOverride) -> None
assert isinstance(override, PTransformOverride)
# From original transform output --> replacement transform output
output_map = {} # type: Dict[pvalue.PValue, pvalue.PValue]
output_replacements = {
} # type: Dict[AppliedPTransform, List[Tuple[pvalue.PValue, Optional[str]]]]
input_replacements = {
} # type: Dict[AppliedPTransform, Mapping[str, Union[pvalue.PBegin, pvalue.PCollection]]]
side_input_replacements = {
} # type: Dict[AppliedPTransform, List[pvalue.AsSideInput]]
class TransformUpdater(PipelineVisitor): # pylint: disable=used-before-assignment
""""A visitor that replaces the matching PTransforms."""
def __init__(self, pipeline):
# type: (Pipeline) -> None
self.pipeline = pipeline
def _replace_if_needed(self, original_transform_node):
# type: (AppliedPTransform) -> None
if override.matches(original_transform_node):
assert isinstance(original_transform_node, AppliedPTransform)
replacement_transform = (
override.get_replacement_transform_for_applied_ptransform(
original_transform_node))
if replacement_transform is original_transform_node.transform:
return
replacement_transform.side_inputs = tuple(
original_transform_node.transform.side_inputs)
replacement_transform_node = AppliedPTransform(
original_transform_node.parent,
replacement_transform,
original_transform_node.full_label,
original_transform_node.main_inputs)
# TODO(https://github.com/apache/beam/issues/21178): Merge rather
# than override.
replacement_transform_node.resource_hints = (
original_transform_node.resource_hints)
# Transform execution could depend on order in which nodes are
# considered. Hence we insert the replacement transform node to same
# index as the original transform node. Note that this operation
# removes the original transform node.
if original_transform_node.parent:
assert isinstance(original_transform_node.parent, AppliedPTransform)
parent_parts = original_transform_node.parent.parts
parent_parts[parent_parts.index(original_transform_node)] = (
replacement_transform_node)
else:
# Original transform has to be a root.
roots = self.pipeline.transforms_stack[0].parts
assert original_transform_node in roots
roots[roots.index(original_transform_node)] = (
replacement_transform_node)
inputs = override.get_replacement_inputs(original_transform_node)
if len(inputs) > 1:
transform_input = inputs
elif len(inputs) == 1:
transform_input = inputs[0]
elif len(inputs) == 0:
transform_input = pvalue.PBegin(self.pipeline)
try:
# We have to add the new AppliedTransform to the stack before
# expand() and pop it out later to make sure that parts get added
# correctly.
self.pipeline.transforms_stack.append(replacement_transform_node)
# Keeping the same label for the replaced node but recursively
# removing labels of child transforms of original transform since
# they will be replaced during the expand below. This is needed in
# case the replacement contains children that have labels that
# conflicts with labels of the children of the original.
self.pipeline._remove_labels_recursively(original_transform_node)
new_output = replacement_transform.expand(transform_input)
assert isinstance(
new_output, (dict, pvalue.PValue, pvalue.DoOutputsTuple))
if isinstance(new_output, pvalue.PValue):
new_output.element_type = None
self.pipeline._infer_result_type(
replacement_transform, inputs, new_output)
if isinstance(new_output, dict):
for new_tag, new_pcoll in new_output.items():
replacement_transform_node.add_output(new_pcoll, new_tag)
elif isinstance(new_output, pvalue.DoOutputsTuple):
replacement_transform_node.add_output(
new_output, new_output._main_tag)
else:
replacement_transform_node.add_output(new_output, new_output.tag)
# Recording updated outputs. This cannot be done in the same
# visitor since if we dynamically update output type here, we'll
# run into errors when visiting child nodes.
#
# NOTE: When replacing multiple outputs, the replacement
# PCollection tags must have a matching tag in the original
# transform.
if isinstance(new_output, pvalue.PValue):
if not new_output.producer:
new_output.producer = replacement_transform_node
output_map[original_transform_node.outputs[new_output.tag]] = \
new_output
elif isinstance(new_output, (pvalue.DoOutputsTuple, tuple)):
for pcoll in new_output:
if not pcoll.producer:
pcoll.producer = replacement_transform_node
output_map[original_transform_node.outputs[pcoll.tag]] = pcoll
elif isinstance(new_output, dict):
for tag, pcoll in new_output.items():
if not pcoll.producer:
pcoll.producer = replacement_transform_node
output_map[original_transform_node.outputs[tag]] = pcoll
finally:
self.pipeline.transforms_stack.pop()
def enter_composite_transform(self, transform_node):
# type: (AppliedPTransform) -> None
self._replace_if_needed(transform_node)
def visit_transform(self, transform_node):
# type: (AppliedPTransform) -> None
self._replace_if_needed(transform_node)
self.visit(TransformUpdater(self))
# Ensure no type information is lost.
for old, new in output_map.items():
if new.element_type == typehints.Any:
# TODO(robertwb): Perhaps take the intersection?
new.element_type = old.element_type
# Adjusting inputs and outputs
class InputOutputUpdater(PipelineVisitor): # pylint: disable=used-before-assignment
""""A visitor that records input and output values to be replaced.
Input and output values that should be updated are recorded in maps
input_replacements and output_replacements respectively.
We cannot update input and output values while visiting since that results
in validation errors.
"""
def __init__(self, pipeline):
# type: (Pipeline) -> None
self.pipeline = pipeline
def enter_composite_transform(self, transform_node):
# type: (AppliedPTransform) -> None
self.visit_transform(transform_node)
def visit_transform(self, transform_node):
# type: (AppliedPTransform) -> None
replace_output = False
for tag in transform_node.outputs:
if transform_node.outputs[tag] in output_map:
replace_output = True
break
replace_input = False
for input in transform_node.inputs:
if input in output_map:
replace_input = True
break
replace_side_inputs = False
for side_input in transform_node.side_inputs:
if side_input.pvalue in output_map:
replace_side_inputs = True
break
if replace_output:
output_replacements[transform_node] = []
for original, replacement in output_map.items():
for tag, output in transform_node.outputs.items():
if output == original:
output_replacements[transform_node].append((tag, replacement))
if replace_input:
new_inputs = {
tag: input if not input in output_map else output_map[input]
for (tag, input) in transform_node.main_inputs.items()
}
input_replacements[transform_node] = new_inputs
if replace_side_inputs:
new_side_inputs = []
for side_input in transform_node.side_inputs:
if side_input.pvalue in output_map:
side_input.pvalue = output_map[side_input.pvalue]
new_side_inputs.append(side_input)
else:
new_side_inputs.append(side_input)
side_input_replacements[transform_node] = new_side_inputs
self.visit(InputOutputUpdater(self))
for transform, output_replacement in output_replacements.items():
for tag, output in output_replacement:
transform.replace_output(output, tag=tag)
for transform, input_replacement in input_replacements.items():
transform.replace_inputs(input_replacement)
for transform, side_input_replacement in side_input_replacements.items():
transform.replace_side_inputs(side_input_replacement)
def _check_replacement(self, override):
# type: (PTransformOverride) -> None
class ReplacementValidator(PipelineVisitor):
def visit_transform(self, transform_node):
# type: (AppliedPTransform) -> None
if override.matches(transform_node):
raise RuntimeError(
'Transform node %r was not replaced as expected.' %
transform_node)
self.visit(ReplacementValidator())
def replace_all(self, replacements):
# type: (Iterable[PTransformOverride]) -> None
""" Dynamically replaces PTransforms in the currently populated hierarchy.
Currently this only works for replacements where input and output types
are exactly the same.
TODO: Update this to also work for transform overrides where input and
output types are different.
Args:
replacements (List[~apache_beam.pipeline.PTransformOverride]): a list of
:class:`~apache_beam.pipeline.PTransformOverride` objects.
"""
for override in replacements:
assert isinstance(override, PTransformOverride)
self._replace(override)
# Checking if the PTransforms have been successfully replaced. This will
# result in a failure if a PTransform that was replaced in a given override
# gets re-added in a subsequent override. This is not allowed and ordering
# of PTransformOverride objects in 'replacements' is important.
for override in replacements:
self._check_replacement(override)
def run(self, test_runner_api='AUTO'):
# type: (Union[bool, str]) -> PipelineResult
"""Runs the pipeline. Returns whatever our runner returns after running."""
# Records whether this pipeline contains any cross-language transforms.
self.contains_external_transforms = (
ExternalTransformFinder.contains_external_transforms(self))
try:
if test_runner_api == 'AUTO':
# Don't pay the cost of a round-trip if we're going to be going through
# the FnApi anyway...
is_fnapi_compatible = self.runner.is_fnapi_compatible() or (
# DirectRunner uses the Fn API for batch only
self.runner.__class__.__name__ == 'SwitchingDirectRunner' and
not self._options.view_as(StandardOptions).streaming)
# Multi-language pipelines that contain external pipeline segments may
# not be able to create a Python pipeline object graph. Hence following
# runner API check should be skipped for such pipelines.
# The InteractiveRunner relies on a constant pipeline reference, skip
# it.
test_runner_api = (
not is_fnapi_compatible and
not self.contains_external_transforms and
self.runner.__class__.__name__ != 'InteractiveRunner')
# When possible, invoke a round trip through the runner API.
if test_runner_api and self._verify_runner_api_compatible():
return Pipeline.from_runner_api(
self.to_runner_api(use_fake_coders=True),
self.runner,
self._options).run(False)
if (self._options.view_as(TypeOptions).runtime_type_check and
self._options.view_as(TypeOptions).performance_runtime_type_check):
raise RuntimeError(
'You cannot turn on runtime_type_check '
'and performance_runtime_type_check simultaneously. '
'Pick one or the other.')
if self._options.view_as(TypeOptions).runtime_type_check:
from apache_beam.typehints import typecheck
self.visit(typecheck.TypeCheckVisitor())
if self._options.view_as(TypeOptions).performance_runtime_type_check:
from apache_beam.typehints import typecheck
self.visit(typecheck.PerformanceTypeCheckVisitor())
if self._options.view_as(SetupOptions).save_main_session:
# If this option is chosen, verify we can pickle the main session early.
tmpdir = tempfile.mkdtemp()
try:
pickler.dump_session(os.path.join(tmpdir, 'main_session.pickle'))
finally:
shutil.rmtree(tmpdir)
return self.runner.run_pipeline(self, self._options)
finally:
if not is_in_ipython():
shutil.rmtree(self.local_tempdir, ignore_errors=True)
# else interactive beam handles the cleanup.
def __enter__(self):
# type: () -> Pipeline
self._extra_context = subprocess_server.JavaJarServer.beam_services(
self._options.view_as(CrossLanguageOptions).beam_services)
self._extra_context.__enter__()
return self
def __exit__(
self,
exc_type, # type: Optional[Type[BaseException]]
exc_val, # type: Optional[BaseException]
exc_tb # type: Optional[TracebackType]
):
# type: (...) -> None
try:
if not exc_type:
self.result = self.run()
self.result.wait_until_finish()
finally:
self._extra_context.__exit__(exc_type, exc_val, exc_tb)
def visit(self, visitor):
# type: (PipelineVisitor) -> None
"""Visits depth-first every node of a pipeline's DAG.
Runner-internal implementation detail; no backwards-compatibility guarantees
Args:
visitor (~apache_beam.pipeline.PipelineVisitor):
:class:`~apache_beam.pipeline.PipelineVisitor` object whose callbacks
will be called for each node visited. See
:class:`~apache_beam.pipeline.PipelineVisitor` comments.
Raises:
TypeError: if node is specified and is not a
:class:`~apache_beam.pvalue.PValue`.
~apache_beam.error.PipelineError: if node is specified and does not
belong to this pipeline instance.
"""
visited = set() # type: Set[pvalue.PValue]
self._root_transform().visit(visitor, self, visited)
def apply(
self,
transform, # type: ptransform.PTransform
pvalueish=None, # type: Optional[pvalue.PValue]
label=None # type: Optional[str]
):
# type: (...) -> pvalue.PValue
"""Applies a custom transform using the pvalueish specified.
Args:
transform (~apache_beam.transforms.ptransform.PTransform): the
:class:`~apache_beam.transforms.ptransform.PTransform` to apply.
pvalueish (~apache_beam.pvalue.PCollection): the input for the
:class:`~apache_beam.transforms.ptransform.PTransform` (typically a
:class:`~apache_beam.pvalue.PCollection`).
label (str): label of the
:class:`~apache_beam.transforms.ptransform.PTransform`.
Raises:
TypeError: if the transform object extracted from the
argument list is not a
:class:`~apache_beam.transforms.ptransform.PTransform`.
RuntimeError: if the transform object was already applied to
this pipeline and needs to be cloned in order to apply again.
"""
if isinstance(transform, ptransform._NamedPTransform):
return self.apply(
transform.transform, pvalueish, label or transform.label)
if not isinstance(transform, ptransform.PTransform):
raise TypeError("Expected a PTransform object, got %s" % transform)
if label:
# Fix self.label as it is inspected by some PTransform operations
# (e.g. to produce error messages for type hint violations).
old_label, transform.label = transform.label, label
try:
return self.apply(transform, pvalueish)
finally:
transform.label = old_label
# Attempts to alter the label of the transform to be applied only when it's
# a top-level transform so that the cell number will not be prepended to
# every child transform in a composite.
if self._current_transform() is self._root_transform():
alter_label_if_ipython(transform, pvalueish)
full_label = '/'.join(
[self._current_transform().full_label, label or
transform.label]).lstrip('/')
if full_label in self.applied_labels:
raise RuntimeError(
'A transform with label "%s" already exists in the pipeline. '
'To apply a transform with a specified label write '
'pvalue | "label" >> transform' % full_label)
self.applied_labels.add(full_label)
pvalueish, inputs = transform._extract_input_pvalues(pvalueish)
try:
if not isinstance(inputs, dict):
inputs = {str(ix): input for (ix, input) in enumerate(inputs)}
except TypeError:
raise NotImplementedError(
'Unable to extract PValue inputs from %s; either %s does not accept '
'inputs of this format, or it does not properly override '
'_extract_input_pvalues' % (pvalueish, transform))
for t, leaf_input in inputs.items():
if not isinstance(leaf_input, pvalue.PValue) or not isinstance(t, str):
raise NotImplementedError(
'%s does not properly override _extract_input_pvalues, '
'returned %s from %s' % (transform, inputs, pvalueish))
current = AppliedPTransform(
self._current_transform(), transform, full_label, inputs)
self._current_transform().add_part(current)
try:
self.transforms_stack.append(current)
type_options = self._options.view_as(TypeOptions)
if type_options.pipeline_type_check:
transform.type_check_inputs(pvalueish)
pvalueish_result = self.runner.apply(transform, pvalueish, self._options)
if type_options is not None and type_options.pipeline_type_check:
transform.type_check_outputs(pvalueish_result)
for tag, result in ptransform.get_named_nested_pvalues(pvalueish_result):
assert isinstance(result, (pvalue.PValue, pvalue.DoOutputsTuple))
# Make sure we set the producer only for a leaf node in the transform
# DAG. This way we preserve the last transform of a composite transform
# as being the real producer of the result.
if result.producer is None:
result.producer = current
# TODO(BEAM-1833): Pass full tuples dict.
self._infer_result_type(transform, tuple(inputs.values()), result)
assert isinstance(result.producer.inputs, tuple)
# The DoOutputsTuple adds the PCollection to the outputs when accessed
# except for the main tag. Add the main tag here.
if isinstance(result, pvalue.DoOutputsTuple):
current.add_output(result, result._main_tag)
continue
# If there is already a tag with the same name, increase a counter for
# the name. This can happen, for example, when a composite outputs a
# list of PCollections where all the tags are None.
base = tag
counter = 0
while tag in current.outputs:
counter += 1
tag = '%s_%d' % (base, counter)
current.add_output(result, tag)
if (type_options is not None and
type_options.type_check_strictness == 'ALL_REQUIRED' and
transform.get_type_hints().output_types is None):
ptransform_name = '%s(%s)' % (transform.__class__.__name__, full_label)
raise TypeCheckError(
'Pipeline type checking is enabled, however no '
'output type-hint was found for the '
'PTransform %s' % ptransform_name)
finally:
self.transforms_stack.pop()
return pvalueish_result
def _infer_result_type(
self,
transform, # type: ptransform.PTransform
inputs, # type: Sequence[Union[pvalue.PBegin, pvalue.PCollection]]
result_pcollection # type: Union[pvalue.PValue, pvalue.DoOutputsTuple]
):
# type: (...) -> None
# TODO(robertwb): Multi-input inference.
type_options = self._options.view_as(TypeOptions)
if type_options is None or not type_options.pipeline_type_check:
return
if (isinstance(result_pcollection, pvalue.PCollection) and
(not result_pcollection.element_type
# TODO(robertwb): Ideally we'd do intersection here.
or result_pcollection.element_type == typehints.Any)):
# {Single, multi}-input, single-output inference.
input_element_types_tuple = tuple(i.element_type for i in inputs)
input_element_type = (
input_element_types_tuple[0] if len(input_element_types_tuple) == 1
else typehints.Union[input_element_types_tuple])
type_hints = transform.get_type_hints()
declared_output_type = type_hints.simple_output_type(transform.label)
if declared_output_type:
input_types = type_hints.input_types
if input_types and input_types[0]:
declared_input_type = input_types[0][0]
result_element_type = typehints.bind_type_variables(
declared_output_type,
typehints.match_type_variables(
declared_input_type, input_element_type))
else:
result_element_type = declared_output_type
else:
result_element_type = transform.infer_output_type(input_element_type)
# Any remaining type variables have no bindings higher than this scope.
result_pcollection.element_type = typehints.bind_type_variables(
result_element_type, {'*': typehints.Any})
elif isinstance(result_pcollection, pvalue.DoOutputsTuple):
# {Single, multi}-input, multi-output inference.
# TODO(https://github.com/apache/beam/issues/18957): Add support for
# tagged type hints.
# https://github.com/apache/beam/pull/9810#discussion_r338765251
for pcoll in result_pcollection:
if pcoll.element_type is None:
pcoll.element_type = typehints.Any
def __reduce__(self):
# type: () -> Tuple[Type, Tuple[str, ...]]
# Some transforms contain a reference to their enclosing pipeline,
# which in turn reference all other transforms (resulting in quadratic
# time/space to pickle each transform individually). As we don't
# require pickled pipelines to be executable, break the chain here.
return str, ('Pickled pipeline stub.', )
def _verify_runner_api_compatible(self):
# type: () -> bool
if self._options.view_as(TypeOptions).runtime_type_check:
# This option is incompatible with the runner API as it requires
# the runner to inspect non-serialized hints on the transform
# itself.
return False
class Visitor(PipelineVisitor): # pylint: disable=used-before-assignment
ok = True # Really a nonlocal.
def enter_composite_transform(self, transform_node):
# type: (AppliedPTransform) -> None
pass
def visit_transform(self, transform_node):
# type: (AppliedPTransform) -> None
try:
# Transforms must be picklable.
pickler.loads(
pickler.dumps(transform_node.transform, enable_trace=False),
enable_trace=False)
except Exception:
Visitor.ok = False
def visit_value(self, value, _):
# type: (pvalue.PValue, AppliedPTransform) -> None
if isinstance(value, pvalue.PDone):
Visitor.ok = False
self.visit(Visitor())
return Visitor.ok
def to_runner_api(
self,
return_context=False, # type: bool
context=None, # type: Optional[PipelineContext]
use_fake_coders=False, # type: bool
default_environment=None # type: Optional[environments.Environment]
):
# type: (...) -> beam_runner_api_pb2.Pipeline
"""For internal use only; no backwards-compatibility guarantees."""
from apache_beam.runners import pipeline_context
if context is None:
context = pipeline_context.PipelineContext(
use_fake_coders=use_fake_coders,
component_id_map=self.component_id_map,
default_environment=default_environment)
elif default_environment is not None:
raise ValueError(
'Only one of context or default_environment may be specified.')
# The RunnerAPI spec requires certain transforms and side-inputs to have KV
# inputs (and corresponding outputs).
# Currently we only upgrade to KV pairs. If there is a need for more
# general shapes, potential conflicts will have to be resolved.
# We also only handle single-input, and (for fixing the output) single
# output, which is sufficient.
# Also marks such values as requiring deterministic key coders.
deterministic_key_coders = not self._options.view_as(
TypeOptions).allow_non_deterministic_key_coders
class ForceKvInputTypes(PipelineVisitor):
def enter_composite_transform(self, transform_node):
# type: (AppliedPTransform) -> None
self.visit_transform(transform_node)
def visit_transform(self, transform_node):
# type: (AppliedPTransform) -> None
if not transform_node.transform:
return
if transform_node.transform.runner_api_requires_keyed_input():
pcoll = transform_node.inputs[0]
pcoll.element_type = typehints.coerce_to_kv_type(
pcoll.element_type, transform_node.full_label)
pcoll.requires_deterministic_key_coder = (
deterministic_key_coders and transform_node.full_label)
if len(transform_node.outputs) == 1:
# The runner often has expectations about the output types as well.
output, = transform_node.outputs.values()
if not output.element_type:
output.element_type = transform_node.transform.infer_output_type(
pcoll.element_type)
if (isinstance(output.element_type,
typehints.TupleHint.TupleConstraint) and
len(output.element_type.tuple_types) == 2 and
pcoll.element_type.tuple_types[0] ==
output.element_type.tuple_types[0]):
output.requires_deterministic_key_coder = (
deterministic_key_coders and transform_node.full_label)
for side_input in transform_node.transform.side_inputs:
if side_input.requires_keyed_input():
side_input.pvalue.element_type = typehints.coerce_to_kv_type(
side_input.pvalue.element_type,
transform_node.full_label,
side_input_producer=side_input.pvalue.producer.full_label)
side_input.pvalue.requires_deterministic_key_coder = (
deterministic_key_coders and transform_node.full_label)
self.visit(ForceKvInputTypes())
# Mutates context; placing inline would force dependence on
# argument evaluation order.
root_transform_id = context.transforms.get_id(self._root_transform())
proto = beam_runner_api_pb2.Pipeline(
root_transform_ids=[root_transform_id],
components=context.to_runner_api(),
requirements=context.requirements())
proto.components.transforms[root_transform_id].unique_name = (
root_transform_id)
self.merge_compatible_environments(proto)
if return_context:
return proto, context # type: ignore # too complicated for now
else:
return proto
@staticmethod
def merge_compatible_environments(proto):
"""Tries to minimize the number of distinct environments by merging
those that are compatible (currently defined as identical).
Mutates proto as contexts may have references to proto.components.
"""
env_map = {}
canonical_env = {}
files_by_hash = {}
for env_id, env in proto.components.environments.items():
# First deduplicate any file dependencies by their hash.
for dep in env.dependencies:
if dep.type_urn == common_urns.artifact_types.FILE.urn:
file_payload = beam_runner_api_pb2.ArtifactFilePayload.FromString(
dep.type_payload)
if file_payload.sha256:
if file_payload.sha256 in files_by_hash:
file_payload.path = files_by_hash[file_payload.sha256]
dep.type_payload = file_payload.SerializeToString()
else:
files_by_hash[file_payload.sha256] = file_payload.path
# Next check if we've ever seen this environment before.
normalized = env.SerializeToString(deterministic=True)
if normalized in canonical_env:
env_map[env_id] = canonical_env[normalized]
else:
canonical_env[normalized] = env_id
for old_env, new_env in env_map.items():
for transform in proto.components.transforms.values():
if transform.environment_id == old_env:
transform.environment_id = new_env
for windowing_strategy in proto.components.windowing_strategies.values():
if windowing_strategy.environment_id == old_env:
windowing_strategy.environment_id = new_env
del proto.components.environments[old_env]
@staticmethod
def from_runner_api(
proto, # type: beam_runner_api_pb2.Pipeline
runner, # type: PipelineRunner
options, # type: PipelineOptions
return_context=False, # type: bool
):
# type: (...) -> Pipeline
"""For internal use only; no backwards-compatibility guarantees."""
p = Pipeline(runner=runner, options=options)
from apache_beam.runners import pipeline_context
context = pipeline_context.PipelineContext(
proto.components, requirements=proto.requirements)
if proto.root_transform_ids:
root_transform_id, = proto.root_transform_ids
p.transforms_stack = [context.transforms.get_by_id(root_transform_id)]
else:
p.transforms_stack = [AppliedPTransform(None, None, '', None)]
# TODO(robertwb): These are only needed to continue construction. Omit?
p.applied_labels = {
t.unique_name
for t in proto.components.transforms.values()
}
for id in proto.components.pcollections:
pcollection = context.pcollections.get_by_id(id)
pcollection.pipeline = p
if not pcollection.producer:
raise ValueError('No producer for %s' % id)
# Inject PBegin input where necessary.
from apache_beam.io.iobase import Read
from apache_beam.transforms.core import Create
has_pbegin = [Read, Create]