Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

introduce flags for customizing standard providers using their own YA… #34158

Open
wants to merge 1 commit into
base: master
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
3 changes: 2 additions & 1 deletion sdks/python/apache_beam/yaml/integration_tests.py
Original file line number Diff line number Diff line change
Expand Up @@ -165,7 +165,8 @@ def test(self, providers=providers): # default arg to capture loop value
stack.enter_context(
mock.patch(
'apache_beam.yaml.yaml_provider.standard_providers',
lambda: providers))
lambda _,
__: providers))
for fixture in spec.get('fixtures', []):
vars[fixture['name']] = stack.enter_context(
python_callable.PythonCallableWithSource.
Expand Down
12 changes: 11 additions & 1 deletion sdks/python/apache_beam/yaml/main.py
Original file line number Diff line number Diff line change
Expand Up @@ -90,6 +90,14 @@ def _parse_arguments(argv):
type=json.loads,
help='A json dict of variables used when invoking the jinja preprocessor '
'on the provided yaml pipeline.')
parser.add_argument(
'--standard_io_yaml_file',
default=None,
help='The YAML file contains definitions for stanard I/O providers.')
parser.add_argument(
'--standard_providers_yaml_file',
default=None,
help='The YAML file contains definitions for other standard providers.')
return parser.parse_known_args(argv)


Expand Down Expand Up @@ -151,7 +159,9 @@ def run(argv=None):
p,
pipeline_spec,
validate_schema=known_args.json_schema_validation,
pipeline_path=known_args.yaml_pipeline_file)
pipeline_path=known_args.yaml_pipeline_file,
standard_io_yaml_file=known_args.standard_io_yaml_file,
standard_providers_yaml_file=known_args.standard_providers_yaml_file)
print("Running pipeline...")


Expand Down
18 changes: 14 additions & 4 deletions sdks/python/apache_beam/yaml/yaml_provider.py
Original file line number Diff line number Diff line change
Expand Up @@ -1348,21 +1348,31 @@ def merge_providers(*provider_sets) -> Mapping[str, Iterable[Provider]]:
return result


def standard_providers():
def standard_providers(
standard_io_yaml_file=None, standard_providers_yaml_file=None):
from apache_beam.yaml.yaml_combine import create_combine_providers
from apache_beam.yaml.yaml_mapping import create_mapping_providers
from apache_beam.yaml.yaml_join import create_join_providers
from apache_beam.yaml.yaml_io import io_providers

if standard_io_yaml_file:
standard_io_providers = load_providers(standard_io_yaml_file)
else:
standard_io_providers = io_providers()
if standard_providers_yaml_file:
standard_providers = load_providers(standard_providers_yaml_file)
else:
standard_providers = load_providers(
os.path.join(os.path.dirname(__file__), 'standard_providers.yaml'))

return merge_providers(
YamlProviders.create_builtin_provider(),
create_java_builtin_provider(),
create_mapping_providers(),
create_combine_providers(),
create_join_providers(),
io_providers(),
load_providers(
os.path.join(os.path.dirname(__file__), 'standard_providers.yaml')))
standard_io_providers,
standard_providers)


def _file_digest(fileobj, digest):
Expand Down
19 changes: 15 additions & 4 deletions sdks/python/apache_beam/yaml/yaml_transform.py
Original file line number Diff line number Diff line change
Expand Up @@ -1009,7 +1009,12 @@ def expand_jinja(


class YamlTransform(beam.PTransform):
def __init__(self, spec, providers={}): # pylint: disable=dangerous-default-value
def __init__( # pylint: disable=dangerous-default-value
self,
spec,
providers={},
standard_io_yaml_file=None,
standard_providers_yaml_file=None):
if isinstance(spec, str):
spec = yaml.load(spec, Loader=SafeLineLoader)
if isinstance(providers, dict):
Expand All @@ -1019,7 +1024,9 @@ def __init__(self, spec, providers={}): # pylint: disable=dangerous-default-val
}
# TODO(BEAM-26941): Validate as a transform.
self._providers = yaml_provider.merge_providers(
providers, yaml_provider.standard_providers())
providers,
yaml_provider.standard_providers(
standard_io_yaml_file, standard_providers_yaml_file))
self._spec = preprocess(spec, known_transforms=self._providers.keys())
self._was_chain = spec['type'] == 'chain'

Expand Down Expand Up @@ -1074,7 +1081,9 @@ def expand_pipeline(
pipeline_spec,
providers=None,
validate_schema='generic' if jsonschema is not None else None,
pipeline_path=''):
pipeline_path='',
standard_io_yaml_file=None,
standard_providers_yaml_file=None):
if isinstance(pipeline_spec, str):
pipeline_spec = yaml.load(pipeline_spec, Loader=SafeLineLoader)
# TODO(robertwb): It's unclear whether this gives as good of errors, but
Expand All @@ -1087,4 +1096,6 @@ def expand_pipeline(
yaml_provider.merge_providers(
yaml_provider.parse_providers(
pipeline_path, pipeline_spec.get('providers', [])),
providers or {})).expand(beam.pvalue.PBegin(pipeline))
providers or {}),
standard_io_yaml_file,
standard_providers_yaml_file).expand(beam.pvalue.PBegin(pipeline))
Loading