-
Notifications
You must be signed in to change notification settings - Fork 4.5k
Expand file tree
/
Copy pathpubsub_test.py
More file actions
1104 lines (955 loc) · 42 KB
/
pubsub_test.py
File metadata and controls
1104 lines (955 loc) · 42 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
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
# coding=utf-8
#
# 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 PubSub sources and sinks."""
# pytype: skip-file
import logging
import unittest
import hamcrest as hc
import mock
import apache_beam as beam
from apache_beam import Pipeline
from apache_beam.io import Read
from apache_beam.io import Write
from apache_beam.io.gcp.pubsub import MultipleReadFromPubSub
from apache_beam.io.gcp.pubsub import PubsubMessage
from apache_beam.io.gcp.pubsub import PubSubSourceDescriptor
from apache_beam.io.gcp.pubsub import ReadFromPubSub
from apache_beam.io.gcp.pubsub import ReadStringsFromPubSub
from apache_beam.io.gcp.pubsub import WriteStringsToPubSub
from apache_beam.io.gcp.pubsub import WriteToPubSub
from apache_beam.io.gcp.pubsub import _PubSubSink
from apache_beam.io.gcp.pubsub import _PubSubSource
from apache_beam.metrics.metric import Lineage
from apache_beam.options.pipeline_options import PipelineOptions
from apache_beam.options.pipeline_options import StandardOptions
from apache_beam.portability import common_urns
from apache_beam.portability.api import beam_runner_api_pb2
from apache_beam.runners import pipeline_context
from apache_beam.runners.direct import transform_evaluator
from apache_beam.runners.direct.direct_runner import _DirectReadFromPubSub
from apache_beam.runners.direct.direct_runner import _get_transform_overrides
from apache_beam.runners.direct.transform_evaluator import _PubSubReadEvaluator
from apache_beam.testing import test_utils
from apache_beam.testing.test_pipeline import TestPipeline
from apache_beam.testing.util import TestWindowedValue
from apache_beam.testing.util import assert_that
from apache_beam.testing.util import equal_to
from apache_beam.transforms import window
from apache_beam.transforms.core import Create
from apache_beam.transforms.display import DisplayData
from apache_beam.transforms.display_test import DisplayDataItemMatcher
from apache_beam.utils import proto_utils
from apache_beam.utils import timestamp
# Protect against environments where the PubSub library is not available.
try:
from google.cloud import pubsub
except ImportError:
pubsub = None
class TestPubsubMessage(unittest.TestCase):
def test_payload_valid(self):
_ = PubsubMessage('', None)
_ = PubsubMessage('data', None)
_ = PubsubMessage(None, {'k': 'v'})
def test_payload_invalid(self):
with self.assertRaisesRegex(ValueError, r'data.*attributes.*must be set'):
_ = PubsubMessage(None, None)
with self.assertRaisesRegex(ValueError, r'data.*attributes.*must be set'):
_ = PubsubMessage(None, {})
@unittest.skipIf(pubsub is None, 'GCP dependencies are not installed')
def test_proto_conversion(self):
data = b'data'
attributes = {'k1': 'v1', 'k2': 'v2'}
m = PubsubMessage(data, attributes)
m_converted = PubsubMessage._from_proto_str(m._to_proto_str())
self.assertEqual(m_converted.data, data)
self.assertEqual(m_converted.attributes, attributes)
@unittest.skipIf(pubsub is None, 'GCP dependencies are not installed')
def test_payload_publish_invalid(self):
with self.assertRaisesRegex(ValueError, r'data field.*10MB'):
msg = PubsubMessage(b'0' * 1024 * 1024 * 11, None)
msg._to_proto_str(for_publish=True)
with self.assertRaisesRegex(ValueError, 'attribute key'):
msg = PubsubMessage(b'0', {'0' * 257: '0'})
msg._to_proto_str(for_publish=True)
with self.assertRaisesRegex(ValueError, 'attribute value'):
msg = PubsubMessage(b'0', {'0' * 100: '0' * 1025})
msg._to_proto_str(for_publish=True)
with self.assertRaisesRegex(ValueError, '100 attributes'):
attributes = {}
for i in range(0, 101):
attributes[str(i)] = str(i)
msg = PubsubMessage(b'0', attributes)
msg._to_proto_str(for_publish=True)
with self.assertRaisesRegex(ValueError, 'ordering key'):
msg = PubsubMessage(b'0', None, ordering_key='0' * 1301)
msg._to_proto_str(for_publish=True)
def test_eq(self):
a = PubsubMessage(b'abc', {1: 2, 3: 4})
b = PubsubMessage(b'abc', {1: 2, 3: 4})
c = PubsubMessage(b'abc', {1: 2})
self.assertTrue(a == b)
self.assertTrue(a != c)
self.assertTrue(b != c)
def test_hash(self):
a = PubsubMessage(b'abc', {1: 2, 3: 4})
b = PubsubMessage(b'abc', {1: 2, 3: 4})
c = PubsubMessage(b'abc', {1: 2})
self.assertTrue(hash(a) == hash(b))
self.assertTrue(hash(a) != hash(c))
self.assertTrue(hash(b) != hash(c))
def test_repr(self):
a = PubsubMessage(b'abc', {1: 2, 3: 4})
b = PubsubMessage(b'abc', {1: 2, 3: 4})
c = PubsubMessage(b'abc', {1: 2})
self.assertTrue(repr(a) == repr(b))
self.assertTrue(repr(a) != repr(c))
self.assertTrue(repr(b) != repr(c))
@unittest.skipIf(pubsub is None, 'GCP dependencies are not installed')
class TestReadFromPubSubOverride(unittest.TestCase):
def test_expand_with_topic(self):
options = PipelineOptions([])
options.view_as(StandardOptions).streaming = True
p = TestPipeline(options=options)
pcoll = (
p
| ReadFromPubSub(
'projects/fakeprj/topics/a_topic',
None,
'a_label',
with_attributes=False,
timestamp_attribute=None)
| beam.Map(lambda x: x))
self.assertEqual(bytes, pcoll.element_type)
# Apply the necessary PTransformOverrides.
overrides = _get_transform_overrides(options)
p.replace_all(overrides)
# Note that the direct output of ReadFromPubSub will be replaced
# by a PTransformOverride, so we use a no-op Map.
read_transform = pcoll.producer.inputs[0].producer.transform
# Ensure that the properties passed through correctly
source = read_transform._source
self.assertEqual('a_topic', source.topic_name)
self.assertEqual('a_label', source.id_label)
def test_expand_with_subscription(self):
options = PipelineOptions([])
options.view_as(StandardOptions).streaming = True
p = TestPipeline(options=options)
pcoll = (
p
| ReadFromPubSub(
None,
'projects/fakeprj/subscriptions/a_subscription',
'a_label',
with_attributes=False,
timestamp_attribute=None)
| beam.Map(lambda x: x))
self.assertEqual(bytes, pcoll.element_type)
# Apply the necessary PTransformOverrides.
overrides = _get_transform_overrides(options)
p.replace_all(overrides)
# Note that the direct output of ReadFromPubSub will be replaced
# by a PTransformOverride, so we use a no-op Map.
read_transform = pcoll.producer.inputs[0].producer.transform
# Ensure that the properties passed through correctly
source = read_transform._source
self.assertEqual('a_subscription', source.subscription_name)
self.assertEqual('a_label', source.id_label)
def test_expand_with_no_topic_or_subscription(self):
with self.assertRaisesRegex(
ValueError, "Either a topic or subscription must be provided."):
ReadFromPubSub(
None,
None,
'a_label',
with_attributes=False,
timestamp_attribute=None)
def test_expand_with_both_topic_and_subscription(self):
with self.assertRaisesRegex(
ValueError, "Only one of topic or subscription should be provided."):
ReadFromPubSub(
'a_topic',
'a_subscription',
'a_label',
with_attributes=False,
timestamp_attribute=None)
def test_expand_with_other_options(self):
options = PipelineOptions([])
options.view_as(StandardOptions).streaming = True
p = TestPipeline(options=options)
pcoll = (
p
| ReadFromPubSub(
'projects/fakeprj/topics/a_topic',
None,
'a_label',
with_attributes=True,
timestamp_attribute='time')
| beam.Map(lambda x: x))
self.assertEqual(PubsubMessage, pcoll.element_type)
# Apply the necessary PTransformOverrides.
overrides = _get_transform_overrides(options)
p.replace_all(overrides)
# Note that the direct output of ReadFromPubSub will be replaced
# by a PTransformOverride, so we use a no-op Map.
read_transform = pcoll.producer.inputs[0].producer.transform
# Ensure that the properties passed through correctly
source = read_transform._source
self.assertTrue(source.with_attributes)
self.assertEqual('time', source.timestamp_attribute)
@unittest.skipIf(pubsub is None, 'GCP dependencies are not installed')
class TestMultiReadFromPubSubOverride(unittest.TestCase):
def test_expand_with_multiple_sources(self):
options = PipelineOptions([])
options.view_as(StandardOptions).streaming = True
p = TestPipeline(options=options)
topics = [
'projects/fakeprj/topics/a_topic', 'projects/fakeprj2/topics/b_topic'
]
subscriptions = ['projects/fakeprj/subscriptions/a_subscription']
pubsub_sources = [
PubSubSourceDescriptor(descriptor)
for descriptor in topics + subscriptions
]
pcoll = (p | MultipleReadFromPubSub(pubsub_sources) | beam.Map(lambda x: x))
# Apply the necessary PTransformOverrides.
overrides = _get_transform_overrides(options)
p.replace_all(overrides)
self.assertEqual(bytes, pcoll.element_type)
# Ensure that the sources are passed through correctly
read_transforms = pcoll.producer.inputs[0].producer.inputs
topics_list = []
subscription_list = []
for read_transform in read_transforms:
source = read_transform.producer.transform._source
if source.full_topic:
topics_list.append(source.full_topic)
else:
subscription_list.append(source.full_subscription)
self.assertEqual(topics_list, topics)
self.assertEqual(subscription_list, subscriptions)
def test_expand_with_multiple_sources_and_attributes(self):
options = PipelineOptions([])
options.view_as(StandardOptions).streaming = True
p = TestPipeline(options=options)
topics = [
'projects/fakeprj/topics/a_topic', 'projects/fakeprj2/topics/b_topic'
]
subscriptions = ['projects/fakeprj/subscriptions/a_subscription']
pubsub_sources = [
PubSubSourceDescriptor(descriptor)
for descriptor in topics + subscriptions
]
pcoll = (
p | MultipleReadFromPubSub(pubsub_sources, with_attributes=True)
| beam.Map(lambda x: x))
# Apply the necessary PTransformOverrides.
overrides = _get_transform_overrides(options)
p.replace_all(overrides)
self.assertEqual(PubsubMessage, pcoll.element_type)
# Ensure that the sources are passed through correctly
read_transforms = pcoll.producer.inputs[0].producer.inputs
topics_list = []
subscription_list = []
for read_transform in read_transforms:
source = read_transform.producer.transform._source
if source.full_topic:
topics_list.append(source.full_topic)
else:
subscription_list.append(source.full_subscription)
self.assertEqual(topics_list, topics)
self.assertEqual(subscription_list, subscriptions)
def test_expand_with_multiple_sources_and_other_options(self):
options = PipelineOptions([])
options.view_as(StandardOptions).streaming = True
p = TestPipeline(options=options)
sources = [
'projects/fakeprj/topics/a_topic',
'projects/fakeprj2/topics/b_topic',
'projects/fakeprj/subscriptions/a_subscription'
]
id_labels = ['a_label_topic', 'b_label_topic', 'a_label_subscription']
timestamp_attributes = ['a_ta_topic', 'b_ta_topic', 'a_ta_subscription']
pubsub_sources = [
PubSubSourceDescriptor(
source=source,
id_label=id_label,
timestamp_attribute=timestamp_attribute)
for source, id_label, timestamp_attribute in zip(
sources, id_labels, timestamp_attributes)
]
pcoll = (p | MultipleReadFromPubSub(pubsub_sources) | beam.Map(lambda x: x))
# Apply the necessary PTransformOverrides.
overrides = _get_transform_overrides(options)
p.replace_all(overrides)
self.assertEqual(bytes, pcoll.element_type)
# Ensure that the sources are passed through correctly
read_transforms = pcoll.producer.inputs[0].producer.inputs
for i, read_transform in enumerate(read_transforms):
id_label = id_labels[i]
timestamp_attribute = timestamp_attributes[i]
source = read_transform.producer.transform._source
self.assertEqual(source.id_label, id_label)
self.assertEqual(source.with_attributes, False)
self.assertEqual(source.timestamp_attribute, timestamp_attribute)
def test_expand_with_wrong_source(self):
with self.assertRaisesRegex(
ValueError,
r'PubSub source descriptor must be in the form '
r'"projects/<project>/topics/<topic>"'
' or "projects/<project>/subscription/<subscription>".*'):
MultipleReadFromPubSub([PubSubSourceDescriptor('not_a_proper_source')])
@unittest.skipIf(pubsub is None, 'GCP dependencies are not installed')
class TestWriteStringsToPubSubOverride(unittest.TestCase):
@mock.patch.object(Pipeline, '_assert_not_applying_PDone', mock.Mock())
def test_expand_deprecated(self):
options = PipelineOptions([])
options.view_as(StandardOptions).streaming = True
p = TestPipeline(options=options)
pcoll = (
p
| ReadFromPubSub('projects/fakeprj/topics/baz')
| WriteStringsToPubSub('projects/fakeprj/topics/a_topic')
| beam.Map(lambda x: x))
# Apply the necessary PTransformOverrides.
overrides = _get_transform_overrides(options)
p.replace_all(overrides)
# Note that the direct output of ReadFromPubSub will be replaced
# by a PTransformOverride, so we use a no-op Map.
write_transform = pcoll.producer.inputs[0].producer.transform
# Ensure that the properties passed through correctly
self.assertEqual('a_topic', write_transform.dofn.short_topic_name)
@mock.patch.object(Pipeline, '_assert_not_applying_PDone', mock.Mock())
def test_expand(self):
options = PipelineOptions([])
options.view_as(StandardOptions).streaming = True
p = TestPipeline(options=options)
pcoll = (
p
| ReadFromPubSub('projects/fakeprj/topics/baz')
| beam.Map(lambda x: PubsubMessage(x))
| WriteToPubSub(
'projects/fakeprj/topics/a_topic', with_attributes=True)
| beam.Map(lambda x: x))
# Apply the necessary PTransformOverrides.
overrides = _get_transform_overrides(options)
p.replace_all(overrides)
# Note that the direct output of ReadFromPubSub will be replaced
# by a PTransformOverride, so we use a no-op Map.
write_transform = pcoll.producer.inputs[0].producer.transform
# Ensure that the properties passed through correctly
self.assertEqual('a_topic', write_transform.dofn.short_topic_name)
self.assertEqual(True, write_transform.dofn.with_attributes)
# TODO(https://github.com/apache/beam/issues/18939): These properties
# aren't supported yet in direct runner.
self.assertEqual(None, write_transform.dofn.id_label)
self.assertEqual(None, write_transform.dofn.timestamp_attribute)
@unittest.skipIf(pubsub is None, 'GCP dependencies are not installed')
class TestPubSubSource(unittest.TestCase):
def test_display_data_topic(self):
source = _PubSubSource('projects/fakeprj/topics/a_topic', None, 'a_label')
dd = DisplayData.create_from(source)
expected_items = [
DisplayDataItemMatcher('topic', 'projects/fakeprj/topics/a_topic'),
DisplayDataItemMatcher('id_label', 'a_label'),
DisplayDataItemMatcher('with_attributes', False),
]
hc.assert_that(dd.items, hc.contains_inanyorder(*expected_items))
def test_display_data_subscription(self):
source = _PubSubSource(
None, 'projects/fakeprj/subscriptions/a_subscription', 'a_label')
dd = DisplayData.create_from(source)
expected_items = [
DisplayDataItemMatcher(
'subscription', 'projects/fakeprj/subscriptions/a_subscription'),
DisplayDataItemMatcher('id_label', 'a_label'),
DisplayDataItemMatcher('with_attributes', False),
]
hc.assert_that(dd.items, hc.contains_inanyorder(*expected_items))
def test_display_data_no_subscription(self):
source = _PubSubSource('projects/fakeprj/topics/a_topic')
dd = DisplayData.create_from(source)
expected_items = [
DisplayDataItemMatcher('topic', 'projects/fakeprj/topics/a_topic'),
DisplayDataItemMatcher('with_attributes', False),
]
hc.assert_that(dd.items, hc.contains_inanyorder(*expected_items))
@unittest.skipIf(pubsub is None, 'GCP dependencies are not installed')
class TestPubSubSink(unittest.TestCase):
def test_display_data(self):
sink = WriteToPubSub(
'projects/fakeprj/topics/a_topic',
id_label='id',
timestamp_attribute='time')
dd = DisplayData.create_from(sink)
expected_items = [
DisplayDataItemMatcher('topic', 'projects/fakeprj/topics/a_topic'),
DisplayDataItemMatcher('id_label', 'id'),
DisplayDataItemMatcher('with_attributes', True),
DisplayDataItemMatcher('timestamp_attribute', 'time'),
]
hc.assert_that(dd.items, hc.contains_inanyorder(*expected_items))
class TestPubSubReadEvaluator(object):
"""Wrapper of _PubSubReadEvaluator that makes it bounded."""
_pubsub_read_evaluator = _PubSubReadEvaluator
def __init__(self, *args, **kwargs):
self._evaluator = self._pubsub_read_evaluator(*args, **kwargs)
def start_bundle(self):
return self._evaluator.start_bundle()
def process_element(self, element):
return self._evaluator.process_element(element)
def finish_bundle(self):
result = self._evaluator.finish_bundle()
result.unprocessed_bundles = []
result.keyed_watermark_holds = {None: None}
return result
transform_evaluator.TransformEvaluatorRegistry._test_evaluators_overrides = {
_DirectReadFromPubSub: TestPubSubReadEvaluator, # type: ignore[dict-item]
}
@unittest.skipIf(pubsub is None, 'GCP dependencies are not installed')
@mock.patch('google.cloud.pubsub.SubscriberClient')
class TestReadFromPubSub(unittest.TestCase):
def test_read_messages_success(self, mock_pubsub):
data = b'data'
publish_time_secs = 1520861821
publish_time_nanos = 234567000
attributes = {'key': 'value'}
ack_id = 'ack_id'
pull_response = test_utils.create_pull_response([
test_utils.PullResponseMessage(
data, attributes, publish_time_secs, publish_time_nanos, ack_id)
])
expected_elements = [
TestWindowedValue(
PubsubMessage(data, attributes),
timestamp.Timestamp(1520861821.234567), [window.GlobalWindow()])
]
mock_pubsub.return_value.pull.return_value = pull_response
options = PipelineOptions([])
options.view_as(StandardOptions).streaming = True
with TestPipeline(options=options) as p:
pcoll = (
p
| ReadFromPubSub(
'projects/fakeprj/topics/a_topic',
None,
None,
with_attributes=True))
assert_that(pcoll, equal_to(expected_elements), reify_windows=True)
mock_pubsub.return_value.acknowledge.assert_has_calls(
[mock.call(subscription=mock.ANY, ack_ids=[ack_id])])
mock_pubsub.return_value.close.assert_has_calls([mock.call()])
def test_read_strings_success(self, mock_pubsub):
data = '🤷 ¯\\_(ツ)_/¯'
data_encoded = data.encode('utf-8')
ack_id = 'ack_id'
pull_response = test_utils.create_pull_response(
[test_utils.PullResponseMessage(data_encoded, ack_id=ack_id)])
expected_elements = [data]
mock_pubsub.return_value.pull.return_value = pull_response
options = PipelineOptions([])
options.view_as(StandardOptions).streaming = True
with TestPipeline(options=options) as p:
pcoll = (
p
| ReadStringsFromPubSub(
'projects/fakeprj/topics/a_topic', None, None))
assert_that(pcoll, equal_to(expected_elements))
mock_pubsub.return_value.acknowledge.assert_has_calls(
[mock.call(subscription=mock.ANY, ack_ids=[ack_id])])
mock_pubsub.return_value.close.assert_has_calls([mock.call()])
def test_read_data_success(self, mock_pubsub):
data_encoded = '🤷 ¯\\_(ツ)_/¯'.encode('utf-8')
ack_id = 'ack_id'
pull_response = test_utils.create_pull_response(
[test_utils.PullResponseMessage(data_encoded, ack_id=ack_id)])
expected_elements = [data_encoded]
mock_pubsub.return_value.pull.return_value = pull_response
options = PipelineOptions([])
options.view_as(StandardOptions).streaming = True
with TestPipeline(options=options) as p:
pcoll = (
p
| ReadFromPubSub('projects/fakeprj/topics/a_topic', None, None))
assert_that(pcoll, equal_to(expected_elements))
mock_pubsub.return_value.acknowledge.assert_has_calls(
[mock.call(subscription=mock.ANY, ack_ids=[ack_id])])
mock_pubsub.return_value.close.assert_has_calls([mock.call()])
def test_read_messages_timestamp_attribute_milli_success(self, mock_pubsub):
data = b'data'
attributes = {'time': '1337'}
publish_time_secs = 1520861821
publish_time_nanos = 234567000
ack_id = 'ack_id'
pull_response = test_utils.create_pull_response([
test_utils.PullResponseMessage(
data, attributes, publish_time_secs, publish_time_nanos, ack_id)
])
expected_elements = [
TestWindowedValue(
PubsubMessage(data, attributes),
timestamp.Timestamp(micros=int(attributes['time']) * 1000),
[window.GlobalWindow()]),
]
mock_pubsub.return_value.pull.return_value = pull_response
options = PipelineOptions([])
options.view_as(StandardOptions).streaming = True
with TestPipeline(options=options) as p:
pcoll = (
p
| ReadFromPubSub(
'projects/fakeprj/topics/a_topic',
None,
None,
with_attributes=True,
timestamp_attribute='time'))
assert_that(pcoll, equal_to(expected_elements), reify_windows=True)
mock_pubsub.return_value.acknowledge.assert_has_calls(
[mock.call(subscription=mock.ANY, ack_ids=[ack_id])])
mock_pubsub.return_value.close.assert_has_calls([mock.call()])
def test_read_messages_timestamp_attribute_rfc3339_success(self, mock_pubsub):
data = b'data'
attributes = {'time': '2018-03-12T13:37:01.234567Z'}
publish_time_secs = 1337000000
publish_time_nanos = 133700000
ack_id = 'ack_id'
pull_response = test_utils.create_pull_response([
test_utils.PullResponseMessage(
data, attributes, publish_time_secs, publish_time_nanos, ack_id)
])
expected_elements = [
TestWindowedValue(
PubsubMessage(data, attributes),
timestamp.Timestamp.from_rfc3339(attributes['time']),
[window.GlobalWindow()]),
]
mock_pubsub.return_value.pull.return_value = pull_response
options = PipelineOptions([])
options.view_as(StandardOptions).streaming = True
with TestPipeline(options=options) as p:
pcoll = (
p
| ReadFromPubSub(
'projects/fakeprj/topics/a_topic',
None,
None,
with_attributes=True,
timestamp_attribute='time'))
assert_that(pcoll, equal_to(expected_elements), reify_windows=True)
mock_pubsub.return_value.acknowledge.assert_has_calls(
[mock.call(subscription=mock.ANY, ack_ids=[ack_id])])
mock_pubsub.return_value.close.assert_has_calls([mock.call()])
def test_read_messages_timestamp_attribute_missing(self, mock_pubsub):
data = b'data'
attributes = {}
publish_time_secs = 1520861821
publish_time_nanos = 234567000
publish_time = '2018-03-12T13:37:01.234567Z'
ack_id = 'ack_id'
pull_response = test_utils.create_pull_response([
test_utils.PullResponseMessage(
data, attributes, publish_time_secs, publish_time_nanos, ack_id)
])
expected_elements = [
TestWindowedValue(
PubsubMessage(data, attributes),
timestamp.Timestamp.from_rfc3339(publish_time),
[window.GlobalWindow()]),
]
mock_pubsub.return_value.pull.return_value = pull_response
options = PipelineOptions([])
options.view_as(StandardOptions).streaming = True
with TestPipeline(options=options) as p:
pcoll = (
p
| ReadFromPubSub(
'projects/fakeprj/topics/a_topic',
None,
None,
with_attributes=True,
timestamp_attribute='nonexistent'))
assert_that(pcoll, equal_to(expected_elements), reify_windows=True)
mock_pubsub.return_value.acknowledge.assert_has_calls(
[mock.call(subscription=mock.ANY, ack_ids=[ack_id])])
mock_pubsub.return_value.close.assert_has_calls([mock.call()])
def test_read_messages_timestamp_attribute_fail_parse(self, mock_pubsub):
data = b'data'
attributes = {'time': '1337 unparseable'}
publish_time_secs = 1520861821
publish_time_nanos = 234567000
ack_id = 'ack_id'
pull_response = test_utils.create_pull_response([
test_utils.PullResponseMessage(
data, attributes, publish_time_secs, publish_time_nanos, ack_id)
])
mock_pubsub.return_value.pull.return_value = pull_response
options = PipelineOptions([])
options.view_as(StandardOptions).streaming = True
p = TestPipeline(options=options)
_ = (
p
| ReadFromPubSub(
'projects/fakeprj/topics/a_topic',
None,
None,
with_attributes=True,
timestamp_attribute='time'))
with self.assertRaisesRegex(ValueError, r'parse'):
p.run()
mock_pubsub.return_value.acknowledge.assert_not_called()
mock_pubsub.return_value.close.assert_has_calls([mock.call()])
def test_read_message_id_label_unsupported(self, unused_mock_pubsub):
# id_label is unsupported in DirectRunner.
options = PipelineOptions([])
options.view_as(StandardOptions).streaming = True
with self.assertRaisesRegex(NotImplementedError,
r'id_label is not supported'):
with TestPipeline(options=options) as p:
_ = (
p | ReadFromPubSub(
'projects/fakeprj/topics/a_topic', None, 'a_label'))
def test_runner_api_transformation_with_topic(self, unused_mock_pubsub):
source = _PubSubSource(
topic='projects/fakeprj/topics/a_topic',
subscription=None,
id_label='a_label',
timestamp_attribute='b_label',
with_attributes=True)
transform = Read(source)
context = pipeline_context.PipelineContext()
proto_transform_spec = transform.to_runner_api(context)
self.assertEqual(
common_urns.composites.PUBSUB_READ.urn, proto_transform_spec.urn)
pubsub_read_payload = (
proto_utils.parse_Bytes(
proto_transform_spec.payload,
beam_runner_api_pb2.PubSubReadPayload))
self.assertEqual(
'projects/fakeprj/topics/a_topic', pubsub_read_payload.topic)
self.assertEqual('a_label', pubsub_read_payload.id_attribute)
self.assertEqual('b_label', pubsub_read_payload.timestamp_attribute)
self.assertEqual('', pubsub_read_payload.subscription)
self.assertTrue(pubsub_read_payload.with_attributes)
proto_transform = beam_runner_api_pb2.PTransform(
unique_name="dummy_label", spec=proto_transform_spec)
transform_from_proto = Read.from_runner_api_parameter(
proto_transform, pubsub_read_payload, None)
self.assertTrue(isinstance(transform_from_proto, Read))
self.assertTrue(isinstance(transform_from_proto.source, _PubSubSource))
self.assertEqual(
'projects/fakeprj/topics/a_topic',
transform_from_proto.source.full_topic)
self.assertTrue(transform_from_proto.source.with_attributes)
def test_runner_api_transformation_properties_none(self, unused_mock_pubsub):
# Confirming that properties stay None after a runner API transformation.
source = _PubSubSource(
topic='projects/fakeprj/topics/a_topic', with_attributes=True)
transform = Read(source)
context = pipeline_context.PipelineContext()
proto_transform_spec = transform.to_runner_api(context)
self.assertEqual(
common_urns.composites.PUBSUB_READ.urn, proto_transform_spec.urn)
pubsub_read_payload = (
proto_utils.parse_Bytes(
proto_transform_spec.payload,
beam_runner_api_pb2.PubSubReadPayload))
proto_transform = beam_runner_api_pb2.PTransform(
unique_name="dummy_label", spec=proto_transform_spec)
transform_from_proto = Read.from_runner_api_parameter(
proto_transform, pubsub_read_payload, None)
self.assertIsNone(transform_from_proto.source.full_subscription)
self.assertIsNone(transform_from_proto.source.id_label)
self.assertIsNone(transform_from_proto.source.timestamp_attribute)
def test_runner_api_transformation_with_subscription(
self, unused_mock_pubsub):
source = _PubSubSource(
topic=None,
subscription='projects/fakeprj/subscriptions/a_subscription',
id_label='a_label',
timestamp_attribute='b_label',
with_attributes=True)
transform = Read(source)
context = pipeline_context.PipelineContext()
proto_transform_spec = transform.to_runner_api(context)
self.assertEqual(
common_urns.composites.PUBSUB_READ.urn, proto_transform_spec.urn)
pubsub_read_payload = (
proto_utils.parse_Bytes(
proto_transform_spec.payload,
beam_runner_api_pb2.PubSubReadPayload))
self.assertEqual(
'projects/fakeprj/subscriptions/a_subscription',
pubsub_read_payload.subscription)
self.assertEqual('a_label', pubsub_read_payload.id_attribute)
self.assertEqual('b_label', pubsub_read_payload.timestamp_attribute)
self.assertEqual('', pubsub_read_payload.topic)
self.assertTrue(pubsub_read_payload.with_attributes)
proto_transform = beam_runner_api_pb2.PTransform(
unique_name="dummy_label", spec=proto_transform_spec)
transform_from_proto = Read.from_runner_api_parameter(
proto_transform, pubsub_read_payload, None)
self.assertTrue(isinstance(transform_from_proto, Read))
self.assertTrue(isinstance(transform_from_proto.source, _PubSubSource))
self.assertTrue(transform_from_proto.source.with_attributes)
self.assertEqual(
'projects/fakeprj/subscriptions/a_subscription',
transform_from_proto.source.full_subscription)
def test_read_from_pubsub_no_overwrite(self, unused_mock):
expected_elements = [
TestWindowedValue(
b'apache',
timestamp.Timestamp(1520861826.234567), [window.GlobalWindow()]),
TestWindowedValue(
b'beam',
timestamp.Timestamp(1520861824.234567), [window.GlobalWindow()])
]
options = PipelineOptions([])
options.view_as(StandardOptions).streaming = True
# TODO(https://github.com/apache/beam/issues/34549): This test relies on
# lineage metrics which Prism doesn't seem to handle correctly. Defaulting
# to FnApiRunner instead.
options.view_as(StandardOptions).runner = 'FnApiRunner'
for test_case in ('topic', 'subscription'):
with TestPipeline(options=options) as p:
# Direct runner currently overwrites the whole ReadFromPubSub transform.
# This test part of composite transform without overwrite.
pcoll = p | beam.Create([b'apache', b'beam']) | beam.Map(
lambda x: window.TimestampedValue(x, 1520861820.234567 + len(x)))
args = {test_case: f'projects/fakeprj/{test_case}s/topic_or_sub'}
pcoll = ReadFromPubSub(**args).expand_continued(pcoll)
assert_that(pcoll, equal_to(expected_elements), reify_windows=True)
self.assertSetEqual(
Lineage.query(p.result.metrics(), Lineage.SOURCE),
set([f"pubsub:{test_case}:fakeprj.topic_or_sub"]))
@unittest.skipIf(pubsub is None, 'GCP dependencies are not installed')
@mock.patch('google.cloud.pubsub.PublisherClient')
class TestWriteToPubSub(unittest.TestCase):
def test_write_messages_success(self, mock_pubsub):
data = 'data'
payloads = [data]
options = PipelineOptions([])
options.view_as(StandardOptions).streaming = True
with TestPipeline(options=options) as p:
_ = (
p
| Create(payloads)
| WriteToPubSub(
'projects/fakeprj/topics/a_topic', with_attributes=False))
# Verify that publish was called (data will be protobuf serialized)
mock_pubsub.return_value.publish.assert_called()
# Check that the call was made with the topic and some data
call_args = mock_pubsub.return_value.publish.call_args
self.assertEqual(len(call_args[0]), 2) # topic and data
def test_write_messages_deprecated(self, mock_pubsub):
data = 'data'
payloads = [data]
options = PipelineOptions([])
options.view_as(StandardOptions).streaming = True
with TestPipeline(options=options) as p:
_ = (
p
| Create(payloads)
| WriteStringsToPubSub('projects/fakeprj/topics/a_topic'))
# Verify that publish was called (data will be protobuf serialized)
mock_pubsub.return_value.publish.assert_called()
# Check that the call was made with the topic and some data
call_args = mock_pubsub.return_value.publish.call_args
self.assertEqual(len(call_args[0]), 2) # topic and data
def test_write_messages_with_attributes_success(self, mock_pubsub):
data = b'data'
attributes = {'key': 'value'}
payloads = [PubsubMessage(data, attributes)]
options = PipelineOptions([])
options.view_as(StandardOptions).streaming = True
with TestPipeline(options=options) as p:
_ = (
p
| Create(payloads)
| WriteToPubSub(
'projects/fakeprj/topics/a_topic', with_attributes=True))
# Verify that publish was called (data will be protobuf serialized)
mock_pubsub.return_value.publish.assert_called()
# Check that the call was made with the topic and some data
call_args = mock_pubsub.return_value.publish.call_args
self.assertEqual(len(call_args[0]), 2) # topic and data
def test_write_messages_batch_mode_success(self, mock_pubsub):
"""Test WriteToPubSub works in batch mode (non-streaming)."""
data = 'data'
payloads = [data]
options = PipelineOptions([])
# Explicitly set streaming to False for batch mode
options.view_as(StandardOptions).streaming = False
with TestPipeline(options=options) as p:
_ = (
p
| Create(payloads)
| WriteToPubSub(
'projects/fakeprj/topics/a_topic', with_attributes=False))
# Verify that publish was called (data will be protobuf serialized)
mock_pubsub.return_value.publish.assert_called()
# Check that the call was made with the topic and some data
call_args = mock_pubsub.return_value.publish.call_args
self.assertEqual(len(call_args[0]), 2) # topic and data
def test_write_messages_with_attributes_batch_mode_success(self, mock_pubsub):
"""Test WriteToPubSub with attributes works in batch mode."""
data = b'data'
attributes = {'key': 'value'}
payloads = [PubsubMessage(data, attributes)]
options = PipelineOptions([])
# Explicitly set streaming to False for batch mode
options.view_as(StandardOptions).streaming = False
with TestPipeline(options=options) as p:
_ = (
p
| Create(payloads)
| WriteToPubSub(
'projects/fakeprj/topics/a_topic', with_attributes=True))
# Verify that publish was called (data will be protobuf serialized)
mock_pubsub.return_value.publish.assert_called()
# Check that the call was made with the topic and some data
call_args = mock_pubsub.return_value.publish.call_args
self.assertEqual(len(call_args[0]), 2) # topic and data
def test_write_messages_with_attributes_error(self, mock_pubsub):
data = 'data'
# Sending raw data when WriteToPubSub expects a PubsubMessage object.
payloads = [data]
options = PipelineOptions([])
options.view_as(StandardOptions).streaming = True
with self.assertRaisesRegex(Exception,
r'requires.*PubsubMessage.*applied.*str'):
with TestPipeline(options=options) as p:
_ = (
p
| Create(payloads)
| WriteToPubSub(
'projects/fakeprj/topics/a_topic', with_attributes=True))
def test_write_messages_unsupported_features(self, mock_pubsub):
data = b'data'
attributes = {'key': 'value'}
payloads = [PubsubMessage(data, attributes)]
options = PipelineOptions([])
options.view_as(StandardOptions).streaming = True
with self.assertRaisesRegex(NotImplementedError,
r'id_label is not supported'):
with TestPipeline(options=options) as p:
_ = (
p
| Create(payloads)
| WriteToPubSub(
'projects/fakeprj/topics/a_topic',
id_label='a_label',
with_attributes=True))
options = PipelineOptions([])
options.view_as(StandardOptions).streaming = True
with self.assertRaisesRegex(NotImplementedError,
r'timestamp_attribute is not supported'):
with TestPipeline(options=options) as p:
_ = (
p
| Create(payloads)
| WriteToPubSub(
'projects/fakeprj/topics/a_topic',
timestamp_attribute='timestamp',
with_attributes=True))