From abc81cfd34c8df199ec3ac1bacb59ae489c6486f Mon Sep 17 00:00:00 2001 From: Derrick Williams Date: Thu, 8 Jan 2026 19:38:57 +0000 Subject: [PATCH 1/2] Add final exand pipeline ut --- .../yaml/yaml_transform_unit_test.py | 27 +++++++++++++++++++ 1 file changed, 27 insertions(+) diff --git a/sdks/python/apache_beam/yaml/yaml_transform_unit_test.py b/sdks/python/apache_beam/yaml/yaml_transform_unit_test.py index 59b1619b6512..5e1d0cb09897 100644 --- a/sdks/python/apache_beam/yaml/yaml_transform_unit_test.py +++ b/sdks/python/apache_beam/yaml/yaml_transform_unit_test.py @@ -1099,6 +1099,33 @@ def test_expand_pipeline_with_incorrect_pipelines_key_fails(self): with self.assertRaises(KeyError): expand_pipeline(p, spec, validate_schema=None) + def test_expand_pipeline_with_valid_schema(self): + spec = ''' + pipeline: + type: chain + transforms: + - type: Create + config: + elements: [1,2,3] + - type: LogForTesting + ''' + with new_pipeline() as p: + expand_pipeline(p, spec, validate_schema='generic') + + def test_expand_pipeline_with_invalid_schema(self): + spec = ''' + pipeline: + type: chain + transforms: + - name: Create + config: + elements: [1,2,3] + - type: LogForTesting + ''' + with new_pipeline() as p: + with self.assertRaises(jsonschema.ValidationError): + expand_pipeline(p, spec, validate_schema='generic') + if __name__ == '__main__': logging.getLogger().setLevel(logging.INFO) From 759b47579c9212f1deeb018fc5ff7fbced7e7259 Mon Sep 17 00:00:00 2001 From: Derrick Williams Date: Fri, 9 Jan 2026 23:44:57 +0000 Subject: [PATCH 2/2] add gemini recommendation --- sdks/python/apache_beam/yaml/yaml_transform_unit_test.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/sdks/python/apache_beam/yaml/yaml_transform_unit_test.py b/sdks/python/apache_beam/yaml/yaml_transform_unit_test.py index 5e1d0cb09897..f83697732598 100644 --- a/sdks/python/apache_beam/yaml/yaml_transform_unit_test.py +++ b/sdks/python/apache_beam/yaml/yaml_transform_unit_test.py @@ -1099,6 +1099,7 @@ def test_expand_pipeline_with_incorrect_pipelines_key_fails(self): with self.assertRaises(KeyError): expand_pipeline(p, spec, validate_schema=None) + @unittest.skipIf(jsonschema is None, "Yaml dependencies not installed") def test_expand_pipeline_with_valid_schema(self): spec = ''' pipeline: @@ -1112,6 +1113,7 @@ def test_expand_pipeline_with_valid_schema(self): with new_pipeline() as p: expand_pipeline(p, spec, validate_schema='generic') + @unittest.skipIf(jsonschema is None, "Yaml dependencies not installed") def test_expand_pipeline_with_invalid_schema(self): spec = ''' pipeline: