This is an automated email from the ASF dual-hosted git repository. robertwb pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/beam.git
The following commit(s) were added to refs/heads/master by this push: new 80d85aa38ff Add docs for YAML AssertThat. (#31448) 80d85aa38ff is described below commit 80d85aa38ff91699a5123f14d5c5df96d826140c Author: Robert Bradshaw <rober...@gmail.com> AuthorDate: Wed May 29 15:59:26 2024 -0700 Add docs for YAML AssertThat. (#31448) This is the first transform in the (alphabetical) list, so it'd be good to not have it empty. Also produce slightly nicer examples for repeated arguments. --- sdks/python/apache_beam/yaml/generate_yaml_docs.py | 24 +++++++++++++++++++-- sdks/python/apache_beam/yaml/yaml_provider.py | 25 +++++++++++++++++++++- 2 files changed, 46 insertions(+), 3 deletions(-) diff --git a/sdks/python/apache_beam/yaml/generate_yaml_docs.py b/sdks/python/apache_beam/yaml/generate_yaml_docs.py index b11062cce4d..4719bc3e66a 100644 --- a/sdks/python/apache_beam/yaml/generate_yaml_docs.py +++ b/sdks/python/apache_beam/yaml/generate_yaml_docs.py @@ -28,6 +28,18 @@ from apache_beam.yaml import json_utils from apache_beam.yaml import yaml_provider +def _singular(name): + # Simply removing an 's' (or 'es', or 'ies', ...) may result in surprising + # manglings. Better to play it safe and leave a correctly-spelled plural + # than a botched singular in our examples configs. + return { + 'args': 'arg', + 'attributes': 'attribute', + 'elements': 'element', + 'fields': 'field', + }.get(name, name) + + def _fake_value(name, beam_type): type_info = beam_type.WhichOneof("type_info") if type_info == "atomic_type": @@ -38,9 +50,17 @@ def _fake_value(name, beam_type): else: return name elif type_info == "array_type": - return [_fake_value(name, beam_type.array_type.element_type), '...'] + return [ + _fake_value(_singular(name), beam_type.array_type.element_type), + _fake_value(_singular(name), beam_type.array_type.element_type), + '...' + ] elif type_info == "iterable_type": - return [_fake_value(name, beam_type.iterable_type.element_type), '...'] + return [ + _fake_value(_singular(name), beam_type.iterable_type.element_type), + _fake_value(_singular(name), beam_type.iterable_type.element_type), + '...' + ] elif type_info == "map_type": if beam_type.map_type.key_type.atomic_type == schema_pb2.STRING: return { diff --git a/sdks/python/apache_beam/yaml/yaml_provider.py b/sdks/python/apache_beam/yaml/yaml_provider.py index 5f53302028c..52452daff7e 100755 --- a/sdks/python/apache_beam/yaml/yaml_provider.py +++ b/sdks/python/apache_beam/yaml/yaml_provider.py @@ -557,7 +557,30 @@ def dicts_to_rows(o): class YamlProviders: class AssertEqual(beam.PTransform): - def __init__(self, elements): + """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):