twang126 commented on code in PR #26451:
URL: https://github.com/apache/beam/pull/26451#discussion_r1180666428
##########
sdks/python/apache_beam/yaml/yaml_transform_test.py:
##########
@@ -166,6 +166,121 @@ def test_csv_to_json(self):
pd.testing.assert_frame_equal(data, result)
+class CreateTimestamped(beam.PTransform):
+ def __init__(self, elements):
+ self._elements = elements
+
+ def expand(self, p):
+ return (
+ p
+ | beam.Create(self._elements)
+ | beam.Map(lambda x: beam.transforms.window.TimestampedValue(x, x)))
+
+
+class SumGlobally(beam.PTransform):
+ def expand(self, pcoll):
+ return pcoll | beam.CombineGlobally(sum).without_defaults()
+
+
+TEST_PROVIDERS = {
+ 'CreateTimestamped': CreateTimestamped, 'SumGlobally': SumGlobally
+}
+
+
+class YamlWindowingTest(unittest.TestCase):
+ def test_explicit_window_into(self):
+ with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions(
+ pickle_library='cloudpickle')) as p:
+ result = p | YamlTransform(
+ '''
+ type: chain
+ transforms:
+ - type: CreateTimestamped
+ elements: [0, 1, 2, 3, 4, 5]
+ - type: WindowInto
+ windowing:
+ type: fixed
+ size: 4
+ - type: SumGlobally
+ ''',
+ providers=TEST_PROVIDERS)
+ assert_that(result, equal_to([6, 9]))
+
+ def test_windowing_on_input(self):
+ with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions(
+ pickle_library='cloudpickle')) as p:
+ result = p | YamlTransform(
+ '''
+ type: chain
+ transforms:
+ - type: CreateTimestamped
+ elements: [0, 1, 2, 3, 4, 5]
+ - type: SumGlobally
+ windowing:
+ type: fixed
+ size: 4
+ ''',
+ providers=TEST_PROVIDERS)
+ assert_that(result, equal_to([6, 9]))
+
+ def test_windowing_multiple_inputs(self):
+ with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions(
+ pickle_library='cloudpickle')) as p:
+ result = p | YamlTransform(
+ '''
+ type: composite
+ transforms:
+ - type: CreateTimestamped
+ name: Create1
+ elements: [0, 2, 4]
+ - type: CreateTimestamped
+ name: Create2
+ elements: [1, 3, 5]
+ - type: SumGlobally
+ input: [Create1, Create2]
+ windowing:
+ type: fixed
+ size: 4
+ output: SumGlobally
+ ''',
+ providers=TEST_PROVIDERS)
+ assert_that(result, equal_to([6, 9]))
+
+ def test_windowing_on_output(self):
+ with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions(
+ pickle_library='cloudpickle')) as p:
+ result = p | YamlTransform(
+ '''
+ type: chain
+ transforms:
+ - type: CreateTimestamped
+ elements: [0, 1, 2, 3, 4, 5]
+ windowing:
+ type: fixed
+ size: 4
+ - type: SumGlobally
+ ''',
+ providers=TEST_PROVIDERS)
+ assert_that(result, equal_to([6, 9]))
+
+ def test_windowing_on_outer(self):
+ with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions(
+ pickle_library='cloudpickle')) as p:
+ result = p | YamlTransform(
+ '''
+ type: chain
+ transforms:
+ - type: CreateTimestamped
+ elements: [0, 1, 2, 3, 4, 5]
+ - type: SumGlobally
+ windowing:
+ type: fixed
+ size: 4
+ ''',
+ providers=TEST_PROVIDERS)
+ assert_that(result, equal_to([6, 9]))
+
Review Comment:
Maybe worth adding a test where there's multiple windows in a pipeline (if
its supported)? e.g. re-windowing one of the outputs of an aggregation.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]