Skip to content
Merged
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
21 changes: 16 additions & 5 deletions sdks/python/apache_beam/yaml/yaml_testing.py
Original file line number Diff line number Diff line change
Expand Up @@ -411,6 +411,13 @@ def create_test(
**yaml_transform.SafeLineLoader.strip_metadata(
pipeline_spec.get('options', {})))

providers = yaml_provider.merge_providers(
yaml_provider.parse_providers('', pipeline_spec.get('providers', [])),
{
'AssertEqualAndRecord': yaml_provider.as_provider_list(
'AssertEqualAndRecord', AssertEqualAndRecord)
})

def get_name(transform):
if 'name' in transform:
return str(transform['name'])
Expand All @@ -428,7 +435,8 @@ def get_name(transform):
mock_outputs = [{
'name': get_name(t),
'elements': [
_try_row_as_dict(row) for row in _first_n(t, options, max_num_inputs)
_try_row_as_dict(row)
for row in _first_n(t, options, max_num_inputs, providers)
],
} for t in input_transforms]

Expand Down Expand Up @@ -504,15 +512,18 @@ def record(element):
return pcoll | beam.Map(record)


def _first_n(transform_spec, options, n):
def _first_n(transform_spec, options, n, providers=None):
recorder = RecordElements(n)
if providers is None:
providers = {
'AssertEqualAndRecord': yaml_provider.as_provider_list(
'AssertEqualAndRecord', AssertEqualAndRecord)
}
try:
with beam.Pipeline(options=options) as p:
_ = (
p
| yaml_transform.YamlTransform(
transform_spec,
providers={'AssertEqualAndRecord': AssertEqualAndRecord})
| yaml_transform.YamlTransform(transform_spec, providers=providers)
| recorder)
except _DoneException:
pass
Expand Down
39 changes: 39 additions & 0 deletions sdks/python/apache_beam/yaml/yaml_testing_test.py
Original file line number Diff line number Diff line change
Expand Up @@ -356,6 +356,45 @@ def test_toplevel_providers(self):
}]
})

def test_create_with_external_providers(self):
"""Test that create_test works with external providers defined in the
pipeline spec.

This test validates the fix for issue #37136 where external providers
defined in YAML files were not recognized when running tests.
"""
pipeline = '''
pipeline:
type: chain
transforms:
- type: Create
config:
elements:
- {a: 1, b: 2}
- {a: 2, b: 3}
- {a: 3, b: 4}
- {a: 4, b: 5}
- {a: 5, b: 6}
- type: MyCustomTransform
- type: LogForTesting
providers:
- type: yaml
transforms:
MyCustomTransform:
body:
type: MapToFields
config:
language: python
fields:
sum_ab: a + b
'''
test_spec = yaml_testing.create_test(
pipeline, max_num_inputs=10, min_num_outputs=3)

self.assertEqual(len(test_spec['expected_inputs']), 1)
self.assertGreaterEqual(len(test_spec['expected_inputs'][0]['elements']), 3)
yaml_testing.run_test(pipeline, test_spec)


if __name__ == '__main__':
logging.getLogger().setLevel(logging.INFO)
Expand Down
Loading