-
Notifications
You must be signed in to change notification settings - Fork 4.5k
Expand file tree
/
Copy pathcore_test.py
More file actions
802 lines (667 loc) · 28.2 KB
/
core_test.py
File metadata and controls
802 lines (667 loc) · 28.2 KB
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
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
#
# 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.
#
"""Unit tests for the core python file."""
# pytype: skip-file
import logging
import os
import tempfile
import unittest
from typing import Iterable
from typing import Literal
from typing import TypeVar
import pytest
import apache_beam as beam
from apache_beam.coders import coders
from apache_beam.options.pipeline_options import PipelineOptions
from apache_beam.testing.util import assert_that
from apache_beam.testing.util import equal_to
from apache_beam.transforms.core import DeadLetter
from apache_beam.transforms.resources import ResourceHint
from apache_beam.transforms.userstate import BagStateSpec
from apache_beam.transforms.userstate import ReadModifyWriteStateSpec
from apache_beam.transforms.userstate import TimerSpec
from apache_beam.transforms.userstate import on_timer
from apache_beam.transforms.window import FixedWindows
from apache_beam.typehints import TypeCheckError
from apache_beam.typehints import row_type
from apache_beam.typehints import typehints
RETURN_NONE_PARTIAL_WARNING = "Process method returned None"
class TestDoFn0(beam.DoFn):
"""Returning without a value is allowed"""
def process(self, element):
if not element:
return
yield element
class TestDoFn1(beam.DoFn):
def process(self, element):
yield element
class TestDoFn2(beam.DoFn):
def process(self, element):
def inner_func(x):
yield x
return inner_func(element)
class TestDoFn3(beam.DoFn):
"""mixing return and yield is not allowed"""
def process(self, element):
if not element:
return -1
yield element
class TestDoFn4(beam.DoFn):
"""test the variable name containing return"""
def process(self, element):
my_return = element
yield my_return
class TestDoFn5(beam.DoFn):
"""test the variable name containing yield"""
def process(self, element):
my_yield = element
return my_yield
class TestDoFn6(beam.DoFn):
"""test the variable name containing return"""
def process(self, element):
return_test = element
yield return_test
class TestDoFn7(beam.DoFn):
"""test the variable name containing yield"""
def process(self, element):
yield_test = element
return yield_test
class TestDoFn8(beam.DoFn):
"""test the code containing yield and yield from"""
def process(self, element):
if not element:
yield from [1, 2, 3]
else:
yield element
class TestDoFn9(beam.DoFn):
def process(self, element):
if len(element) > 3:
raise ValueError('Not allowed to have long elements')
yield element
class TestDoFn10(beam.DoFn):
"""test process returning None explicitly"""
def process(self, element):
return None
class TestDoFn11(beam.DoFn):
"""test process returning None (no return and no yield)"""
def process(self, element):
pass
class TestDoFn12(beam.DoFn):
"""test process returning None in a filter pattern"""
def process(self, element):
if element == 0:
return
return element
class TestDoFnStateful(beam.DoFn):
STATE_SPEC = ReadModifyWriteStateSpec('num_elements', coders.VarIntCoder())
"""test process with a stateful dofn"""
def process(self, element, state=beam.DoFn.StateParam(STATE_SPEC)):
if len(element[1]) > 3:
raise ValueError('Not allowed to have long elements')
current_value = state.read() or 1
state.write(current_value + 1)
yield current_value
class TestDoFnWithTimer(beam.DoFn):
ALL_ELEMENTS = BagStateSpec('buffer', coders.VarIntCoder())
TIMER = TimerSpec('timer', beam.TimeDomain.WATERMARK)
"""test process with a stateful dofn"""
def process(
self,
element,
t=beam.DoFn.TimestampParam,
state=beam.DoFn.StateParam(ALL_ELEMENTS),
timer=beam.DoFn.TimerParam(TIMER)):
if element[1] > 3:
raise ValueError('Not allowed to have large numbers')
state.add(element[1])
timer.set(t)
return []
@on_timer(TIMER)
def expiry_callback(self, state=beam.DoFn.StateParam(ALL_ELEMENTS)):
unique_elements = list(state.read())
state.clear()
return unique_elements
class CreateTest(unittest.TestCase):
@pytest.fixture(autouse=True)
def inject_fixtures(self, caplog):
self._caplog = caplog
def test_dofn_with_yield_and_return(self):
warning_text = 'Using yield and return'
with self._caplog.at_level(logging.WARNING):
assert beam.ParDo(sum)
assert beam.ParDo(TestDoFn0())
assert beam.ParDo(TestDoFn1())
assert beam.ParDo(TestDoFn2())
assert beam.ParDo(TestDoFn4())
assert beam.ParDo(TestDoFn5())
assert beam.ParDo(TestDoFn6())
assert beam.ParDo(TestDoFn7())
assert beam.ParDo(TestDoFn8())
assert warning_text not in self._caplog.text
with self._caplog.at_level(logging.WARNING):
beam.ParDo(TestDoFn3())
assert warning_text in self._caplog.text
def test_dofn_with_explicit_return_none(self):
with self._caplog.at_level(logging.WARNING):
beam.ParDo(TestDoFn10())
assert RETURN_NONE_PARTIAL_WARNING in self._caplog.text
assert str(TestDoFn10) in self._caplog.text
def test_dofn_with_implicit_return_none_missing_return_and_yield(self):
with self._caplog.at_level(logging.WARNING):
beam.ParDo(TestDoFn11())
assert RETURN_NONE_PARTIAL_WARNING not in self._caplog.text
def test_dofn_with_implicit_return_none_and_value(self):
with self._caplog.at_level(logging.WARNING):
beam.ParDo(TestDoFn12())
assert RETURN_NONE_PARTIAL_WARNING not in self._caplog.text
class PartitionTest(unittest.TestCase):
def test_partition_with_bools(self):
with pytest.raises(
(ValueError, RuntimeError),
match=
r"PartitionFn yielded a '([^']*)' when it should only yield integers"):
# Check for RuntimeError too since the portable runner casts
# all exceptions to RuntimeError
invalid_inputs = [True, 1.2, 'string', None]
for input_value in invalid_inputs:
with beam.testing.test_pipeline.TestPipeline() as p:
_ = (
p | beam.Create([input_value])
| beam.Partition(lambda x, _: x, 2))
def test_partition_with_numpy_integers(self):
# Test that numpy integer types are correctly accepted by the
# ApplyPartitionFnFn class
import numpy as np
# Create an instance of the ApplyPartitionFnFn class
apply_partition_fn = beam.Partition.ApplyPartitionFnFn()
# Define a simple partition function
class SimplePartitionFn(beam.PartitionFn):
def partition_for(self, element, num_partitions):
return element % num_partitions
partition_fn = SimplePartitionFn()
# Test with numpy.int32
# This should not raise an exception
outputs = list(apply_partition_fn.process(np.int32(1), partition_fn, 3))
self.assertEqual(len(outputs), 1)
self.assertEqual(outputs[0].tag, '1') # 1 % 3 = 1
# Test with numpy.int64
# This should not raise an exception
outputs = list(apply_partition_fn.process(np.int64(2), partition_fn, 3))
self.assertEqual(len(outputs), 1)
self.assertEqual(outputs[0].tag, '2') # 2 % 3 = 2
def test_partition_fn_returning_numpy_integers(self):
# Test that partition functions can return numpy integer types
import numpy as np
# Create an instance of the ApplyPartitionFnFn class
apply_partition_fn = beam.Partition.ApplyPartitionFnFn()
# Define partition functions that return numpy integer types
class Int32PartitionFn(beam.PartitionFn):
def partition_for(self, element, num_partitions):
return np.int32(element % num_partitions)
class Int64PartitionFn(beam.PartitionFn):
def partition_for(self, element, num_partitions):
return np.int64(element % num_partitions)
# Test with partition function returning numpy.int32
# This should not raise an exception
outputs = list(apply_partition_fn.process(1, Int32PartitionFn(), 3))
self.assertEqual(len(outputs), 1)
self.assertEqual(outputs[0].tag, '1') # 1 % 3 = 1
# Test with partition function returning numpy.int64
# This should not raise an exception
outputs = list(apply_partition_fn.process(2, Int64PartitionFn(), 3))
self.assertEqual(len(outputs), 1)
self.assertEqual(outputs[0].tag, '2') # 2 % 3 = 2
def test_partition_boundedness(self):
def partition_fn(val, num_partitions):
return val % num_partitions
class UnboundedDoFn(beam.DoFn):
@beam.DoFn.unbounded_per_element()
def process(self, element):
yield element
with beam.testing.test_pipeline.TestPipeline() as p:
source = p | beam.Create([1, 2, 3, 4, 5])
p1, p2, p3 = source | "bounded" >> beam.Partition(partition_fn, 3)
self.assertEqual(source.is_bounded, True)
self.assertEqual(p1.is_bounded, True)
self.assertEqual(p2.is_bounded, True)
self.assertEqual(p3.is_bounded, True)
unbounded = source | beam.ParDo(UnboundedDoFn())
p4, p5, p6 = unbounded | "unbounded" >> beam.Partition(partition_fn, 3)
self.assertEqual(unbounded.is_bounded, False)
self.assertEqual(p4.is_bounded, False)
self.assertEqual(p5.is_bounded, False)
self.assertEqual(p6.is_bounded, False)
class FlattenTest(unittest.TestCase):
def test_flatten_identical_windows(self):
with beam.testing.test_pipeline.TestPipeline() as p:
source1 = p | "c1" >> beam.Create(
[1, 2, 3, 4, 5]) | "w1" >> beam.WindowInto(FixedWindows(100))
source2 = p | "c2" >> beam.Create([6, 7, 8]) | "w2" >> beam.WindowInto(
FixedWindows(100))
source3 = p | "c3" >> beam.Create([9, 10]) | "w3" >> beam.WindowInto(
FixedWindows(100))
out = (source1, source2, source3) | "flatten" >> beam.Flatten()
assert_that(out, equal_to([1, 2, 3, 4, 5, 6, 7, 8, 9, 10]))
def test_flatten_no_windows(self):
with beam.testing.test_pipeline.TestPipeline() as p:
source1 = p | "c1" >> beam.Create([1, 2, 3, 4, 5])
source2 = p | "c2" >> beam.Create([6, 7, 8])
source3 = p | "c3" >> beam.Create([9, 10])
out = (source1, source2, source3) | "flatten" >> beam.Flatten()
assert_that(out, equal_to([1, 2, 3, 4, 5, 6, 7, 8, 9, 10]))
def test_flatten_mismatched_windows(self):
with beam.testing.test_pipeline.TestPipeline() as p:
source1 = p | "c1" >> beam.Create(
[1, 2, 3, 4, 5]) | "w1" >> beam.WindowInto(FixedWindows(25))
source2 = p | "c2" >> beam.Create([6, 7, 8]) | "w2" >> beam.WindowInto(
FixedWindows(100))
source3 = p | "c3" >> beam.Create([9, 10]) | "w3" >> beam.WindowInto(
FixedWindows(100))
_ = (source1, source2, source3) | "flatten" >> beam.Flatten()
class ExceptionHandlingTest(unittest.TestCase):
def test_routes_failures(self):
with beam.Pipeline() as pipeline:
good, bad = (
pipeline | beam.Create(['abc', 'long_word', 'foo', 'bar', 'foobar'])
| beam.ParDo(TestDoFn9()).with_exception_handling()
)
bad_elements = bad | beam.Keys()
assert_that(good, equal_to(['abc', 'foo', 'bar']), 'good')
assert_that(bad_elements, equal_to(['long_word', 'foobar']), 'bad')
def test_handles_callbacks(self):
with tempfile.TemporaryDirectory() as tmp_dirname:
tmp_path = os.path.join(tmp_dirname, 'tmp_filename')
file_contents = 'random content'
def failure_callback(e, el):
if type(e) is not ValueError:
raise Exception(f'Failed to pass in correct exception, received {e}')
if el != 'foobar':
raise Exception(f'Failed to pass in correct element, received {el}')
f = open(tmp_path, "a")
logging.warning(tmp_path)
f.write(file_contents)
f.close()
with beam.Pipeline() as pipeline:
good, bad = (
pipeline | beam.Create(['abc', 'bcd', 'foo', 'bar', 'foobar'])
| beam.ParDo(TestDoFn9()).with_exception_handling(
on_failure_callback=failure_callback)
)
bad_elements = bad | beam.Keys()
assert_that(good, equal_to(['abc', 'bcd', 'foo', 'bar']), 'good')
assert_that(bad_elements, equal_to(['foobar']), 'bad')
with open(tmp_path) as f:
s = f.read()
self.assertEqual(s, file_contents)
def test_handles_no_callback_triggered(self):
with tempfile.TemporaryDirectory() as tmp_dirname:
tmp_path = os.path.join(tmp_dirname, 'tmp_filename')
file_contents = 'random content'
def failure_callback(e, el):
f = open(tmp_path, "a")
logging.warning(tmp_path)
f.write(file_contents)
f.close()
with beam.Pipeline() as pipeline:
good, bad = (
pipeline | beam.Create(['abc', 'bcd', 'foo', 'bar'])
| beam.ParDo(TestDoFn9()).with_exception_handling(
on_failure_callback=failure_callback)
)
bad_elements = bad | beam.Keys()
assert_that(good, equal_to(['abc', 'bcd', 'foo', 'bar']), 'good')
assert_that(bad_elements, equal_to([]), 'bad')
self.assertFalse(os.path.isfile(tmp_path))
def test_stateful_exception_handling(self):
with beam.Pipeline() as pipeline:
good, bad = (
pipeline | beam.Create([(1, 'abc'), (1, 'long_word'),
(1, 'foo'), (1, 'bar'), (1, 'foobar')])
| beam.ParDo(TestDoFnStateful()).with_exception_handling(
allow_unsafe_userstate_in_process=True)
)
bad_elements = bad | beam.Keys()
assert_that(good, equal_to([1, 2, 3]), 'good')
assert_that(
bad_elements, equal_to([(1, 'long_word'), (1, 'foobar')]), 'bad')
def test_timer_exception_handling(self):
with beam.Pipeline() as pipeline:
good, bad = (
pipeline | beam.Create([(1, 0), (1, 1), (1, 2), (1, 5), (1, 10)])
| beam.ParDo(TestDoFnWithTimer()).with_exception_handling(
allow_unsafe_userstate_in_process=True)
)
bad_elements = bad | beam.Keys()
assert_that(good, equal_to([0, 1, 2]), 'good')
assert_that(bad_elements, equal_to([(1, 5), (1, 10)]), 'bad')
def test_tags_with_exception_handling_then_resource_hint(self):
class TagHint(ResourceHint):
urn = 'beam:resources:tags:v1'
ResourceHint.register_resource_hint('tags', TagHint)
with beam.Pipeline() as pipeline:
ok, unused_errors = (
pipeline
| beam.Create([1])
| beam.Map(lambda x: x)
.with_exception_handling()
.with_resource_hints(tags='test_tag')
)
pd = ok.producer.transform
self.assertIsInstance(pd, beam.transforms.core.ParDo)
while hasattr(pd.fn, 'fn'):
pd = pd.fn
self.assertEqual(
pd.get_resource_hints(),
{'beam:resources:tags:v1': b'test_tag'},
)
def test_tags_with_exception_handling_timeout_then_resource_hint(self):
class TagHint(ResourceHint):
urn = 'beam:resources:tags:v1'
ResourceHint.register_resource_hint('tags', TagHint)
with beam.Pipeline() as pipeline:
ok, unused_errors = (
pipeline
| beam.Create([1])
| beam.Map(lambda x: x)
.with_exception_handling(timeout=1)
.with_resource_hints(tags='test_tag')
)
pd = ok.producer.transform
self.assertIsInstance(pd, beam.transforms.core.ParDo)
while hasattr(pd.fn, 'fn'):
pd = pd.fn
self.assertEqual(
pd.get_resource_hints(),
{'beam:resources:tags:v1': b'test_tag'},
)
def test_tags_with_resource_hint_then_exception_handling(self):
class TagHint(ResourceHint):
urn = 'beam:resources:tags:v1'
ResourceHint.register_resource_hint('tags', TagHint)
with beam.Pipeline() as pipeline:
ok, unused_errors = (
pipeline
| beam.Create([1])
| beam.Map(lambda x: x)
.with_resource_hints(tags='test_tag')
.with_exception_handling()
)
pd = ok.producer.transform
self.assertIsInstance(pd, beam.transforms.core.ParDo)
while hasattr(pd.fn, 'fn'):
pd = pd.fn
self.assertEqual(
pd.get_resource_hints(),
{'beam:resources:tags:v1': b'test_tag'},
)
def test_tags_with_resource_hint_then_exception_handling_timeout(self):
class TagHint(ResourceHint):
urn = 'beam:resources:tags:v1'
ResourceHint.register_resource_hint('tags', TagHint)
with beam.Pipeline() as pipeline:
ok, unused_errors = (
pipeline
| beam.Create([1])
| beam.Map(lambda x: x)
.with_resource_hints(tags='test_tag')
.with_exception_handling(timeout=1)
)
pd = ok.producer.transform
self.assertIsInstance(pd, beam.transforms.core.ParDo)
while hasattr(pd.fn, 'fn'):
pd = pd.fn
self.assertEqual(
pd.get_resource_hints(),
{'beam:resources:tags:v1': b'test_tag'},
)
class ExceptionHandlingWithOutputsTest(unittest.TestCase):
"""Tests for combining with_exception_handling() and with_outputs()."""
def _create_dofn_with_tagged_outputs(self):
"""A DoFn that yields tagged outputs and can raise on even numbers."""
class DoWithFailures(beam.DoFn):
def process(
self, element: int
) -> Iterable[int
| beam.pvalue.TaggedOutput[Literal['threes'], int]
| beam.pvalue.TaggedOutput[Literal['fives'], str]]:
if element % 2 == 0:
raise ValueError(f'Even numbers not allowed {element}')
if element % 3 == 0:
yield beam.pvalue.TaggedOutput('threes', element) # type: ignore[misc]
elif element % 5 == 0:
yield beam.pvalue.TaggedOutput('fives', str(element)) # type: ignore[misc]
else:
yield element
return DoWithFailures()
def test_with_exception_handling_then_with_outputs(self):
"""Direction 1: .with_exception_handling().with_outputs()"""
with beam.Pipeline() as p:
results = (
p
| beam.Create([1, 2, 3, 4, 5, 6, 7])
| beam.ParDo(self._create_dofn_with_tagged_outputs()).
with_exception_handling().with_outputs(
'threes', 'fives', main='main'))
assert_that(results.main, equal_to([1, 7]), 'main')
assert_that(results.threes, equal_to([3]), 'threes')
assert_that(results.fives, equal_to(['5']), 'fives')
bad_elements = results.bad | beam.Keys()
assert_that(bad_elements, equal_to([2, 4, 6]), 'bad')
# Verify type hints from annotations are propagated
self.assertEqual(results.main.element_type, int)
self.assertEqual(results.threes.element_type, int)
self.assertEqual(results.fives.element_type, str)
self.assertEqual(results.bad.element_type, DeadLetter[int])
def test_with_outputs_then_with_exception_handling(self):
"""Direction 2: .with_outputs().with_exception_handling()"""
with beam.Pipeline() as p:
results = (
p
| beam.Create([1, 2, 3, 4, 5, 6, 7])
| beam.ParDo(self._create_dofn_with_tagged_outputs()).with_outputs(
'threes', 'fives', main='main').with_exception_handling())
assert_that(results.main, equal_to([1, 7]), 'main')
assert_that(results.threes, equal_to([3]), 'threes')
assert_that(results.fives, equal_to(['5']), 'fives')
bad_elements = results.bad | beam.Keys()
assert_that(bad_elements, equal_to([2, 4, 6]), 'bad')
# Verify type hints from annotations are propagated
self.assertEqual(results.main.element_type, int)
self.assertEqual(results.threes.element_type, int)
self.assertEqual(results.fives.element_type, str)
self.assertEqual(results.bad.element_type, DeadLetter[int])
def test_with_outputs_then_with_exception_handling_custom_dead_letter_tag(
self):
"""Direction 2 with custom dead_letter_tag."""
with beam.Pipeline() as p:
results = (
p
| beam.Create([1, 2, 3])
| beam.ParDo(self._create_dofn_with_tagged_outputs()).with_outputs(
'threes',
main='main').with_exception_handling(dead_letter_tag='errors'))
assert_that(results.main, equal_to([1]), 'main')
assert_that(results.threes, equal_to([3]), 'threes')
bad_elements = results.errors | beam.Keys()
assert_that(bad_elements, equal_to([2]), 'errors')
self.assertEqual(results.threes.element_type, int)
self.assertEqual(results.errors.element_type, DeadLetter[int])
def test_with_exception_handling_then_with_outputs_custom_dead_letter_tag(
self):
"""Direction 1 with custom dead_letter_tag."""
with beam.Pipeline() as p:
results = (
p
| beam.Create([1, 2, 3])
| beam.ParDo(
self._create_dofn_with_tagged_outputs()).with_exception_handling(
dead_letter_tag='errors').with_outputs('threes', main='main'))
assert_that(results.main, equal_to([1]), 'main')
assert_that(results.threes, equal_to([3]), 'threes')
bad_elements = results.errors | beam.Keys()
assert_that(bad_elements, equal_to([2]), 'errors')
self.assertEqual(results.threes.element_type, int)
self.assertEqual(results.errors.element_type, DeadLetter[int])
def test_exception_handling_no_with_outputs_backward_compat(self):
"""Without with_outputs(), behavior is unchanged."""
with beam.Pipeline() as p:
good, bad = (
p
| beam.Create([1, 2, 7])
| beam.ParDo(self._create_dofn_with_tagged_outputs())
.with_exception_handling())
assert_that(good, equal_to([1, 7]), 'good')
bad_elements = bad | beam.Keys()
assert_that(bad_elements, equal_to([2]), 'bad')
def test_exception_handling_compat_version_uses_old_behavior(self):
"""With compat version < 2.73.0, old expand path is used."""
options = PipelineOptions(update_compatibility_version="2.72.0")
with beam.Pipeline(options=options) as p:
good, bad = (
p
| beam.Create([1, 2, 7])
| beam.ParDo(self._create_dofn_with_tagged_outputs())
.with_exception_handling())
assert_that(good, equal_to([1, 7]), 'good')
bad_elements = bad | beam.Keys()
assert_that(bad_elements, equal_to([2]), 'bad')
def test_exception_handling_compat_version_element_type_set_manually(self):
"""With compat version < 2.73.0, element_type is set via manual override
(the old behavior) rather than via with_output_types."""
options = PipelineOptions(update_compatibility_version="2.72.0")
with beam.Pipeline(options=options) as p:
results = (
p
| beam.Create([1, 2, 3])
| beam.ParDo(self._create_dofn_with_tagged_outputs()).
with_exception_handling().with_outputs('threes', main='main'))
# In old path, dead letter type is Any (no with_output_types call)
self.assertEqual(results.bad.element_type, typehints.Any)
# Tagged outputs still get types from DoFn Literal annotations
# (via DoOutputsTuple.__getitem__ reading tagged_output_types)
self.assertEqual(results.threes.element_type, int)
# Main output type should still be inferred via manual override
assert_that(results.main, equal_to([1]), 'main')
def test_with_outputs_then_exception_handling_with_map(self):
"""with_outputs().with_exception_handling() also works on Map."""
with beam.Pipeline() as p:
results = (
p
| beam.Create([1, 2, 3, 4, 5])
| beam.Map(lambda x: x if x % 2 != 0 else 1 / 0).with_outputs(
main='main').with_exception_handling())
assert_that(results.main, equal_to([1, 3, 5]), 'main')
bad_elements = results.bad | beam.Keys()
assert_that(bad_elements, equal_to([2, 4]), 'bad')
def test_with_output_types_chained_on_pardo(self):
"""When type hints are chained on the ParDo (not annotations on the DoFn),
tagged output types should still be propagated through
with_exception_handling().with_outputs()."""
class DoWithFailuresNoAnnotations(beam.DoFn):
def process(self, element):
if element % 2 == 0:
raise ValueError(f'Even numbers not allowed {element}')
if element % 3 == 0:
yield beam.pvalue.TaggedOutput('threes', element)
else:
yield element
with beam.Pipeline() as p:
results = (
p
| beam.Create([1, 2, 3, 7])
| beam.ParDo(DoWithFailuresNoAnnotations()).with_output_types(
int, threes=int).with_exception_handling().with_outputs(
'threes', main='main'))
assert_that(results.main, equal_to([1, 7]), 'main')
assert_that(results.threes, equal_to([3]), 'threes')
bad_elements = results.bad | beam.Keys()
assert_that(bad_elements, equal_to([2]), 'bad')
self.assertEqual(results.main.element_type, int)
self.assertEqual(results.threes.element_type, int)
def test_with_outputs_and_error_handler(self):
"""with_outputs() + error_handler should return DoOutputsTuple, not a
bare PCollection."""
from apache_beam.transforms.error_handling import ErrorHandler
with beam.Pipeline() as p:
with ErrorHandler(beam.Map(lambda x: x)) as handler:
results = (
p
| beam.Create([1, 2, 3, 4, 5, 6, 7])
| beam.ParDo(self._create_dofn_with_tagged_outputs()).with_outputs(
'threes', 'fives',
main='main').with_exception_handling(error_handler=handler))
assert_that(results.main, equal_to([1, 7]), 'main')
assert_that(results.threes, equal_to([3]), 'threes')
assert_that(results.fives, equal_to(['5']), 'fives')
def test_callablewrapper_typehint():
T = TypeVar("T")
def identity(x: T) -> T:
return x
dofn = beam.core.CallableWrapperDoFn(identity)
assert dofn.get_type_hints().strip_iterable()[1][0][0] == typehints.Any
class FlatMapTest(unittest.TestCase):
def test_default(self):
with beam.Pipeline() as pipeline:
letters = (
pipeline
| beam.Create(['abc', 'def'], reshuffle=False)
| beam.FlatMap())
assert_that(letters, equal_to(['a', 'b', 'c', 'd', 'e', 'f']))
def test_default_identity_function_with_typehint(self):
with beam.Pipeline() as pipeline:
letters = (
pipeline
| beam.Create([["abc"]], reshuffle=False)
| beam.FlatMap()
| beam.Map(lambda s: s.upper()).with_input_types(str))
assert_that(letters, equal_to(["ABC"]))
def test_typecheck_with_default(self):
with pytest.raises(TypeCheckError):
with beam.Pipeline() as pipeline:
_ = (
pipeline
| beam.Create([[1, 2, 3]], reshuffle=False)
| beam.FlatMap()
| beam.Map(lambda s: s.upper()).with_input_types(str))
class CreateInferOutputSchemaTest(unittest.TestCase):
def test_multiple_types_for_field(self):
output_type = beam.Create([beam.Row(a=1),
beam.Row(a='foo')]).infer_output_type(None)
self.assertEqual(
output_type,
row_type.RowTypeConstraint.from_fields([
('a', typehints.Union[int, str])
]))
def test_single_type_for_field(self):
output_type = beam.Create([beam.Row(a=1),
beam.Row(a=2)]).infer_output_type(None)
self.assertEqual(
output_type, row_type.RowTypeConstraint.from_fields([('a', int)]))
def test_optional_type_for_field(self):
output_type = beam.Create([beam.Row(a=1),
beam.Row(a=None)]).infer_output_type(None)
self.assertEqual(
output_type,
row_type.RowTypeConstraint.from_fields([('a', typehints.Optional[int])
]))
def test_none_type_for_field_raises_error(self):
with self.assertRaisesRegex(TypeError,
"('No types found for field %s', 'a')"):
beam.Create([beam.Row(a=None), beam.Row(a=None)]).infer_output_type(None)
if __name__ == '__main__':
logging.getLogger().setLevel(logging.INFO)
unittest.main()