-
Notifications
You must be signed in to change notification settings - Fork 4.5k
Expand file tree
/
Copy pathyaml_provider.py
More file actions
executable file
·1715 lines (1439 loc) · 58.7 KB
/
yaml_provider.py
File metadata and controls
executable file
·1715 lines (1439 loc) · 58.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.
#
"""This module defines Providers usable from yaml, which is a specification
for where to find and how to invoke services that vend implementations of
various PTransforms."""
import abc
import collections
import functools
import hashlib
import inspect
import json
import logging
import os
import re
import shutil
import subprocess
import sys
import tempfile
import urllib.parse
import warnings
from collections.abc import Callable
from collections.abc import Iterable
from collections.abc import Mapping
from typing import Any
from typing import Optional
from typing import Union
import docstring_parser
import yaml
import apache_beam as beam
import apache_beam.dataframe.io
import apache_beam.io
import apache_beam.transforms.util
from apache_beam import ManagedReplacement
from apache_beam.io.filesystems import FileSystems
from apache_beam.portability.api import schema_pb2
from apache_beam.runners import pipeline_context
from apache_beam.testing.util import assert_that
from apache_beam.testing.util import equal_to
from apache_beam.transforms import external
from apache_beam.transforms import window
from apache_beam.transforms.fully_qualified_named_transform import FullyQualifiedNamedTransform
from apache_beam.typehints import schemas
from apache_beam.typehints import trivial_inference
from apache_beam.typehints.schemas import named_tuple_to_schema
from apache_beam.typehints.schemas import typing_to_runner_api
from apache_beam.utils import python_callable
from apache_beam.utils import subprocess_server
from apache_beam.version import __version__ as beam_version
from apache_beam.yaml import json_utils
from apache_beam.yaml import yaml_utils
from apache_beam.yaml.yaml_errors import maybe_with_exception_handling_transform_fn
_LOGGER = logging.getLogger(__name__)
class NotAvailableWithReason:
"""A False value that provides additional content.
Primarily used to return a value from Provider.available().
"""
def __init__(self, reason):
self.reason = reason
def __bool__(self):
return False
class Provider(abc.ABC):
"""Maps transform types names and args to concrete PTransform instances."""
@abc.abstractmethod
def available(self) -> Union[bool, NotAvailableWithReason]:
"""Returns whether this provider is available to use in this environment."""
raise NotImplementedError(type(self))
@abc.abstractmethod
def cache_artifacts(self) -> Optional[Iterable[str]]:
raise NotImplementedError(type(self))
@abc.abstractmethod
def provided_transforms(self) -> Iterable[str]:
"""Returns a list of transform type names this provider can handle."""
raise NotImplementedError(type(self))
def config_schema(self, type):
return None
def description(self, type):
return None
def requires_inputs(self, typ: str, args: Mapping[str, Any]) -> bool:
"""Returns whether this transform requires inputs.
Specifically, if this returns True and inputs are not provided than an error
will be thrown.
This is best-effort, primarily for better and earlier error messages.
"""
return not typ.startswith('Read')
@abc.abstractmethod
def create_transform(
self,
typ: str,
args: Mapping[str, Any],
yaml_create_transform: Callable[
[Mapping[str, Any], Iterable[beam.PCollection]], beam.PTransform]
) -> beam.PTransform:
"""Creates a PTransform instance for the given transform type and arguments.
"""
raise NotImplementedError(type(self))
def underlying_provider(self):
"""If this provider is simply a proxy to another provider, return the
provider that should actually be used for affinity checking.
"""
return self
def affinity(self, other: "Provider"):
"""Returns a value approximating how good it would be for this provider
to be used immediately following a transform from the other provider
(e.g. to encourage fusion).
"""
# TODO(yaml): This is a very rough heuristic. Consider doing better.
# E.g. we could look at the expected environments themselves.
# Possibly, we could provide multiple expansions and have the runner itself
# choose the actual implementation based on fusion (and other) criteria.
a = self.underlying_provider()
b = other.underlying_provider()
return a._affinity(b) + b._affinity(a)
def _affinity(self, other: "Provider"):
if self is other or self == other:
return 100
elif type(self) == type(other):
return 10
else:
return 0
@functools.cache # pylint: disable=method-cache-max-size-none
def with_extra_dependencies(self, dependencies: Iterable[str]):
result = self._with_extra_dependencies(dependencies)
if not hasattr(result, 'to_json'):
result.to_json = lambda: {'type': type(result).__name__}
return result
def _with_extra_dependencies(self, dependencies: Iterable[str]):
raise ValueError(
'This provider of type %s does not support additional dependencies.' %
type(self).__name__)
def as_provider(name, provider_or_constructor):
if isinstance(provider_or_constructor, Provider):
return provider_or_constructor
else:
return InlineProvider({name: provider_or_constructor})
def as_provider_list(name, lst):
if not isinstance(lst, list):
return as_provider_list(name, [lst])
return [as_provider(name, x) for x in lst]
class ExternalProvider(Provider):
"""A Provider implemented via the cross language transform service."""
_provider_types: dict[str, Callable[..., Provider]] = {}
def __init__(self, urns, service, managed_replacement=None):
"""Initializes the ExternalProvider.
Args:
urns: a set of URNs that uniquely identify the transforms supported.
service: the gradle target that identified the expansion service jar.
managed_replacement (Optional): a map that defines the transform for
which the SDK may replace the transform with an available managed
transform.
"""
self._urns = urns
self._service = service
self._schema_transforms = None
self._managed_replacement = managed_replacement
def provided_transforms(self):
return self._urns.keys()
def schema_transforms(self):
if callable(self._service):
self._service = self._service()
if self._schema_transforms is None:
try:
self._schema_transforms = {
config.identifier: config
for config in external.SchemaAwareExternalTransform.discover(
self._service, ignore_errors=True)
}
except Exception:
# It's possible this service doesn't vend schema transforms.
self._schema_transforms = {}
return self._schema_transforms
def config_schema(self, type):
if self._urns[type] in self.schema_transforms():
return named_tuple_to_schema(
self.schema_transforms()[self._urns[type]].configuration_schema)
def description(self, type):
if self._urns[type] in self.schema_transforms():
return self.schema_transforms()[self._urns[type]].description
def requires_inputs(self, typ, args):
if self._urns[typ] in self.schema_transforms():
return bool(self.schema_transforms()[self._urns[typ]].inputs)
else:
return super().requires_inputs(typ, args)
def create_transform(self, type, args, yaml_create_transform):
if callable(self._service):
self._service = self._service()
urn = self._urns[type]
if urn in self.schema_transforms():
managed_replacement = None
if self._managed_replacement and type in self._managed_replacement:
managed_replacement = ManagedReplacement(
underlying_transform_identifier=urn,
update_compatibility_version=self._managed_replacement[type])
return external.SchemaAwareExternalTransform(
urn,
self._service,
rearrange_based_on_discovery=True,
managed_replacement=managed_replacement,
**args)
else:
return type >> self.create_external_transform(urn, args)
def create_external_transform(self, urn, args):
return external.ExternalTransform(
urn,
external.ImplicitSchemaPayloadBuilder(args).payload(),
self._service)
@classmethod
def provider_from_spec(cls, source_path, spec):
from apache_beam.yaml.yaml_transform import SafeLineLoader
for required in ('type', 'transforms'):
if required not in spec:
raise ValueError(
f'Missing {required} in provider '
f'at line {SafeLineLoader.get_line(spec)}')
urns = SafeLineLoader.strip_metadata(spec['transforms'])
type = spec['type']
config = SafeLineLoader.strip_metadata(spec.get('config', {}))
extra_params = set(SafeLineLoader.strip_metadata(spec).keys()) - {
'transforms', 'type', 'config'
}
if extra_params:
raise ValueError(
f'Unexpected parameters in provider of type {type} '
f'at line {SafeLineLoader.get_line(spec)}: {extra_params}')
if config.get('version', None) == 'BEAM_VERSION':
config['version'] = beam_version
if type in cls._provider_types:
try:
constructor = cls._provider_types[type]
if 'provider_base_path' in inspect.signature(constructor).parameters:
config['provider_base_path'] = source_path
result = constructor(urns, **config)
if not hasattr(result, 'to_json'):
result.to_json = lambda: spec
return result
except Exception as exn:
raise ValueError(
f'Unable to instantiate provider of type {type} '
f'at line {SafeLineLoader.get_line(spec)}: {exn}') from exn
else:
raise NotImplementedError(
f'Unknown provider type: {type} '
f'at line {SafeLineLoader.get_line(spec)}.')
@classmethod
def register_provider_type(cls, type_name):
def apply(constructor):
cls._provider_types[type_name] = constructor
return constructor
return apply
@ExternalProvider.register_provider_type('javaJar')
def java_jar(urns, provider_base_path, jar: str):
if not os.path.exists(jar):
parsed = urllib.parse.urlparse(jar)
if not parsed.scheme or not parsed.netloc:
raise ValueError(f'Invalid path or url: {jar}')
return ExternalJavaProvider(
urns, lambda: _join_url_or_filepath(provider_base_path, jar))
@ExternalProvider.register_provider_type('mavenJar')
def maven_jar(
urns,
*,
artifact_id,
group_id,
version,
repository=subprocess_server.JavaJarServer.MAVEN_CENTRAL_REPOSITORY,
classifier=None,
appendix=None):
return ExternalJavaProvider(
urns, lambda: subprocess_server.JavaJarServer.path_to_maven_jar(
artifact_id=artifact_id, group_id=group_id, version=version,
repository=repository, classifier=classifier, appendix=appendix))
@ExternalProvider.register_provider_type('beamJar')
def beam_jar(
urns,
*,
gradle_target,
managed_replacement=None,
appendix=None,
version=beam_version,
artifact_id=None):
return ExternalJavaProvider(
urns, lambda: subprocess_server.JavaJarServer.path_to_beam_jar(
gradle_target=gradle_target, version=version, artifact_id=artifact_id
),
managed_replacement=managed_replacement)
@ExternalProvider.register_provider_type('docker')
def docker(urns, **config):
raise NotImplementedError()
@ExternalProvider.register_provider_type('remote')
class RemoteProvider(ExternalProvider):
_is_available = None
def __init__(self, urns, address: str):
super().__init__(urns, service=address)
self._address = address
def available(self):
if self._is_available is None:
try:
with external.ExternalTransform.service(self._service) as service:
service.ready(1)
self._is_available = True
except Exception:
self._is_available = NotAvailableWithReason(
f'Remote provider not reachable at {self._address}.')
return self._is_available
def cache_artifacts(self):
pass
class ExternalJavaProvider(ExternalProvider):
def __init__(
self, urns, jar_provider, managed_replacement=None, classpath=None):
super().__init__(
urns, lambda: external.JavaJarExpansionService(
jar_provider(), classpath=classpath),
managed_replacement)
self._jar_provider = jar_provider
self._classpath = classpath
def available(self):
# Directly use shutil.which to find the Java executable cross-platform
java_path = shutil.which(subprocess_server.JavaHelper.get_java())
if java_path:
return True
# Return error message when not found
return NotAvailableWithReason(
'Unable to locate java executable: java not found in PATH or JAVA_HOME')
def cache_artifacts(self):
return [self._jar_provider()]
def _with_extra_dependencies(self, dependencies: Iterable[str]):
jars = sum((
external.JavaJarExpansionService._expand_jars(dep)
for dep in dependencies), [])
return ExternalJavaProvider(
self._urns,
jar_provider=self._jar_provider,
classpath=(list(self._classpath or []) + list(jars)))
@ExternalProvider.register_provider_type('python')
def python(urns, provider_base_path, packages=()):
if packages:
return ExternalPythonProvider(urns, provider_base_path, packages)
else:
return InlineProvider({
name: python_callable.PythonCallableWithSource.load_from_source(
constructor)
for (name, constructor) in urns.items()
})
@ExternalProvider.register_provider_type('pythonPackage')
class ExternalPythonProvider(ExternalProvider):
def __init__(self, urns, provider_base_path, packages: Iterable[str]):
def is_path_or_urn(package):
return (
'/' in package or urllib.parse.urlparse(package).scheme or
os.path.exists(package))
super().__init__(
urns,
PypiExpansionService([
_join_url_or_filepath(provider_base_path, package)
if is_path_or_urn(package) else package for package in packages
]))
self._packages = packages
def available(self):
return True # If we're running this script, we have Python installed.
def cache_artifacts(self):
return [self._service._venv()]
def create_external_transform(self, urn, args):
# Python transforms are "registered" by fully qualified name.
if not re.match(r'^[\w.]*$', urn):
# Treat it as source.
args = {'source': urn, **args}
urn = '__constructor__'
return external.ExternalTransform(
"beam:transforms:python:fully_qualified_named",
external.ImplicitSchemaPayloadBuilder({
'constructor': urn,
'kwargs': args,
}).payload(),
self._service)
def _affinity(self, other: "Provider"):
if isinstance(other, InlineProvider):
return 50
else:
return super()._affinity(other)
def _with_extra_dependencies(self, dependencies: Iterable[str]):
return ExternalPythonProvider(
self._urns, None, set(self._packages).union(set(dependencies)))
@ExternalProvider.register_provider_type('yaml')
class YamlProvider(Provider):
def __init__(self, transforms: Mapping[str, Mapping[str, Any]]):
if not isinstance(transforms, dict):
raise ValueError('Transform mapping must be a dict.')
self._transforms = transforms
def available(self):
return True
def cache_artifacts(self):
pass
def provided_transforms(self):
return self._transforms.keys()
def config_schema(self, type):
return json_utils.json_schema_to_beam_schema(self.json_config_schema(type))
def json_config_schema(self, type):
return dict(
type='object',
additionalProperties=False,
**self._transforms[type].get('config_schema', {}))
def description(self, type):
return self._transforms[type].get('description')
def requires_inputs(self, type, args):
return self._transforms[type].get(
'requires_inputs', super().requires_inputs(type, args))
def create_transform(
self,
type: str,
args: Mapping[str, Any],
yaml_create_transform: Callable[
[Mapping[str, Any], Iterable[beam.PCollection]], beam.PTransform]
) -> beam.PTransform:
from apache_beam.yaml.yaml_transform import SafeLineLoader
from apache_beam.yaml.yaml_transform import expand_jinja
from apache_beam.yaml.yaml_transform import preprocess
spec = self._transforms[type]
try:
import jsonschema
jsonschema.validate(args, self.json_config_schema(type))
except ImportError:
warnings.warn(
'Please install jsonschema '
f'for better provider validation of "{type}"')
body = spec['body']
# Stringify to apply jinja.
if isinstance(body, str):
body_str = body
else:
body_str = yaml.safe_dump(SafeLineLoader.strip_metadata(body))
# Now re-parse resolved templatization.
body = yaml.load(expand_jinja(body_str, args), Loader=SafeLineLoader)
if (body.get('type') == 'chain' and 'input' not in body and
spec.get('requires_inputs', True)):
body['input'] = 'input'
return yaml_create_transform(preprocess(body)) # type: ignore
# This is needed because type inference can't handle *args, **kwargs forwarding.
# TODO(BEAM-24755): Add support for type inference of through kwargs calls.
def fix_pycallable():
from apache_beam.transforms.ptransform import label_from_callable
def default_label(self):
src = self._source.strip()
last_line = src.split('\n')[-1]
if last_line[0] != ' ' and len(last_line) < 72:
return last_line
return label_from_callable(self._callable)
def _argspec_fn(self):
return self._callable
python_callable.PythonCallableWithSource.default_label = default_label
python_callable.PythonCallableWithSource._argspec_fn = property(_argspec_fn)
original_infer_return_type = trivial_inference.infer_return_type
def infer_return_type(fn, *args, **kwargs):
if isinstance(fn, python_callable.PythonCallableWithSource):
fn = fn._callable
return original_infer_return_type(fn, *args, **kwargs)
trivial_inference.infer_return_type = infer_return_type
original_fn_takes_side_inputs = (
apache_beam.transforms.util.fn_takes_side_inputs)
def fn_takes_side_inputs(fn):
if isinstance(fn, python_callable.PythonCallableWithSource):
fn = fn._callable
return original_fn_takes_side_inputs(fn)
apache_beam.transforms.util.fn_takes_side_inputs = fn_takes_side_inputs
class InlineProvider(Provider):
def __init__(self, transform_factories, no_input_transforms=()):
self._transform_factories = transform_factories
self._no_input_transforms = set(no_input_transforms)
def available(self):
return True
def cache_artifacts(self):
pass
def provided_transforms(self):
return self._transform_factories.keys()
def config_schema(self, typ):
return self.config_schema_from_callable(self._transform_factories[typ])
@classmethod
def config_schema_from_callable(cls, factory):
if isinstance(factory, type) and issubclass(factory, beam.PTransform):
# https://bugs.python.org/issue40897
params = dict(inspect.signature(factory.__init__).parameters)
if 'self' in params:
del params['self']
else:
params = inspect.signature(factory).parameters
def type_of(p):
t = p.annotation
if t == p.empty:
return Any
else:
return t
docs = {
param.arg_name: param.description
for param in cls.get_docs(factory).params
}
names_and_types = [(name, typing_to_runner_api(type_of(p)))
for name, p in params.items()]
return schema_pb2.Schema(
fields=[
schema_pb2.Field(name=name, type=type, description=docs.get(name))
for (name, type) in names_and_types
])
def description(self, typ):
return self.description_from_callable(self._transform_factories[typ])
@classmethod
def description_from_callable(cls, factory):
def empty_if_none(s):
return s or ''
docs = cls.get_docs(factory)
return (
empty_if_none(docs.short_description) +
('\n\n' if docs.blank_after_short_description else '\n') +
empty_if_none(docs.long_description)).strip() or None
@classmethod
def get_docs(cls, factory):
docstring = factory.__doc__ or ''
# These "extra" docstring parameters are not relevant for YAML and mess
# up the parsing.
docstring = re.sub(
r'Pandas Parameters\s+-----.*', '', docstring, flags=re.S)
return docstring_parser.parse(
docstring, docstring_parser.DocstringStyle.GOOGLE)
def create_transform(self, type, args, yaml_create_transform):
return self._transform_factories[type](**args)
def to_json(self):
return {'type': "InlineProvider"}
def requires_inputs(self, typ, args):
if typ in self._no_input_transforms:
return False
elif hasattr(self._transform_factories[typ], '_yaml_requires_inputs'):
return self._transform_factories[typ]._yaml_requires_inputs
else:
return super().requires_inputs(typ, args)
def _with_extra_dependencies(self, dependencies):
external_provider = ExternalPythonProvider( # disable yapf
{
typ: 'apache_beam.yaml.yaml_provider.standard_inline_providers.' +
typ.replace('-', '_')
for typ in self._transform_factories.keys()
},
'__inline__',
dependencies)
external_provider.to_json = self.to_json
return external_provider
class MetaInlineProvider(InlineProvider):
def create_transform(self, type, args, yaml_create_transform):
return self._transform_factories[type](yaml_create_transform, **args)
# Note: This function is used to override the default provider by some
# users, so a change here will be breaking to those users. Change with
# caution.
def get_default_sql_provider():
return beam_jar(
urns={'Sql': 'beam:external:java:sql:v1'},
gradle_target='sdks:java:extensions:sql:expansion-service:shadowJar')
class SqlBackedProvider(Provider):
def __init__(
self,
transforms: Mapping[str, Callable[..., beam.PTransform]],
sql_provider: Optional[Provider] = None):
self._transforms = transforms
if sql_provider is None:
sql_provider = get_default_sql_provider()
self._sql_provider = sql_provider
def sql_provider(self):
return self._sql_provider
def provided_transforms(self):
return self._transforms.keys()
def config_schema(self, type):
full_config = InlineProvider.config_schema_from_callable(
self._transforms[type])
# Omit the (first) query -> transform parameter.
return schema_pb2.Schema(fields=full_config.fields[1:])
def description(self, type):
return InlineProvider.description_from_callable(self._transforms[type])
def available(self):
return self.sql_provider().available()
def cache_artifacts(self):
return self.sql_provider().cache_artifacts()
def underlying_provider(self):
return self.sql_provider()
def to_json(self):
return {'type': "SqlBackedProvider"}
def create_transform(
self, typ: str, args: Mapping[str, Any],
yaml_create_transform: Any) -> beam.PTransform:
return self._transforms[typ](
lambda query: self.sql_provider().create_transform(
'Sql', {'query': query}, yaml_create_transform),
**args)
PRIMITIVE_NAMES_TO_ATOMIC_TYPE = {
py_type.__name__: schema_type
for (py_type, schema_type) in schemas.PRIMITIVE_TO_ATOMIC_TYPE.items()
if py_type.__module__ != 'typing'
}
def element_to_rows(e):
if isinstance(e, dict):
return dicts_to_rows(e)
else:
return beam.Row(element=dicts_to_rows(e))
def dicts_to_rows(o):
if isinstance(o, dict):
return beam.Row(**{k: dicts_to_rows(v) for k, v in o.items()})
elif isinstance(o, list):
return [dicts_to_rows(e) for e in o]
else:
return o
def _unify_element_with_schema(element, target_schema):
"""Convert an element to match the target schema, preserving existing
fields only."""
if target_schema is None:
return element
# If element is already a named tuple, convert to dict first
if hasattr(element, '_asdict'):
element_dict = element._asdict()
elif isinstance(element, dict):
element_dict = element
else:
# This element is not a row-like object. If the target schema has a single
# field, assume this element is the value for that field.
if len(target_schema._fields) == 1:
return target_schema(**{target_schema._fields[0]: element})
else:
return element
# Create new element with only the fields that exist in the original
# element plus None for fields that are expected but missing
unified_dict = {}
for field_name in target_schema._fields:
if field_name in element_dict:
value = element_dict[field_name]
# Ensure the value matches the expected type
# This is particularly important for list fields
if value is not None and not isinstance(value, list) and hasattr(
value, '__iter__') and not isinstance(
value, (str, bytes)) and not hasattr(value, '_asdict'):
# Convert iterables to lists if needed
unified_dict[field_name] = list(value)
else:
unified_dict[field_name] = value
else:
unified_dict[field_name] = None
return target_schema(**unified_dict)
class YamlProviders:
class AssertEqual(beam.PTransform):
"""Asserts that the input contains exactly the elements provided.
This is primarily used for testing; it will cause the entire pipeline to
fail if the input to this transform is not exactly the set of `elements`
given in the config parameter.
As with Create, YAML/JSON-style mappings are interpreted as Beam rows,
e.g.::
type: AssertEqual
input: SomeTransform
config:
elements:
- {a: 0, b: "foo"}
- {a: 1, b: "bar"}
would ensure that `SomeTransform` produced exactly two elements with values
`(a=0, b="foo")` and `(a=1, b="bar")` respectively.
Args:
elements: The set of elements that should belong to the PCollection.
YAML/JSON-style mappings will be interpreted as Beam rows.
"""
def __init__(self, elements: Iterable[Any]):
self._elements = elements
def expand(self, pcoll):
def to_dict(row):
# filter None when comparing
temp_dict = {k: v for k, v in row._asdict().items() if v is not None}
return dict(temp_dict.items())
return assert_that(
pcoll | beam.Map(to_dict),
equal_to([to_dict(e) for e in dicts_to_rows(self._elements)]))
@staticmethod
def create(elements: Iterable[Any], reshuffle: Optional[bool] = True):
"""Creates a collection containing a specified set of elements.
This transform always produces schema'd data. For example::
type: Create
config:
elements: [1, 2, 3]
will result in an output with three elements with a schema of
Row(element=int) whereas YAML/JSON-style mappings will be interpreted
directly as Beam rows, e.g.::
type: Create
config:
elements:
- {first: 0, second: {str: "foo", values: [1, 2, 3]}}
- {first: 1, second: {str: "bar", values: [4, 5, 6]}}
will result in a schema of the form (int, Row(string, list[int])).
This can also be expressed as YAML::
type: Create
config:
elements:
- first: 0
second:
str: "foo"
values: [1, 2, 3]
- first: 1
second:
str: "bar"
values: [4, 5, 6]
If the elements are a mix of dicts and non-dicts, the non-dict elements
will be wrapped in a Row with a single field "element". For example::
type: Create
config:
elements: [1, {"a": 2}]
will result in an output with two elements with a schema of
Row(element=int, a=int) looking like:
Row(element=1, a=None)
Row(element=None, a=2)
Args:
elements: The set of elements that should belong to the PCollection.
YAML/JSON-style mappings will be interpreted as Beam rows.
Primitives will be mapped to rows with a single "element" field.
reshuffle: (optional) Whether to introduce a reshuffle (to possibly
redistribute the work) if there is more than one element in the
collection. Defaults to True.
"""
# Though str and dict are technically iterable, we disallow them
# as using the characters or keys respectively is almost certainly
# not the intent.
if not isinstance(elements, Iterable) or isinstance(elements, (dict, str)):
raise TypeError('elements must be a list of elements')
if elements:
# Normalize elements to be all dicts or all primitives.
has_dict = False
has_non_dict = False
for e in elements:
if isinstance(e, dict):
has_dict = True
else:
has_non_dict = True
if has_dict and has_non_dict:
break
if has_dict and has_non_dict:
elements = [
e if isinstance(e, dict) else {
'element': e
} for e in elements
]
# Check if elements have different keys
updated_elements = elements
if elements and all(isinstance(e, dict) for e in elements):
keys = [set(e.keys()) for e in elements]
if len(set.union(*keys)) > min(len(k) for k in keys):
# Merge all dictionaries to get all possible keys
all_keys = set()
for element in elements:
if isinstance(element, dict):
all_keys.update(element.keys())
# Create a merged dictionary with all keys
merged_dict = {}
for key in all_keys:
merged_dict[key] = None # Use None as a default value
# Update each element with the merged dictionary
updated_elements = []
for e in elements:
if isinstance(e, dict):
updated_elements.append({**merged_dict, **e})
else:
updated_elements.append(e)
return beam.Create([element_to_rows(e) for e in updated_elements],
reshuffle=reshuffle is not False)
# Or should this be posargs, args?
# pylint: disable=dangerous-default-value
@staticmethod
def fully_qualified_named_transform(
constructor: str,
args: Optional[Iterable[Any]] = (),
kwargs: Optional[Mapping[str, Any]] = {}):
"""A Python PTransform identified by fully qualified name.
This allows one to import, construct, and apply any Beam Python transform.
This can be useful for using transforms that have not yet been exposed
via a YAML interface. Note, however, that conversion may be required if this
transform does not accept or produce Beam Rows.
For example::
type: PyTransform
config:
constructor: apache_beam.pkg.mod.SomeClass
args: [1, 'foo']
kwargs:
baz: 3
can be used to access the transform
`apache_beam.pkg.mod.SomeClass(1, 'foo', baz=3)`.
See also the documentation on
[Inlining
Python](https://beam.apache.org/documentation/sdks/yaml-inline-python/).
Args:
constructor: Fully qualified name of a callable used to construct the
transform. Often this is a class such as
`apache_beam.pkg.mod.SomeClass` but it can also be a function or
any other callable that returns a PTransform.
args: A list of parameters to pass to the callable as positional
arguments.
kwargs: A list of parameters to pass to the callable as keyword
arguments.
"""
with FullyQualifiedNamedTransform.with_filter('*'):
return constructor >> FullyQualifiedNamedTransform(
constructor, args, kwargs)
# This intermediate is needed because there is no way to specify a tuple of
# exactly zero or one PCollection in yaml (as they would be interpreted as
# PBegin and the PCollection itself respectively).
class Flatten(beam.PTransform):
"""Flattens multiple PCollections into a single PCollection.
The elements of the resulting PCollection will be the (disjoint) union of
all the elements of all the inputs.
Note that in YAML transforms can always take a list of inputs which will
be implicitly flattened.
"""
def __init__(self):
# Suppress the "label" argument from the superclass for better docs.
# pylint: disable=useless-parent-delegation
super().__init__()