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