github.com/apache/beam/sdks/v2@v2.48.2/python/apache_beam/yaml/yaml_transform_test.py (about) 1 # 2 # Licensed to the Apache Software Foundation (ASF) under one or more 3 # contributor license agreements. See the NOTICE file distributed with 4 # this work for additional information regarding copyright ownership. 5 # The ASF licenses this file to You under the Apache License, Version 2.0 6 # (the "License"); you may not use this file except in compliance with 7 # the License. You may obtain a copy of the License at 8 # 9 # http://www.apache.org/licenses/LICENSE-2.0 10 # 11 # Unless required by applicable law or agreed to in writing, software 12 # distributed under the License is distributed on an "AS IS" BASIS, 13 # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 # See the License for the specific language governing permissions and 15 # limitations under the License. 16 # 17 18 import glob 19 import logging 20 import os 21 import tempfile 22 import unittest 23 24 import apache_beam as beam 25 from apache_beam.testing.util import assert_that 26 from apache_beam.testing.util import equal_to 27 from apache_beam.yaml.yaml_transform import YamlTransform 28 29 30 class YamlTransformTest(unittest.TestCase): 31 def test_composite(self): 32 with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions( 33 pickle_library='cloudpickle')) as p: 34 elements = p | beam.Create([1, 2, 3]) 35 # TODO(robertwb): Consider making the input implicit (and below). 36 result = elements | YamlTransform( 37 ''' 38 type: composite 39 input: 40 elements: input 41 transforms: 42 - type: PyMap 43 name: Square 44 input: elements 45 fn: "lambda x: x * x" 46 - type: PyMap 47 name: Cube 48 input: elements 49 fn: "lambda x: x * x * x" 50 - type: Flatten 51 input: [Square, Cube] 52 output: 53 Flatten 54 ''') 55 assert_that(result, equal_to([1, 4, 9, 1, 8, 27])) 56 57 def test_chain_with_input(self): 58 with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions( 59 pickle_library='cloudpickle')) as p: 60 elements = p | beam.Create(range(10)) 61 result = elements | YamlTransform( 62 ''' 63 type: chain 64 input: 65 elements: input 66 transforms: 67 - type: PyMap 68 fn: "lambda x: x * x + x" 69 - type: PyMap 70 fn: "lambda x: x + 41" 71 ''') 72 assert_that(result, equal_to([41, 43, 47, 53, 61, 71, 83, 97, 113, 131])) 73 74 def test_chain_with_source_sink(self): 75 with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions( 76 pickle_library='cloudpickle')) as p: 77 result = p | YamlTransform( 78 ''' 79 type: chain 80 source: 81 type: Create 82 elements: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9] 83 transforms: 84 - type: PyMap 85 fn: "lambda x: x * x + x" 86 sink: 87 type: PyMap 88 fn: "lambda x: x + 41" 89 ''') 90 assert_that(result, equal_to([41, 43, 47, 53, 61, 71, 83, 97, 113, 131])) 91 92 def test_chain_with_root(self): 93 with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions( 94 pickle_library='cloudpickle')) as p: 95 result = p | YamlTransform( 96 ''' 97 type: chain 98 transforms: 99 - type: Create 100 elements: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9] 101 - type: PyMap 102 fn: "lambda x: x * x + x" 103 - type: PyMap 104 fn: "lambda x: x + 41" 105 ''') 106 assert_that(result, equal_to([41, 43, 47, 53, 61, 71, 83, 97, 113, 131])) 107 108 def test_implicit_flatten(self): 109 with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions( 110 pickle_library='cloudpickle')) as p: 111 result = p | YamlTransform( 112 ''' 113 type: composite 114 transforms: 115 - type: Create 116 name: CreateSmall 117 elements: [1, 2, 3] 118 - type: Create 119 name: CreateBig 120 elements: [100, 200] 121 - type: PyMap 122 input: [CreateBig, CreateSmall] 123 fn: "lambda x: x * x" 124 output: PyMap 125 ''') 126 assert_that(result, equal_to([1, 4, 9, 10000, 40000])) 127 128 def test_csv_to_json(self): 129 try: 130 import pandas as pd 131 except ImportError: 132 raise unittest.SkipTest('Pandas not available.') 133 134 with tempfile.TemporaryDirectory() as tmpdir: 135 data = pd.DataFrame([ 136 { 137 'label': '11a', 'rank': 0 138 }, 139 { 140 'label': '37a', 'rank': 1 141 }, 142 { 143 'label': '389a', 'rank': 2 144 }, 145 ]) 146 input = os.path.join(tmpdir, 'input.csv') 147 output = os.path.join(tmpdir, 'output.json') 148 data.to_csv(input, index=False) 149 150 with beam.Pipeline() as p: 151 result = p | YamlTransform( 152 ''' 153 type: chain 154 transforms: 155 - type: ReadFromCsv 156 path: %s 157 - type: WriteToJson 158 path: %s 159 num_shards: 1 160 ''' % (repr(input), repr(output))) 161 162 output_shard = list(glob.glob(output + "*"))[0] 163 result = pd.read_json( 164 output_shard, orient='records', 165 lines=True).sort_values('rank').reindex() 166 pd.testing.assert_frame_equal(data, result) 167 168 169 class CreateTimestamped(beam.PTransform): 170 def __init__(self, elements): 171 self._elements = elements 172 173 def expand(self, p): 174 return ( 175 p 176 | beam.Create(self._elements) 177 | beam.Map(lambda x: beam.transforms.window.TimestampedValue(x, x))) 178 179 180 class SumGlobally(beam.PTransform): 181 def expand(self, pcoll): 182 return pcoll | beam.CombineGlobally(sum).without_defaults() 183 184 185 TEST_PROVIDERS = { 186 'CreateTimestamped': CreateTimestamped, 'SumGlobally': SumGlobally 187 } 188 189 190 class YamlWindowingTest(unittest.TestCase): 191 def test_explicit_window_into(self): 192 with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions( 193 pickle_library='cloudpickle')) as p: 194 result = p | YamlTransform( 195 ''' 196 type: chain 197 transforms: 198 - type: CreateTimestamped 199 elements: [0, 1, 2, 3, 4, 5] 200 - type: WindowInto 201 windowing: 202 type: fixed 203 size: 4 204 - type: SumGlobally 205 ''', 206 providers=TEST_PROVIDERS) 207 assert_that(result, equal_to([6, 9])) 208 209 def test_windowing_on_input(self): 210 with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions( 211 pickle_library='cloudpickle')) as p: 212 result = p | YamlTransform( 213 ''' 214 type: chain 215 transforms: 216 - type: CreateTimestamped 217 elements: [0, 1, 2, 3, 4, 5] 218 - type: SumGlobally 219 windowing: 220 type: fixed 221 size: 4 222 ''', 223 providers=TEST_PROVIDERS) 224 assert_that(result, equal_to([6, 9])) 225 226 def test_windowing_multiple_inputs(self): 227 with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions( 228 pickle_library='cloudpickle')) as p: 229 result = p | YamlTransform( 230 ''' 231 type: composite 232 transforms: 233 - type: CreateTimestamped 234 name: Create1 235 elements: [0, 2, 4] 236 - type: CreateTimestamped 237 name: Create2 238 elements: [1, 3, 5] 239 - type: SumGlobally 240 input: [Create1, Create2] 241 windowing: 242 type: fixed 243 size: 4 244 output: SumGlobally 245 ''', 246 providers=TEST_PROVIDERS) 247 assert_that(result, equal_to([6, 9])) 248 249 def test_windowing_on_output(self): 250 with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions( 251 pickle_library='cloudpickle')) as p: 252 result = p | YamlTransform( 253 ''' 254 type: chain 255 transforms: 256 - type: CreateTimestamped 257 elements: [0, 1, 2, 3, 4, 5] 258 windowing: 259 type: fixed 260 size: 4 261 - type: SumGlobally 262 ''', 263 providers=TEST_PROVIDERS) 264 assert_that(result, equal_to([6, 9])) 265 266 def test_windowing_on_outer(self): 267 with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions( 268 pickle_library='cloudpickle')) as p: 269 result = p | YamlTransform( 270 ''' 271 type: chain 272 transforms: 273 - type: CreateTimestamped 274 elements: [0, 1, 2, 3, 4, 5] 275 - type: SumGlobally 276 windowing: 277 type: fixed 278 size: 4 279 ''', 280 providers=TEST_PROVIDERS) 281 assert_that(result, equal_to([6, 9])) 282 283 284 if __name__ == '__main__': 285 logging.getLogger().setLevel(logging.INFO) 286 unittest.main()