Skip to content

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

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

Closed
wants to merge 1 commit into from
Closed
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
2 changes: 1 addition & 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,7 @@ 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
21 changes: 20 additions & 1 deletion sdks/python/apache_beam/yaml/main.py
Original file line number Diff line number Diff line change
Expand Up @@ -93,6 +93,23 @@ def _parse_arguments(argv):
return parser.parse_known_args(argv)


def _parse_overrides_arguments(argv):
parser = argparse.ArgumentParser()
parser.add_argument(
'--pipeline_schema_yaml_file',
default=None,
help='The YAML file contains definitions for pipeline schema.')
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)


def _pipeline_spec_from_args(known_args):
if known_args.yaml_pipeline_file and known_args.yaml_pipeline:
raise ValueError(
Expand Down Expand Up @@ -124,6 +141,7 @@ def _fix_xlang_instant_coding():
def run(argv=None):
argv = _preparse_jinja_flags(argv)
known_args, pipeline_args = _parse_arguments(argv)
overrides_args, pipeline_args = _parse_overrides_arguments(pipeline_args)
pipeline_template = _pipeline_spec_from_args(known_args)
pipeline_yaml = yaml_transform.expand_jinja(
pipeline_template, known_args.jinja_variables or {})
Expand Down Expand Up @@ -151,7 +169,8 @@ 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,
pipeline_overrides=vars(overrides_args)) # convert Namespace to dict.
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(overrides=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 overrides and overrides.get('standard_io_yaml_file'):
standard_io_providers = load_providers(overrides['standard_io_yaml_file'])
else:
standard_io_providers = io_providers()
if overrides and overrides.get('standard_providers_yaml_file'):
other_standard_providers = load_providers(
overrides['standard_providers_yaml_file'])
else:
other_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,
other_standard_providers)


def _file_digest(fileobj, digest):
Expand Down
27 changes: 17 additions & 10 deletions sdks/python/apache_beam/yaml/yaml_transform.py
Original file line number Diff line number Diff line change
Expand Up @@ -52,9 +52,8 @@


@functools.lru_cache
def pipeline_schema(strictness):
with open(os.path.join(os.path.dirname(__file__),
'pipeline.schema.yaml')) as yaml_file:
def pipeline_schema(schema_file, strictness):
with open(schema_file) as yaml_file:
pipeline_schema = yaml.safe_load(yaml_file)
if strictness == 'per_transform':
transform_schemas_path = os.path.join(
Expand All @@ -80,9 +79,15 @@ def _closest_line(o, path):
return best_line


def validate_against_schema(pipeline, strictness):
def validate_against_schema(pipeline, strictness, pipeline_overrides=None):
try:
jsonschema.validate(pipeline, pipeline_schema(strictness))
if pipeline_overrides and pipeline_overrides.get(
'pipeline_schema_yaml_file'):
schema_file = pipeline_overrides['pipeline_schema_yaml_file']
else:
schema_file = os.path.join(
os.path.dirname(__file__), 'pipeline.schema.yaml')
jsonschema.validate(pipeline, pipeline_schema(schema_file, strictness))
except jsonschema.ValidationError as exn:
exn.message += f" around line {_closest_line(pipeline, exn.path)}"
raise exn
Expand Down Expand Up @@ -1009,7 +1014,7 @@ def expand_jinja(


class YamlTransform(beam.PTransform):
def __init__(self, spec, providers={}): # pylint: disable=dangerous-default-value
def __init__(self, spec, providers={}, pipeline_overrides=None): # pylint: disable=dangerous-default-value
if isinstance(spec, str):
spec = yaml.load(spec, Loader=SafeLineLoader)
if isinstance(providers, dict):
Expand All @@ -1019,7 +1024,7 @@ 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(pipeline_overrides))
self._spec = preprocess(spec, known_transforms=self._providers.keys())
self._was_chain = spec['type'] == 'chain'

Expand Down Expand Up @@ -1074,17 +1079,19 @@ def expand_pipeline(
pipeline_spec,
providers=None,
validate_schema='generic' if jsonschema is not None else None,
pipeline_path=''):
pipeline_path='',
pipeline_overrides=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
# this could certainly be handy as a first pass when Beam is not available.
if validate_schema and validate_schema != 'none':
validate_against_schema(pipeline_spec, validate_schema)
validate_against_schema(pipeline_spec, validate_schema, pipeline_overrides)
# Calling expand directly to avoid outer layer of nesting.
return YamlTransform(
pipeline_as_composite(pipeline_spec['pipeline']),
yaml_provider.merge_providers(
yaml_provider.parse_providers(
pipeline_path, pipeline_spec.get('providers', [])),
providers or {})).expand(beam.pvalue.PBegin(pipeline))
providers or {}),
pipeline_overrides).expand(beam.pvalue.PBegin(pipeline))
Loading