github.com/apache/beam/sdks/v2@v2.48.2/python/apache_beam/transforms/sideinputs_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 """Unit tests for side inputs.""" 19 20 # pytype: skip-file 21 22 import itertools 23 import logging 24 import unittest 25 26 import pytest 27 28 import apache_beam as beam 29 from apache_beam.testing.test_pipeline import TestPipeline 30 from apache_beam.testing.test_stream import TestStream 31 from apache_beam.testing.util import assert_that 32 from apache_beam.testing.util import equal_to 33 from apache_beam.testing.util import equal_to_per_window 34 from apache_beam.transforms import Map 35 from apache_beam.transforms import trigger 36 from apache_beam.transforms import window 37 from apache_beam.utils.timestamp import Timestamp 38 39 40 class SideInputsTest(unittest.TestCase): 41 def create_pipeline(self): 42 return TestPipeline() 43 44 def run_windowed_side_inputs( 45 self, 46 elements, 47 main_window_fn, 48 side_window_fn=None, 49 side_input_type=beam.pvalue.AsList, 50 combine_fn=None, 51 expected=None): 52 with self.create_pipeline() as p: 53 pcoll = p | beam.Create(elements) | beam.Map( 54 lambda t: window.TimestampedValue(t, t)) 55 main = pcoll | 'WindowMain' >> beam.WindowInto(main_window_fn) 56 side = pcoll | 'WindowSide' >> beam.WindowInto( 57 side_window_fn or main_window_fn) 58 kw = {} 59 if combine_fn is not None: 60 side |= beam.CombineGlobally(combine_fn).without_defaults() 61 kw['default_value'] = 0 62 elif side_input_type == beam.pvalue.AsDict: 63 side |= beam.Map(lambda x: ('k%s' % x, 'v%s' % x)) 64 res = main | beam.Map(lambda x, s: (x, s), side_input_type(side, **kw)) 65 if side_input_type in (beam.pvalue.AsIter, beam.pvalue.AsList): 66 res |= beam.Map(lambda x_s: (x_s[0], sorted(x_s[1]))) 67 assert_that(res, equal_to(expected)) 68 69 def test_global_global_windows(self): 70 self.run_windowed_side_inputs([1, 2, 3], 71 window.GlobalWindows(), 72 expected=[(1, [1, 2, 3]), (2, [1, 2, 3]), 73 (3, [1, 2, 3])]) 74 75 def test_same_fixed_windows(self): 76 self.run_windowed_side_inputs([1, 2, 11], 77 window.FixedWindows(10), 78 expected=[(1, [1, 2]), (2, [1, 2]), 79 (11, [11])]) 80 81 def test_different_fixed_windows(self): 82 self.run_windowed_side_inputs([1, 2, 11, 21, 31], 83 window.FixedWindows(10), 84 window.FixedWindows(20), 85 expected=[(1, [1, 2, 11]), (2, [1, 2, 11]), 86 (11, [1, 2, 11]), (21, [21, 31]), 87 (31, [21, 31])]) 88 89 def test_fixed_global_window(self): 90 self.run_windowed_side_inputs([1, 2, 11], 91 window.FixedWindows(10), 92 window.GlobalWindows(), 93 expected=[(1, [1, 2, 11]), (2, [1, 2, 11]), 94 (11, [1, 2, 11])]) 95 96 def test_sliding_windows(self): 97 self.run_windowed_side_inputs( 98 [1, 2, 4], 99 window.SlidingWindows(size=6, period=2), 100 window.SlidingWindows(size=6, period=2), 101 expected=[ 102 # Element 1 falls in three windows 103 (1, [1]), # [-4, 2) 104 (1, [1, 2]), # [-2, 4) 105 (1, [1, 2, 4]), # [0, 6) 106 # as does 2, 107 (2, [1, 2]), # [-2, 4) 108 (2, [1, 2, 4]), # [0, 6) 109 (2, [2, 4]), # [2, 8) 110 # and 4. 111 (4, [1, 2, 4]), # [0, 6) 112 (4, [2, 4]), # [2, 8) 113 (4, [4]), # [4, 10) 114 ]) 115 116 def test_windowed_iter(self): 117 self.run_windowed_side_inputs([1, 2, 11], 118 window.FixedWindows(10), 119 side_input_type=beam.pvalue.AsIter, 120 expected=[(1, [1, 2]), (2, [1, 2]), 121 (11, [11])]) 122 123 def test_windowed_singleton(self): 124 self.run_windowed_side_inputs([1, 2, 11], 125 window.FixedWindows(10), 126 side_input_type=beam.pvalue.AsSingleton, 127 combine_fn=sum, 128 expected=[(1, 3), (2, 3), (11, 11)]) 129 130 def test_windowed_dict(self): 131 self.run_windowed_side_inputs([1, 2, 11], 132 window.FixedWindows(10), 133 side_input_type=beam.pvalue.AsDict, 134 expected=[ 135 (1, { 136 'k1': 'v1', 'k2': 'v2' 137 }), 138 (2, { 139 'k1': 'v1', 'k2': 'v2' 140 }), 141 (11, { 142 'k11': 'v11' 143 }), 144 ]) 145 146 @pytest.mark.it_validatesrunner 147 def test_empty_singleton_side_input(self): 148 pipeline = self.create_pipeline() 149 pcol = pipeline | 'start' >> beam.Create([1, 2]) 150 side = pipeline | 'side' >> beam.Create([]) # Empty side input. 151 152 def my_fn(k, s): 153 # TODO(robertwb): Should this be an error as in Java? 154 v = ('empty' if isinstance(s, beam.pvalue.EmptySideInput) else 'full') 155 return [(k, v)] 156 157 result = pcol | 'compute' >> beam.FlatMap( 158 my_fn, beam.pvalue.AsSingleton(side)) 159 assert_that(result, equal_to([(1, 'empty'), (2, 'empty')])) 160 pipeline.run() 161 162 # TODO(https://github.com/apache/beam/issues/19012): Disable this test in 163 # streaming temporarily. Remove sickbay-streaming tag after it's fixed. 164 @pytest.mark.no_sickbay_streaming 165 @pytest.mark.it_validatesrunner 166 def test_multi_valued_singleton_side_input(self): 167 pipeline = self.create_pipeline() 168 pcol = pipeline | 'start' >> beam.Create([1, 2]) 169 side = pipeline | 'side' >> beam.Create([3, 4]) # 2 values in side input. 170 pcol | 'compute' >> beam.FlatMap( # pylint: disable=expression-not-assigned 171 lambda x, s: [x * s], beam.pvalue.AsSingleton(side)) 172 with self.assertRaises(Exception): 173 pipeline.run() 174 175 @pytest.mark.it_validatesrunner 176 def test_default_value_singleton_side_input(self): 177 pipeline = self.create_pipeline() 178 pcol = pipeline | 'start' >> beam.Create([1, 2]) 179 side = pipeline | 'side' >> beam.Create([]) # 0 values in side input. 180 result = pcol | beam.FlatMap( 181 lambda x, s: [x * s], beam.pvalue.AsSingleton(side, 10)) 182 assert_that(result, equal_to([10, 20])) 183 pipeline.run() 184 185 @pytest.mark.it_validatesrunner 186 def test_iterable_side_input(self): 187 pipeline = self.create_pipeline() 188 pcol = pipeline | 'start' >> beam.Create([1, 2]) 189 side = pipeline | 'side' >> beam.Create([3, 4]) # 2 values in side input. 190 result = pcol | 'compute' >> beam.FlatMap( 191 lambda x, s: [x * y for y in s], beam.pvalue.AsIter(side)) 192 assert_that(result, equal_to([3, 4, 6, 8])) 193 pipeline.run() 194 195 @pytest.mark.it_validatesrunner 196 def test_reiterable_side_input(self): 197 expected_side = frozenset(range(100)) 198 199 def check_reiteration(main, side): 200 assert expected_side == set(side), side 201 # Iterate a second time. 202 assert expected_side == set(side), side 203 # Iterate over two copies of the input at the same time. 204 both = zip(side, side) 205 first, second = zip(*both) 206 assert expected_side == set(first), first 207 assert expected_side == set(second), second 208 # This will iterate over two copies of the side input, but offset. 209 offset = [None] * (len(expected_side) // 2) 210 both = zip(itertools.chain(side, offset), itertools.chain(offset, side)) 211 first, second = zip(*both) 212 expected_and_none = frozenset.union(expected_side, [None]) 213 assert expected_and_none == set(first), first 214 assert expected_and_none == set(second), second 215 216 pipeline = self.create_pipeline() 217 pcol = pipeline | 'start' >> beam.Create(['A', 'B']) 218 side = pipeline | 'side' >> beam.Create(expected_side) 219 _ = pcol | 'check' >> beam.Map(check_reiteration, beam.pvalue.AsIter(side)) 220 pipeline.run() 221 222 @pytest.mark.it_validatesrunner 223 def test_as_list_and_as_dict_side_inputs(self): 224 a_list = [5, 1, 3, 2, 9] 225 some_pairs = [('crouton', 17), ('supreme', None)] 226 pipeline = self.create_pipeline() 227 main_input = pipeline | 'main input' >> beam.Create([1]) 228 side_list = pipeline | 'side list' >> beam.Create(a_list) 229 side_pairs = pipeline | 'side pairs' >> beam.Create(some_pairs) 230 results = main_input | 'concatenate' >> beam.Map( 231 lambda x, 232 the_list, 233 the_dict: [x, the_list, the_dict], 234 beam.pvalue.AsList(side_list), 235 beam.pvalue.AsDict(side_pairs)) 236 237 def matcher(expected_elem, expected_list, expected_pairs): 238 def match(actual): 239 [[actual_elem, actual_list, actual_dict]] = actual 240 equal_to([expected_elem])([actual_elem]) 241 equal_to(expected_list)(actual_list) 242 equal_to(expected_pairs)(actual_dict.items()) 243 244 return match 245 246 assert_that(results, matcher(1, a_list, some_pairs)) 247 pipeline.run() 248 249 @pytest.mark.it_validatesrunner 250 def test_as_singleton_without_unique_labels(self): 251 # This should succeed as calling beam.pvalue.AsSingleton on the same 252 # PCollection twice with the same defaults will return the same 253 # view. 254 a_list = [2] 255 pipeline = self.create_pipeline() 256 main_input = pipeline | 'main input' >> beam.Create([1]) 257 side_list = pipeline | 'side list' >> beam.Create(a_list) 258 results = main_input | beam.Map( 259 lambda x, 260 s1, 261 s2: [x, s1, s2], 262 beam.pvalue.AsSingleton(side_list), 263 beam.pvalue.AsSingleton(side_list)) 264 265 def matcher(expected_elem, expected_singleton): 266 def match(actual): 267 [[actual_elem, actual_singleton1, actual_singleton2]] = actual 268 equal_to([expected_elem])([actual_elem]) 269 equal_to([expected_singleton])([actual_singleton1]) 270 equal_to([expected_singleton])([actual_singleton2]) 271 272 return match 273 274 assert_that(results, matcher(1, 2)) 275 pipeline.run() 276 277 @pytest.mark.it_validatesrunner 278 def test_as_singleton_with_different_defaults(self): 279 a_list = [] 280 pipeline = self.create_pipeline() 281 main_input = pipeline | 'main input' >> beam.Create([1]) 282 side_list = pipeline | 'side list' >> beam.Create(a_list) 283 results = main_input | beam.Map( 284 lambda x, 285 s1, 286 s2: [x, s1, s2], 287 beam.pvalue.AsSingleton(side_list, default_value=2), 288 beam.pvalue.AsSingleton(side_list, default_value=3)) 289 290 def matcher(expected_elem, expected_singleton1, expected_singleton2): 291 def match(actual): 292 [[actual_elem, actual_singleton1, actual_singleton2]] = actual 293 equal_to([expected_elem])([actual_elem]) 294 equal_to([expected_singleton1])([actual_singleton1]) 295 equal_to([expected_singleton2])([actual_singleton2]) 296 297 return match 298 299 assert_that(results, matcher(1, 2, 3)) 300 pipeline.run() 301 302 @pytest.mark.it_validatesrunner 303 def test_as_list_twice(self): 304 # This should succeed as calling beam.pvalue.AsList on the same 305 # PCollection twice will return the same view. 306 a_list = [1, 2, 3] 307 pipeline = self.create_pipeline() 308 main_input = pipeline | 'main input' >> beam.Create([1]) 309 side_list = pipeline | 'side list' >> beam.Create(a_list) 310 results = main_input | beam.Map( 311 lambda x, 312 ls1, 313 ls2: [x, ls1, ls2], 314 beam.pvalue.AsList(side_list), 315 beam.pvalue.AsList(side_list)) 316 317 def matcher(expected_elem, expected_list): 318 def match(actual): 319 [[actual_elem, actual_list1, actual_list2]] = actual 320 equal_to([expected_elem])([actual_elem]) 321 equal_to(expected_list)(actual_list1) 322 equal_to(expected_list)(actual_list2) 323 324 return match 325 326 assert_that(results, matcher(1, [1, 2, 3])) 327 pipeline.run() 328 329 @pytest.mark.it_validatesrunner 330 def test_as_dict_twice(self): 331 some_kvs = [('a', 1), ('b', 2)] 332 pipeline = self.create_pipeline() 333 main_input = pipeline | 'main input' >> beam.Create([1]) 334 side_kvs = pipeline | 'side kvs' >> beam.Create(some_kvs) 335 results = main_input | beam.Map( 336 lambda x, 337 dct1, 338 dct2: [x, dct1, dct2], 339 beam.pvalue.AsDict(side_kvs), 340 beam.pvalue.AsDict(side_kvs)) 341 342 def matcher(expected_elem, expected_kvs): 343 def match(actual): 344 [[actual_elem, actual_dict1, actual_dict2]] = actual 345 equal_to([expected_elem])([actual_elem]) 346 equal_to(expected_kvs)(actual_dict1.items()) 347 equal_to(expected_kvs)(actual_dict2.items()) 348 349 return match 350 351 assert_that(results, matcher(1, some_kvs)) 352 pipeline.run() 353 354 @pytest.mark.it_validatesrunner 355 def test_flattened_side_input(self): 356 pipeline = self.create_pipeline() 357 main_input = pipeline | 'main input' >> beam.Create([None]) 358 side_input = ( 359 pipeline | 'side1' >> beam.Create(['a']), 360 pipeline | 'side2' >> beam.Create(['b'])) | beam.Flatten() 361 results = main_input | beam.FlatMap( 362 lambda _, ab: ab, beam.pvalue.AsList(side_input)) 363 364 assert_that(results, equal_to(['a', 'b'])) 365 pipeline.run() 366 367 # TODO(BEAM-9499): Disable this test in streaming temporarily. 368 @pytest.mark.no_sickbay_batch 369 @pytest.mark.no_sickbay_streaming 370 @pytest.mark.it_validatesrunner 371 def test_multi_triggered_gbk_side_input(self): 372 """Test a GBK sideinput, with multiple triggering.""" 373 # TODO(https://github.com/apache/beam/issues/20065): Remove use of this 374 # experiment. This flag is only necessary when using the multi-output 375 # TestStream b/c it relies on using the PCollection output tags as the 376 # PCollection output ids. 377 with TestPipeline() as p: 378 379 test_stream = ( 380 p 381 | 'Mixed TestStream' >> TestStream().advance_watermark_to( 382 3, 383 tag='main').add_elements(['a1'], tag='main').advance_watermark_to( 384 8, tag='main').add_elements(['a2'], tag='main').add_elements( 385 [window.TimestampedValue(('k', 100), 2)], tag='side'). 386 add_elements([window.TimestampedValue( 387 ('k', 400), 7)], tag='side').advance_watermark_to_infinity( 388 tag='main').advance_watermark_to_infinity(tag='side')) 389 390 main_data = ( 391 test_stream['main'] 392 | 'Main windowInto' >> beam.WindowInto( 393 window.FixedWindows(5), 394 accumulation_mode=trigger.AccumulationMode.DISCARDING)) 395 396 side_data = ( 397 test_stream['side'] 398 | 'Side windowInto' >> beam.WindowInto( 399 window.FixedWindows(5), 400 trigger=trigger.AfterWatermark(early=trigger.AfterCount(1)), 401 accumulation_mode=trigger.AccumulationMode.DISCARDING) 402 | beam.CombinePerKey(sum) 403 | 'Values' >> Map(lambda k_vs: k_vs[1])) 404 405 class RecordFn(beam.DoFn): 406 def process( 407 self, 408 elm=beam.DoFn.ElementParam, 409 ts=beam.DoFn.TimestampParam, 410 side=beam.DoFn.SideInputParam): 411 yield (elm, ts, side) 412 413 records = ( 414 main_data 415 | beam.ParDo(RecordFn(), beam.pvalue.AsList(side_data))) 416 417 expected_window_to_elements = { 418 window.IntervalWindow(0, 5): [ 419 ('a1', Timestamp(3), [100, 0]), 420 ], 421 window.IntervalWindow(5, 10): [('a2', Timestamp(8), [400, 0])], 422 } 423 424 assert_that( 425 records, 426 equal_to_per_window(expected_window_to_elements), 427 use_global_window=False, 428 label='assert per window') 429 430 431 if __name__ == '__main__': 432 logging.getLogger().setLevel(logging.DEBUG) 433 unittest.main()