@@ -993,6 +993,61 @@ def test_explicit_window_into(self):
993993 providers = TEST_PROVIDERS )
994994 assert_that (result , equal_to ([6 , 9 ]))
995995
996+ def test_explicit_window_into_with_json_string_config_one_line (self ):
997+ with beam .Pipeline (options = beam .options .pipeline_options .PipelineOptions (
998+ pickle_library = 'cloudpickle' )) as p :
999+ result = p | YamlTransform (
1000+ '''
1001+ type: chain
1002+ transforms:
1003+ - type: CreateTimestamped
1004+ config:
1005+ elements: [0, 1, 2, 3, 4, 5]
1006+ - type: WindowInto
1007+ config:
1008+ windowing: {"type": "fixed", "size": "4s"}
1009+ - type: SumGlobally
1010+ ''' ,
1011+ providers = TEST_PROVIDERS )
1012+ assert_that (result , equal_to ([6 , 9 ]))
1013+
1014+ def test_explicit_window_into_with_json_string_config_multi_line (self ):
1015+ with beam .Pipeline (options = beam .options .pipeline_options .PipelineOptions (
1016+ pickle_library = 'cloudpickle' )) as p :
1017+ result = p | YamlTransform (
1018+ '''
1019+ type: chain
1020+ transforms:
1021+ - type: CreateTimestamped
1022+ config:
1023+ elements: [0, 1, 2, 3, 4, 5]
1024+ - type: WindowInto
1025+ config:
1026+ windowing: |
1027+ {"type": "fixed", "size": "4s"}
1028+ - type: SumGlobally
1029+ ''' ,
1030+ providers = TEST_PROVIDERS )
1031+ assert_that (result , equal_to ([6 , 9 ]))
1032+
1033+ def test_explicit_window_into_with_string_config_fails (self ):
1034+ with self .assertRaisesRegex (ValueError , 'Error parsing windowing config' ):
1035+ with beam .Pipeline (options = beam .options .pipeline_options .PipelineOptions (
1036+ pickle_library = 'cloudpickle' )) as p :
1037+ _ = p | YamlTransform (
1038+ '''
1039+ type: chain
1040+ transforms:
1041+ - type: CreateTimestamped
1042+ config:
1043+ elements: [0, 1, 2, 3, 4, 5]
1044+ - type: WindowInto
1045+ config:
1046+ windowing: |
1047+ 'not a valid yaml'
1048+ ''' ,
1049+ providers = TEST_PROVIDERS )
1050+
9961051 def test_windowing_on_input (self ):
9971052 with beam .Pipeline (options = beam .options .pipeline_options .PipelineOptions (
9981053 pickle_library = 'cloudpickle' )) as p :
0 commit comments