-
Notifications
You must be signed in to change notification settings - Fork 4.5k
Expand file tree
/
Copy pathyaml_transform.py
More file actions
1493 lines (1299 loc) · 52.7 KB
/
yaml_transform.py
File metadata and controls
1493 lines (1299 loc) · 52.7 KB
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.
#
import collections
import datetime
import functools
import json
import logging
import os
import pprint
import re
from collections.abc import Iterable
from collections.abc import Mapping
from typing import Any
import jinja2
import yaml
import apache_beam as beam
from apache_beam.io.filesystems import FileSystems
from apache_beam.options.pipeline_options import GoogleCloudOptions
from apache_beam.transforms.fully_qualified_named_transform import FullyQualifiedNamedTransform
from apache_beam.typehints import schemas
from apache_beam.typehints import typehints
from apache_beam.yaml import json_utils
from apache_beam.yaml import yaml_provider
from apache_beam.yaml import yaml_utils
from apache_beam.yaml.yaml_combine import normalize_combine
from apache_beam.yaml.yaml_mapping import Validate
from apache_beam.yaml.yaml_mapping import normalize_mapping
from apache_beam.yaml.yaml_mapping import validate_generic_expressions
from apache_beam.yaml.yaml_utils import SafeLineLoader
__all__ = ["YamlTransform"]
_LOGGER = logging.getLogger(__name__)
yaml_provider.fix_pycallable()
try:
import jsonschema
except ImportError:
jsonschema = None
@functools.lru_cache
def pipeline_schema(strictness):
with open(yaml_utils.locate_data_file('pipeline.schema.yaml')) as yaml_file:
pipeline_schema = yaml.safe_load(yaml_file)
if strictness == 'per_transform':
transform_schemas_path = yaml_utils.locate_data_file(
'transforms.schema.yaml')
if not os.path.exists(transform_schemas_path):
raise RuntimeError(
"Please run "
"python -m apache_beam.yaml.generate_yaml_docs "
f"--schema_file='{transform_schemas_path}' "
"to run with transform-specific validation.")
with open(transform_schemas_path) as fin:
pipeline_schema['$defs']['transform']['allOf'].extend(yaml.safe_load(fin))
return pipeline_schema
def _closest_line(o, path):
best_line = SafeLineLoader.get_line(o)
for step in path:
o = o[step]
maybe_line = SafeLineLoader.get_line(o)
if maybe_line != 'unknown':
best_line = maybe_line
return best_line
def validate_against_schema(pipeline, strictness):
try:
jsonschema.validate(pipeline, pipeline_schema(strictness))
except jsonschema.ValidationError as exn:
exn.message += f" around line {_closest_line(pipeline, exn.path)}"
# validation message for chain-type transform
if (exn.schema_path[-1] == 'not' and
exn.schema_path[-2] in ['input', 'output']):
exn.message = (
f"'{exn.schema_path[-2]}' should not be used "
"along with 'chain' type transforms. " + exn.message)
raise exn
def memoize_method(func):
def wrapper(self, *args):
if not hasattr(self, '_cache'):
self._cache = {}
key = func.__name__, args
if key not in self._cache:
self._cache[key] = func(self, *args)
return self._cache[key]
return wrapper
def only_element(xs):
x, = xs
return x
# These allow a user to explicitly pass no input to a transform (i.e. use it
# as a root transform) without an error even if the transform is not known to
# handle it.
def explicitly_empty():
return {'__explicitly_empty__': None}
def is_explicitly_empty(io):
return io == explicitly_empty()
def is_empty(io):
return not io or is_explicitly_empty(io)
def empty_if_explicitly_empty(io):
if is_explicitly_empty(io):
return {}
else:
return io
class LightweightScope(object):
def __init__(self, transforms):
self._transforms = transforms
self._transforms_by_uuid = {t['__uuid__']: t for t in self._transforms}
self._uuid_by_name = collections.defaultdict(set)
for spec in self._transforms:
if 'name' in spec:
self._uuid_by_name[spec['name']].add(spec['__uuid__'])
if 'type' in spec:
self._uuid_by_name[spec['type']].add(spec['__uuid__'])
def get_transform_id_and_output_name(self, name):
if '.' in name:
transform_name, output = name.rsplit('.', 1)
else:
transform_name, output = name, None
return self.get_transform_id(transform_name), output
def get_transform_id(self, transform_name):
if transform_name in self._transforms_by_uuid:
return transform_name
else:
candidates = self._uuid_by_name[transform_name]
if not candidates:
raise ValueError(
f'Unknown transform at line '
f'{SafeLineLoader.get_line(transform_name)}: {transform_name}')
elif len(candidates) > 1:
raise ValueError(
f'Ambiguous transform at line '
f'{SafeLineLoader.get_line(transform_name)}: {transform_name}')
else:
return only_element(candidates)
def get_transform_spec(self, transform_name_or_id):
return self._transforms_by_uuid[self.get_transform_id(transform_name_or_id)]
class Scope(LightweightScope):
"""To look up PCollections (typically outputs of prior transforms) by name."""
def __init__(
self,
root,
inputs: Mapping[str, Any],
transforms: Iterable[dict],
providers: Mapping[str, Iterable[yaml_provider.Provider]],
input_providers: Iterable[yaml_provider.Provider]):
super().__init__(transforms)
self.root = root
self._inputs = inputs
self.providers = providers
self._seen_names: set[str] = set()
self.input_providers = input_providers
self._all_followers = None
def followers(self, transform_name):
if self._all_followers is None:
self._all_followers = collections.defaultdict(list)
# TODO(yaml): Also trace through outputs and composites.
for transform in self._transforms:
if transform['type'] != 'composite':
for input in empty_if_explicitly_empty(transform['input']).values():
if input not in self._inputs:
transform_id, _ = self.get_transform_id_and_output_name(input)
self._all_followers[transform_id].append(transform['__uuid__'])
return self._all_followers[self.get_transform_id(transform_name)]
def compute_all(self):
for transform_id in self._transforms_by_uuid.keys():
self.compute_outputs(transform_id)
def get_pcollection(self, name):
if name in self._inputs:
return self._inputs[name]
elif '.' in name:
transform, output = name.rsplit('.', 1)
outputs = self.get_outputs(transform)
if output in outputs:
return outputs[output]
elif len(outputs) == 1 and outputs[next(iter(outputs))].tag == output:
return outputs[next(iter(outputs))]
else:
raise ValueError(
f'Unknown output {repr(output)} '
f'at line {SafeLineLoader.get_line(name)}: '
f'{transform} only has outputs {list(outputs.keys())}')
else:
outputs = self.get_outputs(name)
if len(outputs) == 1:
return only_element(outputs.values())
else:
error_output = self._transforms_by_uuid[self.get_transform_id(
name)]['config'].get('error_handling', {}).get('output')
if error_output and error_output in outputs and len(outputs) == 2:
return next(
output for tag, output in outputs.items() if tag != error_output)
raise ValueError(
f'Ambiguous output at line {SafeLineLoader.get_line(name)}: '
f'{name} has outputs {list(outputs.keys())}')
def get_outputs(self, transform_name):
return self.compute_outputs(self.get_transform_id(transform_name))
@memoize_method
def compute_outputs(self, transform_id):
return expand_transform(self._transforms_by_uuid[transform_id], self)
def best_provider(
self, t, input_providers: yaml_provider.Iterable[yaml_provider.Provider]):
if isinstance(t, dict):
spec = t
else:
spec = self._transforms_by_uuid[self.get_transform_id(t)]
possible_providers = []
unavailable_provider_messages = []
for p in self.providers[spec['type']]:
is_available = p.available()
if is_available:
possible_providers.append(p)
else:
reason = getattr(is_available, 'reason', 'no reason given')
unavailable_provider_messages.append(
f'{p.__class__.__name__} ({reason})')
if not possible_providers:
if unavailable_provider_messages:
unavailable_provider_message = (
'\nThe following providers were found but not available: ' +
'\n'.join(unavailable_provider_messages))
else:
unavailable_provider_message = ''
raise ValueError(
'No available provider for type %r at %s%s' %
(spec['type'], identify_object(spec), unavailable_provider_message))
# From here on, we have the invariant that possible_providers is not empty.
# Only one possible provider, no need to rank further.
if len(possible_providers) == 1:
return possible_providers[0]
def best_matches(
possible_providers: Iterable[yaml_provider.Provider],
adjacent_provider_options: Iterable[Iterable[yaml_provider.Provider]]
) -> list[yaml_provider.Provider]:
"""Given a set of possible providers, and a set of providers for each
adjacent transform, returns the top possible providers as ranked by
affinity to the adjacent transforms' providers.
"""
providers_by_score = collections.defaultdict(list)
for p in possible_providers:
# The sum of the affinity of the best provider
# for each adjacent transform.
providers_by_score[sum(
max(p.affinity(ap) for ap in apo)
for apo in adjacent_provider_options)].append(p)
return providers_by_score[max(providers_by_score.keys())]
# If there are any inputs, prefer to match them.
if input_providers:
possible_providers = best_matches(
possible_providers, [[p] for p in input_providers])
# Without __uuid__ we can't find downstream operations.
if '__uuid__' not in spec:
return possible_providers[0]
# Match against downstream transforms, continuing until there is no tie
# or we run out of downstream transforms.
if len(possible_providers) > 1:
adjacent_transforms = list(self.followers(spec['__uuid__']))
while adjacent_transforms:
# This is a list of all possible providers for each adjacent transform.
adjacent_provider_options = [[
p for p in self.providers[self._transforms_by_uuid[t]['type']]
if p.available()
] for t in adjacent_transforms]
if any(not apo for apo in adjacent_provider_options):
# One of the transforms had no available providers.
# We will throw an error later, doesn't matter what we return.
break
# Filter down the set of possible providers to the best ones.
possible_providers = best_matches(
possible_providers, adjacent_provider_options)
# If we are down to one option, no need to go further.
if len(possible_providers) == 1:
break
# Go downstream one more step.
adjacent_transforms = sum(
[list(self.followers(t)) for t in adjacent_transforms], [])
return possible_providers[0]
# A method on scope as providers may be scoped...
def create_ptransform(self, spec, input_pcolls):
def maybe_with_resource_hints(transform):
if 'resource_hints' in spec:
return transform.with_resource_hints(
**SafeLineLoader.strip_metadata(spec['resource_hints']))
else:
return transform
if 'type' not in spec:
raise ValueError(f'Missing transform type: {identify_object(spec)}')
if spec['type'] == 'composite':
class _CompositeTransformStub(beam.PTransform):
@staticmethod
def expand(pcolls):
if isinstance(pcolls, beam.PCollection):
pcolls = {'input': pcolls}
elif isinstance(pcolls, beam.pvalue.PBegin):
pcolls = {}
inner_scope = Scope(
self.root,
pcolls,
spec['transforms'],
self.providers,
self.input_providers)
inner_scope.compute_all()
if '__implicit_outputs__' in spec['output']:
return inner_scope.get_outputs(
spec['output']['__implicit_outputs__'])
else:
return {
key: inner_scope.get_pcollection(value)
for (key, value) in spec['output'].items()
}
return maybe_with_resource_hints(_CompositeTransformStub())
if spec['type'] not in self.providers:
raise ValueError(
'Unknown transform type %r at %s' %
(spec['type'], identify_object(spec)))
# TODO(yaml): Perhaps we can do better than a greedy choice here.
# TODO(yaml): Figure out why this is needed.
providers_by_input = {k: v for k, v in self.input_providers.items()}
input_providers = [
providers_by_input[pcoll] for pcoll in input_pcolls
if pcoll in providers_by_input
]
provider = self.best_provider(spec, input_providers)
extra_dependencies, spec = extract_extra_dependencies(spec)
if extra_dependencies:
provider = provider.with_extra_dependencies(frozenset(extra_dependencies))
config = SafeLineLoader.strip_metadata(spec.get('config', {}))
if not isinstance(config, dict):
raise ValueError(
'Config for transform at %s must be a mapping.' %
identify_object(spec))
if (not input_pcolls and not is_explicitly_empty(spec.get('input', {})) and
provider.requires_inputs(spec['type'], config)):
raise ValueError(
f'Missing inputs for transform at {identify_object(spec)}')
try:
if spec['type'].endswith('-generic'):
# Centralize the validation rather than require every implementation
# to do it.
validate_generic_expressions(
spec['type'].rsplit('-', 1)[0], config, input_pcolls)
# pylint: disable=undefined-loop-variable
ptransform = maybe_with_resource_hints(
provider.create_transform(
spec['type'],
config,
lambda config, input_pcolls=input_pcolls: self.create_ptransform(
config, input_pcolls)))
# TODO(robertwb): Should we have a better API for adding annotations
# than this?
annotations = {
**{
'yaml_type': spec['type'],
'yaml_args': json.dumps(config),
'yaml_provider': json.dumps(provider.to_json())
},
**ptransform.annotations()
}
ptransform.annotations = lambda: annotations
original_expand = ptransform.expand
def recording_expand(pvalue):
result = original_expand(pvalue)
def record_providers(pvalueish):
if isinstance(pvalueish, (tuple, list)):
for p in pvalueish:
record_providers(p)
elif isinstance(pvalueish, dict):
for p in pvalueish.values():
record_providers(p)
elif isinstance(pvalueish, beam.PCollection):
if pvalueish not in self.input_providers:
self.input_providers[pvalueish] = provider
record_providers(result)
return result
ptransform.expand = recording_expand
return ptransform
except Exception as exn:
if isinstance(exn, TypeError):
# Create a slightly more generic error message for argument errors.
msg = str(exn).replace('positional', '').replace('keyword', '')
msg = re.sub(r'\S+lambda\S+', '', msg)
msg = re.sub(' +', ' ', msg).strip()
else:
msg = str(exn)
raise ValueError(
f'Invalid transform specification at {identify_object(spec)}: {msg}'
) from exn
def unique_name(self, spec, ptransform, strictness=0):
if 'name' in spec:
name = spec['name']
strictness += 1
elif ('ExternalTransform' not in ptransform.label and
not ptransform.label.startswith('_')):
# The label may have interesting information.
name = ptransform.label
else:
name = spec['type']
if name in self._seen_names:
if strictness >= 2:
raise ValueError(f'Duplicate name at {identify_object(spec)}: {name}')
else:
name = f'{name}@{SafeLineLoader.get_line(spec)}'
self._seen_names.add(name)
return name
def expand_transform(spec, scope):
if 'type' not in spec:
raise TypeError(
f'Missing type parameter for transform at {identify_object(spec)}')
type = spec['type']
if type == 'composite':
return expand_composite_transform(spec, scope)
else:
return expand_leaf_transform(spec, scope)
def expand_leaf_transform(spec, scope):
spec = spec.copy()
# Check for optional output_schema to verify on.
# The idea is to pass this output_schema config to the ValidateWithSchema
# transform.
output_schema_spec = {}
if 'output_schema' in spec.get('config', {}):
output_schema_spec = spec.get('config').pop('output_schema')
spec = normalize_inputs_outputs(spec)
inputs_dict = {
key: scope.get_pcollection(value)
for (key, value) in empty_if_explicitly_empty(spec['input']).items()
}
input_type = spec.get('input_type', 'default')
if input_type == 'list':
inputs = tuple(inputs_dict.values())
elif input_type == 'map':
inputs = inputs_dict
else:
if len(inputs_dict) == 0:
inputs = scope.root
elif len(inputs_dict) == 1:
inputs = next(iter(inputs_dict.values()))
else:
inputs = inputs_dict
_LOGGER.info("Expanding %s ", identify_object(spec))
ptransform = scope.create_ptransform(spec, inputs_dict.values())
try:
# TODO: Move validation to construction?
with FullyQualifiedNamedTransform.with_filter('*'):
outputs = inputs | scope.unique_name(spec, ptransform) >> ptransform
except Exception as exn:
raise ValueError(
f"Error applying transform {identify_object(spec)}: {exn}") from exn
# Optional output_schema was found, so lets expand on that before returning.
if output_schema_spec:
error_handling_spec = {}
# Obtain original transform error_handling_spec, so that all validate
# schema errors use that.
if 'error_handling' in spec.get('config', None):
error_handling_spec = spec.get('config').get('error_handling', {})
outputs = expand_output_schema_transform(
spec=output_schema_spec,
outputs=outputs,
error_handling_spec=error_handling_spec)
if isinstance(outputs, dict):
# TODO: Handle (or at least reject) nested case.
return outputs
elif isinstance(outputs, (tuple, list)):
return {f'out{ix}': pcoll for (ix, pcoll) in enumerate(outputs)}
elif isinstance(outputs, beam.PCollection):
return {'out': outputs}
elif outputs is None or isinstance(outputs, beam.pvalue.PDone):
return {}
else:
raise ValueError(
f'Transform {identify_object(spec)} returned an unexpected type '
f'{type(outputs)}')
def expand_output_schema_transform(spec, outputs, error_handling_spec):
"""Applies a `Validate` transform to the output of another transform.
This function is called when an `output_schema` is defined on a transform.
It wraps the original transform's output(s) with a `Validate` transform
to ensure the data conforms to the specified schema.
If the original transform has error handling configured, validation errors
will be routed to the specified error output. If not, validation failures
will cause the pipeline to fail.
Args:
spec (dict): The `output_schema` specification from the YAML config.
outputs (beam.PCollection or dict[str, beam.PCollection]): The output(s)
from the transform to be validated.
error_handling_spec (dict): The `error_handling` configuration from the
original transform.
Returns:
The validated PCollection(s). If error handling is enabled, this will be a
dictionary containing the 'good' output and any error outputs.
Raises:
ValueError: If `error_handling` is incorrectly specified within the
`output_schema` spec itself, or if the main output of a multi-output
transform cannot be determined.
"""
if 'error_handling' in spec:
raise ValueError(
'error_handling config is not supported directly in '
'the output_schema. Please use error_handling config in '
'the transform, if possible, or use ValidateWithSchema transform '
'instead.')
# Strip metadata such as __line__ and __uuid__ as these will interfere with
# the validation downstream.
clean_schema = SafeLineLoader.strip_metadata(spec)
# If no error handling is specified for the main transform, warn the user
# that the pipeline may fail if any output data fails the output schema
# validation.
if not error_handling_spec:
_LOGGER.warning("Output_schema config is attached to a transform that has "\
"no error_handling config specified. Any failures validating on output" \
"schema will fail the pipeline unless the user specifies an" \
"error_handling config on a capable transform. Alternatively, you can " \
"remove the output_schema config on this transform and add a " \
"ValidateWithSchema transform with separate error handling downstream of " \
"the current transform.")
# The transform produced outputs with a single beam.PCollection
if isinstance(outputs, beam.PCollection):
outputs = _enforce_schema(
outputs, 'EnforceOutputSchema', error_handling_spec, clean_schema)
if isinstance(outputs, dict):
main_tag = error_handling_spec.get('main_tag', 'good')
main_output = outputs.pop(main_tag)
if error_handling_spec:
error_output_tag = error_handling_spec.get('output')
if error_output_tag in outputs:
return {
'output': main_output,
error_output_tag: outputs.pop(error_output_tag)
}
return main_output
# The transform produced outputs with many named PCollections and need to
# determine which PCollection should be validated on.
elif isinstance(outputs, dict):
main_output_key = get_main_output_key(spec, outputs, error_handling_spec)
validation_result = _enforce_schema(
outputs[main_output_key],
f'EnforceOutputSchema_{main_output_key}',
error_handling_spec,
clean_schema)
outputs = _integrate_validation_results(
outputs, validation_result, main_output_key, error_handling_spec)
return outputs
def get_main_output_key(spec, outputs, error_handling_spec):
"""Determines the main output key from a dictionary of PCollections.
This is used to identify which output of a multi-output transform should be
validated against an `output_schema`.
The main output is determined using the following precedence:
1. An output with the key 'output'.
2. An output with the key 'good'.
3. The single output if there is only one.
Args:
spec: The transform specification, used for creating informative error
messages.
outputs: A dictionary mapping output tags to their corresponding
PCollections.
error_handling_spec (dict): The `error_handling` configuration from the
original transform.
Returns:
The key of the main output PCollection.
Raises:
ValueError: If a main output cannot be determined because there are
multiple outputs and none are named 'output' or 'good'.
"""
main_output_key = 'output'
if main_output_key not in outputs:
if 'good' in outputs:
main_output_key = 'good'
elif len(outputs) == 1:
main_output_key = next(iter(outputs.keys()))
else:
raise ValueError(
f"Transform {identify_object(spec)} has outputs "
f"{list(outputs.keys())}, but none are named 'output' or 'good'. To "
"apply an 'output_schema', please ensure the transform has exactly "
"one output, or that the main output is named 'output' or 'good'.")
if len(outputs) >= 3 or \
(len(outputs) == 2 and error_handling_spec.get('output') not in outputs):
_LOGGER.warning(
"There are currently %s outputs: %s. Only the main output will be "
"validated.",
len(outputs),
outputs)
return main_output_key
def _integrate_validation_results(
outputs, validation_result, main_output_key, error_handling_spec):
"""
Integrates the results of a validation transform back into the outputs of
the original transform.
This function handles merging the "good" and "bad" outputs from a
`Validate` transform with the existing outputs of the transform that was
validated.
Args:
outputs: The original dictionary of output PCollections from the transform.
validation_result: The output of the `Validate` transform. This can be a
single PCollection (if all elements passed) or a dictionary of
PCollections (if error handling was enabled for validation).
main_output_key: The key in the `outputs` dictionary corresponding to the
PCollection that was validated.
error_handling_spec: The error handling configuration of the original
transform.
Returns:
The updated dictionary of output PCollections, with validation results
integrated.
Raises:
ValueError: If the validation transform produces unexpected outputs.
"""
if not isinstance(validation_result, dict):
outputs[main_output_key] = validation_result
return outputs
# The main output from validation is the good output.
main_tag = error_handling_spec.get('main_tag', 'good')
outputs[main_output_key] = validation_result.pop(main_tag)
if error_handling_spec:
error_output_tag = error_handling_spec['output']
if error_output_tag in validation_result:
schema_error_pcoll = validation_result.pop(error_output_tag)
# The original transform also had an error output. Merge them.
outputs[error_output_tag] = (
(outputs[error_output_tag], schema_error_pcoll)
| f'FlattenErrors_{main_output_key}' >> beam.Flatten())
# There should be no other outputs from validation.
if validation_result:
raise ValueError(
"Unexpected outputs from validation: "
f"{list(validation_result.keys())}")
return outputs
def _enforce_schema(pcoll, label, error_handling_spec, clean_schema):
"""Applies schema to PCollection elements if necessary, then validates.
This function ensures that the input PCollection conforms to a specified
schema. If the PCollection is schemaless (i.e., its element_type is Any),
it attempts to convert its elements into schema-aware `beam.Row` objects
based on the provided `clean_schema`. After ensuring the PCollection has
a defined schema, it applies a `Validate` transform to perform the actual
schema validation.
Args:
pcoll: The input PCollection to be schema-enforced and validated.
label: A string label to be used for the Beam transforms created within this
function.
error_handling_spec: A dictionary specifying how to handle validation
errors.
clean_schema: A dictionary representing the schema to enforce and validate
against.
Returns:
A PCollection (or PCollectionTuple if error handling is enabled) resulting
from the `Validate` transform.
"""
if pcoll.element_type == typehints.Any:
_LOGGER.info(
"PCollection for %s has no schema (element_type=Any). "
"Converting elements to beam.Row based on provided output_schema.",
label)
try:
# Attempt to confer the schemaless elements into schema-aware beam.Row
# objects
beam_schema = json_utils.json_schema_to_beam_schema(clean_schema)
row_type_constraint = schemas.named_tuple_from_schema(beam_schema)
def to_row(element):
"""
Convert a single element into the row type constraint type.
"""
if isinstance(element, dict):
return row_type_constraint(**element)
elif hasattr(element, '_asdict'): # Handle NamedTuple, beam.Row
return row_type_constraint(**element._asdict())
else:
raise TypeError(
f"Cannot convert element of type {type(element)} to beam.Row "
f"for validation in {label}. Element: {element}")
pcoll = pcoll | f'{label}_ConvertToRow' >> beam.Map(
to_row).with_output_types(row_type_constraint)
except Exception as e:
raise ValueError(
f"Failed to prepare schemaless PCollection for \
validation in {label}: {e}") from e
# Add Validation step downstream of current transform
return pcoll | label >> Validate(
schema=clean_schema, error_handling=error_handling_spec)
def expand_composite_transform(spec, scope):
spec = normalize_inputs_outputs(normalize_source_sink(spec))
original_transforms = spec['transforms']
# Check if any transform has a NON-EMPTY explicit input or output.
# Note: {} (empty dict) means "no explicit input specified" and should
# NOT count as having explicit io.
# However, if the composite has no input, we can't do implicit chaining.
has_explicit_io = any(
io is not None and not is_empty(t.get(io, {}))
for t in original_transforms for io in ('input', 'output'))
# If the composite has no input, we can't do implicit chaining
composite_has_input = not is_empty(spec.get('input', {}))
# Only do implicit chaining if:
# 1. No transform has explicit io, AND
# 2. The composite has an input to chain from
if not has_explicit_io and composite_has_input:
new_transforms = []
for ix, transform in enumerate(original_transforms):
transform = dict(transform)
if ix == 0:
composite_input = spec.get('input', {})
if is_explicitly_empty(composite_input):
transform['input'] = composite_input
elif is_empty(composite_input):
# No explicit input - the composite input IS the pipeline input.
# Reference the 'input' key from the Scope's inputs.
transform['input'] = 'input'
else:
transform['input'] = {key: key for key in composite_input.keys()}
else:
transform['input'] = new_transforms[-1]['__uuid__']
new_transforms.append(transform)
if new_transforms:
spec = dict(spec, transforms=new_transforms)
# Check if output is empty, not just present (normalization sets it to {})
if is_empty(spec.get('output', {})):
spec['output'] = {
'__implicit_outputs__': new_transforms[-1]['__uuid__']
}
# Compute the inputs for the inner scope.
# If the composite has an empty input dict ({}), it means the composite
# should use the parent scope's inputs directly.
composite_input = spec.get('input', {})
if is_empty(composite_input):
# No explicit input - use the parent scope's inputs directly
inner_scope_inputs = dict(scope._inputs)
else:
# The composite has explicit input references
# They can reference either:
# 1. A parent scope input (e.g., 'input' key in scope._inputs)
# 2. A transform output (e.g., 'uuid' -> the output of a transform)
inner_scope_inputs = {}
for key, value in composite_input.items():
if isinstance(value, str) and value in scope._inputs:
# Reference to a parent scope input
inner_scope_inputs[key] = scope._inputs[value]
else:
# Reference to a transform output
inner_scope_inputs[key] = scope.get_pcollection(value)
inner_scope = Scope(
scope.root,
inner_scope_inputs,
spec['transforms'],
# TODO(robertwb): Are scoped providers ever used? Worth supporting?
yaml_provider.merge_providers(
yaml_provider.parse_providers('', spec.get('providers', [])),
scope.providers),
scope.input_providers)
class CompositePTransform(beam.PTransform):
@staticmethod
def expand(inputs):
inner_scope.compute_all()
if '__implicit_outputs__' in spec['output']:
result = inner_scope.get_outputs(spec['output']['__implicit_outputs__'])
return result
else:
return {
key: inner_scope.get_pcollection(value)
for (key, value) in spec['output'].items()
}
transform = CompositePTransform()
if 'resource_hints' in spec:
transform = transform.with_resource_hints(
**SafeLineLoader.strip_metadata(spec['resource_hints']))
# Always set a name for the composite to ensure proper return value
if 'name' not in spec:
spec['name'] = 'Composite'
if spec['name'] is None: # top-level pipeline, don't nest
return transform.expand(None)
else:
_LOGGER.info("Expanding %s ", identify_object(spec))
# When the input references a scope input (not a transform output),
# we need to use the scope's inputs directly
input_dict = {}
for key, value in empty_if_explicitly_empty(spec['input']).items():
if isinstance(value, str) and value in scope._inputs:
# Reference to a scope input
input_dict[key] = scope._inputs[value]
else:
# Reference to a transform output
input_dict[key] = scope.get_pcollection(value)
return (input_dict or
scope.root) | scope.unique_name(spec, None) >> transform
def expand_chain_transform(spec, scope):
return expand_composite_transform(chain_as_composite(spec), scope)
def chain_as_composite(spec):
def is_not_output_of_last_transform(new_transforms, value):
return (
('name' in new_transforms[-1] and
value != new_transforms[-1]['name']) or
('type' in new_transforms[-1] and value != new_transforms[-1]['type']))
# A chain is simply a composite transform where all inputs and outputs
# are implicit.
spec = normalize_source_sink(spec)
if 'transforms' not in spec:
raise TypeError(
f"Chain at {identify_object(spec)} missing transforms property.")
has_explicit_outputs = 'output' in spec
composite_spec = dict(normalize_inputs_outputs(tag_explicit_inputs(spec)))
new_transforms = []
for ix, transform in enumerate(composite_spec['transforms']):
transform = dict(transform)
if any(io in transform for io in ('input', 'output')):
if (ix == 0 and 'input' in transform and 'output' not in transform and
is_explicitly_empty(transform['input'])):
# This is OK as source clause sets an explicitly empty input.
pass
else:
raise ValueError(
f'Transform {identify_object(transform)} is part of a chain. '
'Cannot define explicit inputs on chain pipeline')
if ix == 0:
if is_explicitly_empty(transform.get('input', None)):
pass
elif is_explicitly_empty(composite_spec['input']):
transform['input'] = composite_spec['input']
elif is_empty(composite_spec['input']):
del composite_spec['input']
else:
transform['input'] = {
key: key
for key in composite_spec['input'].keys()
}
else:
transform['input'] = new_transforms[-1]['__uuid__']
new_transforms.append(transform)
new_transforms.extend(spec.get('extra_transforms', []))
composite_spec['transforms'] = new_transforms
last_transform = new_transforms[-1]['__uuid__']
if has_explicit_outputs:
for (key, value) in composite_spec['output'].items():
if is_not_output_of_last_transform(new_transforms, value):
raise ValueError(
f"Explicit output {identify_object(value)} of the chain transform"
f" is not an output of the last transform.")
composite_spec['output'] = {
key: f'{last_transform}.{value}'
for (key, value) in composite_spec['output'].items()
}
else:
composite_spec['output'] = {'__implicit_outputs__': last_transform}
if 'name' not in composite_spec:
composite_spec['name'] = 'Chain'
composite_spec['type'] = 'composite'
return composite_spec
def preprocess_chain(spec):
if spec['type'] == 'chain':
return chain_as_composite(spec)
else:
return spec
def pipeline_as_composite(spec):
if isinstance(spec, list):
return {
'type': 'composite',
'name': None,
'transforms': spec,
'__line__': spec[0]['__line__'],
'__uuid__': SafeLineLoader.create_uuid(),
}
else:
return dict(spec, name=None, type=spec.get('type', 'composite'))
def normalize_source_sink(spec):