github.com/apache/beam/sdks/v2@v2.48.2/python/apache_beam/runners/common_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 # pytype: skip-file 19 20 import unittest 21 22 import hamcrest as hc 23 24 import apache_beam as beam 25 from apache_beam.io.restriction_trackers import OffsetRange 26 from apache_beam.io.restriction_trackers import OffsetRestrictionTracker 27 from apache_beam.io.watermark_estimators import ManualWatermarkEstimator 28 from apache_beam.options.pipeline_options import PipelineOptions 29 from apache_beam.runners.common import DoFnSignature 30 from apache_beam.runners.common import PerWindowInvoker 31 from apache_beam.runners.sdf_utils import SplitResultPrimary 32 from apache_beam.runners.sdf_utils import SplitResultResidual 33 from apache_beam.testing.test_pipeline import TestPipeline 34 from apache_beam.testing.test_stream import TestStream 35 from apache_beam.transforms import trigger 36 from apache_beam.transforms import window 37 from apache_beam.transforms.core import DoFn 38 from apache_beam.transforms.core import RestrictionProvider 39 from apache_beam.transforms.window import IntervalWindow 40 from apache_beam.utils.timestamp import Timestamp 41 from apache_beam.utils.windowed_value import WindowedValue 42 43 44 class DoFnSignatureTest(unittest.TestCase): 45 def test_dofn_validate_process_error(self): 46 class MyDoFn(DoFn): 47 def process(self, element, w1=DoFn.WindowParam, w2=DoFn.WindowParam): 48 pass 49 50 with self.assertRaises(ValueError): 51 DoFnSignature(MyDoFn()) 52 53 def test_dofn_get_defaults(self): 54 class MyDoFn(DoFn): 55 def process(self, element, w=DoFn.WindowParam): 56 pass 57 58 signature = DoFnSignature(MyDoFn()) 59 60 self.assertEqual(signature.process_method.defaults, [DoFn.WindowParam]) 61 62 @unittest.skip('BEAM-5878') 63 def test_dofn_get_defaults_kwonly(self): 64 class MyDoFn(DoFn): 65 def process(self, element, *, w=DoFn.WindowParam): 66 pass 67 68 signature = DoFnSignature(MyDoFn()) 69 70 self.assertEqual(signature.process_method.defaults, [DoFn.WindowParam]) 71 72 def test_dofn_validate_start_bundle_error(self): 73 class MyDoFn(DoFn): 74 def process(self, element): 75 pass 76 77 def start_bundle(self, w1=DoFn.WindowParam): 78 pass 79 80 with self.assertRaises(ValueError): 81 DoFnSignature(MyDoFn()) 82 83 def test_dofn_validate_finish_bundle_error(self): 84 class MyDoFn(DoFn): 85 def process(self, element): 86 pass 87 88 def finish_bundle(self, w1=DoFn.WindowParam): 89 pass 90 91 with self.assertRaises(ValueError): 92 DoFnSignature(MyDoFn()) 93 94 def test_unbounded_element_process_fn(self): 95 class UnboundedDoFn(DoFn): 96 @DoFn.unbounded_per_element() 97 def process(self, element): 98 pass 99 100 class BoundedDoFn(DoFn): 101 def process(self, element): 102 pass 103 104 signature = DoFnSignature(UnboundedDoFn()) 105 self.assertTrue(signature.is_unbounded_per_element()) 106 signature = DoFnSignature(BoundedDoFn()) 107 self.assertFalse(signature.is_unbounded_per_element()) 108 109 110 class DoFnProcessTest(unittest.TestCase): 111 # pylint: disable=expression-not-assigned 112 all_records = None 113 114 def setUp(self): 115 DoFnProcessTest.all_records = [] 116 117 def record_dofn(self): 118 class RecordDoFn(DoFn): 119 def process(self, element): 120 DoFnProcessTest.all_records.append(element) 121 122 return RecordDoFn() 123 124 def test_dofn_process_keyparam(self): 125 class DoFnProcessWithKeyparam(DoFn): 126 def process(self, element, mykey=DoFn.KeyParam): 127 yield "{key}-verify".format(key=mykey) 128 129 pipeline_options = PipelineOptions() 130 131 with TestPipeline(options=pipeline_options) as p: 132 test_stream = (TestStream().advance_watermark_to(10).add_elements([1, 2])) 133 ( 134 p 135 | test_stream 136 | beam.Map(lambda x: (x, "some-value")) 137 | "window_into" >> beam.WindowInto( 138 window.FixedWindows(5), 139 accumulation_mode=trigger.AccumulationMode.DISCARDING) 140 | beam.ParDo(DoFnProcessWithKeyparam()) 141 | beam.ParDo(self.record_dofn())) 142 143 self.assertEqual(['1-verify', '2-verify'], 144 sorted(DoFnProcessTest.all_records)) 145 146 def test_dofn_process_keyparam_error_no_key(self): 147 class DoFnProcessWithKeyparam(DoFn): 148 def process(self, element, mykey=DoFn.KeyParam): 149 yield "{key}-verify".format(key=mykey) 150 151 pipeline_options = PipelineOptions() 152 with self.assertRaises(ValueError),\ 153 TestPipeline(options=pipeline_options) as p: 154 test_stream = (TestStream().advance_watermark_to(10).add_elements([1, 2])) 155 (p | test_stream | beam.ParDo(DoFnProcessWithKeyparam())) 156 157 def test_pardo_with_unbounded_per_element_dofn(self): 158 class UnboundedDoFn(beam.DoFn): 159 @beam.DoFn.unbounded_per_element() 160 def process(self, element): 161 pass 162 163 class BoundedDoFn(beam.DoFn): 164 def process(self, element): 165 pass 166 167 with TestPipeline() as p: 168 source = p | beam.Impulse() 169 unbounded_pcoll = source | beam.ParDo(UnboundedDoFn()) 170 bounded_pcoll = source | beam.ParDo(BoundedDoFn()) 171 172 self.assertEqual(unbounded_pcoll.is_bounded, False) 173 self.assertEqual(bounded_pcoll.is_bounded, True) 174 175 176 class TestOffsetRestrictionProvider(RestrictionProvider): 177 def restriction_size(self, element, restriction): 178 return restriction.size() 179 180 181 class PerWindowInvokerSplitTest(unittest.TestCase): 182 def setUp(self): 183 self.window1 = IntervalWindow(0, 10) 184 self.window2 = IntervalWindow(10, 20) 185 self.window3 = IntervalWindow(20, 30) 186 self.windowed_value = WindowedValue( 187 'a', 57, (self.window1, self.window2, self.window3)) 188 self.restriction = OffsetRange(0, 100) 189 self.watermark_estimator_state = Timestamp(21) 190 self.restriction_provider = TestOffsetRestrictionProvider() 191 self.watermark_estimator = ManualWatermarkEstimator(Timestamp(42)) 192 self.maxDiff = None 193 194 def create_split_in_window(self, offset_index, windows): 195 return ( 196 SplitResultPrimary( 197 primary_value=WindowedValue((( 198 'a', 199 (OffsetRange(0, offset_index), self.watermark_estimator_state)), 200 offset_index), 201 57, 202 windows)), 203 SplitResultResidual( 204 residual_value=WindowedValue((( 205 'a', 206 ( 207 OffsetRange(offset_index, 100), 208 self.watermark_estimator.get_estimator_state())), 209 100 - offset_index), 210 57, 211 windows), 212 current_watermark=self.watermark_estimator.current_watermark(), 213 deferred_timestamp=None)) 214 215 def create_split_across_windows(self, primary_windows, residual_windows): 216 primary = SplitResultPrimary( 217 primary_value=WindowedValue( 218 (('a', (OffsetRange(0, 100), self.watermark_estimator_state)), 100), 219 57, 220 primary_windows)) if primary_windows else None 221 residual = SplitResultResidual( 222 residual_value=WindowedValue( 223 (('a', (OffsetRange(0, 100), self.watermark_estimator_state)), 100), 224 57, 225 residual_windows), 226 current_watermark=None, 227 deferred_timestamp=None) if residual_windows else None 228 return primary, residual 229 230 def test_non_window_observing_checkpoint(self): 231 # test checkpoint 232 restriction_tracker = OffsetRestrictionTracker(self.restriction) 233 restriction_tracker.try_claim(30) 234 (primaries, residuals, stop_index) = PerWindowInvoker._try_split( 235 0.0, 236 None, 237 None, 238 self.windowed_value, 239 self.restriction, 240 self.watermark_estimator_state, 241 self.restriction_provider, 242 restriction_tracker, 243 self.watermark_estimator) 244 expected_primary_split, expected_residual_split = ( 245 self.create_split_in_window(31, self.windowed_value.windows)) 246 self.assertEqual([expected_primary_split], primaries) 247 self.assertEqual([expected_residual_split], residuals) 248 # We don't expect the stop index to be set for non window observing splits 249 self.assertIsNone(stop_index) 250 251 def test_non_window_observing_split(self): 252 restriction_tracker = OffsetRestrictionTracker(self.restriction) 253 restriction_tracker.try_claim(30) 254 (primaries, residuals, stop_index) = PerWindowInvoker._try_split( 255 0.1, 256 None, 257 None, 258 self.windowed_value, 259 self.restriction, 260 self.watermark_estimator_state, 261 self.restriction_provider, 262 restriction_tracker, 263 self.watermark_estimator) 264 expected_primary_split, expected_residual_split = ( 265 self.create_split_in_window(37, self.windowed_value.windows)) 266 self.assertEqual([expected_primary_split], primaries) 267 self.assertEqual([expected_residual_split], residuals) 268 # We don't expect the stop index to be set for non window observing splits 269 self.assertIsNone(stop_index) 270 271 def test_non_window_observing_split_when_restriction_is_done(self): 272 restriction_tracker = OffsetRestrictionTracker(self.restriction) 273 restriction_tracker.try_claim(100) 274 self.assertIsNone( 275 PerWindowInvoker._try_split( 276 0.1, 277 None, 278 None, 279 self.windowed_value, 280 self.restriction, 281 self.watermark_estimator_state, 282 self.restriction_provider, 283 restriction_tracker, 284 self.watermark_estimator)) 285 286 def test_window_observing_checkpoint_on_first_window(self): 287 restriction_tracker = OffsetRestrictionTracker(self.restriction) 288 restriction_tracker.try_claim(30) 289 (primaries, residuals, stop_index) = PerWindowInvoker._try_split( 290 0.0, 291 0, 292 3, 293 self.windowed_value, 294 self.restriction, 295 self.watermark_estimator_state, 296 self.restriction_provider, 297 restriction_tracker, 298 self.watermark_estimator) 299 expected_primary_split, expected_residual_split = ( 300 self.create_split_in_window(31, (self.window1, ))) 301 _, expected_residual_windows = ( 302 self.create_split_across_windows(None, (self.window2, self.window3,))) 303 hc.assert_that(primaries, hc.contains_inanyorder(expected_primary_split)) 304 hc.assert_that( 305 residuals, 306 hc.contains_inanyorder( 307 expected_residual_split, 308 expected_residual_windows, 309 )) 310 self.assertEqual(stop_index, 1) 311 312 def test_window_observing_checkpoint_on_first_window_after_prior_split(self): 313 restriction_tracker = OffsetRestrictionTracker(self.restriction) 314 restriction_tracker.try_claim(30) 315 (primaries, residuals, stop_index) = PerWindowInvoker._try_split( 316 0.0, 317 0, 318 2, # stop index < len(windows) representing a prior split had occurred 319 self.windowed_value, 320 self.restriction, 321 self.watermark_estimator_state, 322 self.restriction_provider, 323 restriction_tracker, 324 self.watermark_estimator) 325 expected_primary_split, expected_residual_split = ( 326 self.create_split_in_window(31, (self.window1, ))) 327 _, expected_residual_windows = ( 328 self.create_split_across_windows(None, (self.window2, ))) 329 hc.assert_that(primaries, hc.contains_inanyorder(expected_primary_split)) 330 hc.assert_that( 331 residuals, 332 hc.contains_inanyorder( 333 expected_residual_split, 334 expected_residual_windows, 335 )) 336 self.assertEqual(stop_index, 1) 337 338 def test_window_observing_split_on_first_window(self): 339 restriction_tracker = OffsetRestrictionTracker(self.restriction) 340 restriction_tracker.try_claim(30) 341 (primaries, residuals, stop_index) = PerWindowInvoker._try_split( 342 0.2, 343 0, 344 3, 345 self.windowed_value, 346 self.restriction, 347 self.watermark_estimator_state, 348 self.restriction_provider, 349 restriction_tracker, 350 self.watermark_estimator) 351 # 20% of 2.7 windows = 20% of 270 offset left = 54 offset 352 # 30 + 54 = 84 split offset 353 expected_primary_split, expected_residual_split = ( 354 self.create_split_in_window(84, (self.window1, ))) 355 _, expected_residual_windows = ( 356 self.create_split_across_windows(None, (self.window2, self.window3, ))) 357 hc.assert_that(primaries, hc.contains_inanyorder(expected_primary_split)) 358 hc.assert_that( 359 residuals, 360 hc.contains_inanyorder( 361 expected_residual_split, 362 expected_residual_windows, 363 )) 364 self.assertEqual(stop_index, 1) 365 366 def test_window_observing_split_on_middle_window(self): 367 restriction_tracker = OffsetRestrictionTracker(self.restriction) 368 restriction_tracker.try_claim(30) 369 (primaries, residuals, stop_index) = PerWindowInvoker._try_split( 370 0.2, 371 1, 372 3, 373 self.windowed_value, 374 self.restriction, 375 self.watermark_estimator_state, 376 self.restriction_provider, 377 restriction_tracker, 378 self.watermark_estimator) 379 # 20% of 1.7 windows = 20% of 170 offset left = 34 offset 380 # 30 + 34 = 64 split offset 381 expected_primary_split, expected_residual_split = ( 382 self.create_split_in_window(64, (self.window2, ))) 383 expected_primary_windows, expected_residual_windows = ( 384 self.create_split_across_windows((self.window1, ), (self.window3, ))) 385 hc.assert_that( 386 primaries, 387 hc.contains_inanyorder( 388 expected_primary_split, 389 expected_primary_windows, 390 )) 391 hc.assert_that( 392 residuals, 393 hc.contains_inanyorder( 394 expected_residual_split, 395 expected_residual_windows, 396 )) 397 self.assertEqual(stop_index, 2) 398 399 def test_window_observing_split_on_last_window(self): 400 restriction_tracker = OffsetRestrictionTracker(self.restriction) 401 restriction_tracker.try_claim(30) 402 (primaries, residuals, stop_index) = PerWindowInvoker._try_split( 403 0.2, 404 2, 405 3, 406 self.windowed_value, 407 self.restriction, 408 self.watermark_estimator_state, 409 self.restriction_provider, 410 restriction_tracker, 411 self.watermark_estimator) 412 # 20% of 0.7 windows = 20% of 70 offset left = 14 offset 413 # 30 + 14 = 44 split offset 414 expected_primary_split, expected_residual_split = ( 415 self.create_split_in_window(44, (self.window3, ))) 416 expected_primary_windows, _ = ( 417 self.create_split_across_windows((self.window1, self.window2, ), None)) 418 hc.assert_that( 419 primaries, 420 hc.contains_inanyorder( 421 expected_primary_split, 422 expected_primary_windows, 423 )) 424 hc.assert_that(residuals, hc.contains_inanyorder(expected_residual_split, )) 425 self.assertEqual(stop_index, 3) 426 427 def test_window_observing_split_on_first_window_fallback(self): 428 restriction_tracker = OffsetRestrictionTracker(self.restriction) 429 restriction_tracker.try_claim(100) 430 # We assume that we can't split this fully claimed restriction 431 self.assertIsNone(restriction_tracker.try_split(0)) 432 (primaries, residuals, stop_index) = PerWindowInvoker._try_split( 433 0.0, 434 0, 435 3, 436 self.windowed_value, 437 self.restriction, 438 self.watermark_estimator_state, 439 self.restriction_provider, 440 restriction_tracker, 441 self.watermark_estimator) 442 expected_primary_windows, expected_residual_windows = ( 443 self.create_split_across_windows( 444 (self.window1, ), (self.window2, self.window3, ))) 445 hc.assert_that( 446 primaries, hc.contains_inanyorder( 447 expected_primary_windows, 448 )) 449 hc.assert_that( 450 residuals, hc.contains_inanyorder( 451 expected_residual_windows, 452 )) 453 self.assertEqual(stop_index, 1) 454 455 def test_window_observing_split_on_middle_window_fallback(self): 456 restriction_tracker = OffsetRestrictionTracker(self.restriction) 457 restriction_tracker.try_claim(100) 458 # We assume that we can't split this fully claimed restriction 459 self.assertIsNone(restriction_tracker.try_split(0)) 460 (primaries, residuals, stop_index) = PerWindowInvoker._try_split( 461 0.0, 462 1, 463 3, 464 self.windowed_value, 465 self.restriction, 466 self.watermark_estimator_state, 467 self.restriction_provider, 468 restriction_tracker, 469 self.watermark_estimator) 470 expected_primary_windows, expected_residual_windows = ( 471 self.create_split_across_windows( 472 (self.window1, self.window2, ), (self.window3, ))) 473 hc.assert_that( 474 primaries, hc.contains_inanyorder( 475 expected_primary_windows, 476 )) 477 hc.assert_that( 478 residuals, hc.contains_inanyorder( 479 expected_residual_windows, 480 )) 481 self.assertEqual(stop_index, 2) 482 483 def test_window_observing_split_on_last_window_when_split_not_possible(self): 484 restriction_tracker = OffsetRestrictionTracker(self.restriction) 485 restriction_tracker.try_claim(100) 486 # We assume that we can't split this fully claimed restriction 487 self.assertIsNone(restriction_tracker.try_split(0)) 488 self.assertIsNone( 489 PerWindowInvoker._try_split( 490 0.0, 491 2, 492 3, 493 self.windowed_value, 494 self.restriction, 495 self.watermark_estimator_state, 496 self.restriction_provider, 497 restriction_tracker, 498 self.watermark_estimator)) 499 500 def test_window_observing_split_on_window_boundary_round_up(self): 501 restriction_tracker = OffsetRestrictionTracker(self.restriction) 502 restriction_tracker.try_claim(30) 503 (primaries, residuals, stop_index) = PerWindowInvoker._try_split( 504 0.6, 505 0, 506 3, 507 self.windowed_value, 508 self.restriction, 509 self.watermark_estimator_state, 510 self.restriction_provider, 511 restriction_tracker, 512 self.watermark_estimator) 513 # 60% of 2.7 windows = 60% of 270 offset left = 162 offset 514 # 30 + 162 = 192 offset --> round to end of window 2 515 expected_primary_windows, expected_residual_windows = ( 516 self.create_split_across_windows( 517 (self.window1, self.window2, ), (self.window3, ))) 518 hc.assert_that( 519 primaries, hc.contains_inanyorder( 520 expected_primary_windows, 521 )) 522 hc.assert_that( 523 residuals, hc.contains_inanyorder( 524 expected_residual_windows, 525 )) 526 self.assertEqual(stop_index, 2) 527 528 def test_window_observing_split_on_window_boundary_round_down(self): 529 restriction_tracker = OffsetRestrictionTracker(self.restriction) 530 restriction_tracker.try_claim(30) 531 (primaries, residuals, stop_index) = PerWindowInvoker._try_split( 532 0.3, 533 0, 534 3, 535 self.windowed_value, 536 self.restriction, 537 self.watermark_estimator_state, 538 self.restriction_provider, 539 restriction_tracker, 540 self.watermark_estimator) 541 # 30% of 2.7 windows = 30% of 270 offset left = 81 offset 542 # 30 + 81 = 111 offset --> round to end of window 1 543 expected_primary_windows, expected_residual_windows = ( 544 self.create_split_across_windows( 545 (self.window1, ), (self.window2, self.window3, ))) 546 hc.assert_that( 547 primaries, hc.contains_inanyorder( 548 expected_primary_windows, 549 )) 550 hc.assert_that( 551 residuals, hc.contains_inanyorder( 552 expected_residual_windows, 553 )) 554 self.assertEqual(stop_index, 1) 555 556 def test_window_observing_split_on_window_boundary_round_down_on_last_window( 557 self): 558 restriction_tracker = OffsetRestrictionTracker(self.restriction) 559 restriction_tracker.try_claim(30) 560 (primaries, residuals, stop_index) = PerWindowInvoker._try_split( 561 0.9, 562 0, 563 3, 564 self.windowed_value, 565 self.restriction, 566 self.watermark_estimator_state, 567 self.restriction_provider, 568 restriction_tracker, 569 self.watermark_estimator) 570 # 90% of 2.7 windows = 90% of 270 offset left = 243 offset 571 # 30 + 243 = 273 offset --> prefer a split so round to end of window 2 572 # instead of no split 573 expected_primary_windows, expected_residual_windows = ( 574 self.create_split_across_windows( 575 (self.window1, self.window2, ), (self.window3, ))) 576 hc.assert_that( 577 primaries, hc.contains_inanyorder( 578 expected_primary_windows, 579 )) 580 hc.assert_that( 581 residuals, hc.contains_inanyorder( 582 expected_residual_windows, 583 )) 584 self.assertEqual(stop_index, 2) 585 586 587 if __name__ == '__main__': 588 unittest.main()