-
Notifications
You must be signed in to change notification settings - Fork 4.3k
/
bigquery_test.py
2693 lines (2416 loc) · 98.1 KB
/
bigquery_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
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
#
# 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 BigQuery sources and sinks."""
# pytype: skip-file
import datetime
import decimal
import gc
import json
import logging
import os
import pickle
import re
import secrets
import time
import unittest
import uuid
import hamcrest as hc
import mock
import pytest
import pytz
import requests
from parameterized import param
from parameterized import parameterized
import apache_beam as beam
from apache_beam.internal import pickler
from apache_beam.internal.gcp.json_value import to_json_value
from apache_beam.io.filebasedsink_test import _TestCaseWithTempDirCleanUp
from apache_beam.io.filesystems import FileSystems
from apache_beam.io.gcp import bigquery as beam_bq
from apache_beam.io.gcp import bigquery_tools
from apache_beam.io.gcp.bigquery import ReadFromBigQuery
from apache_beam.io.gcp.bigquery import TableRowJsonCoder
from apache_beam.io.gcp.bigquery import WriteToBigQuery
from apache_beam.io.gcp.bigquery import _StreamToBigQuery
from apache_beam.io.gcp.bigquery_read_internal import _BigQueryReadSplit
from apache_beam.io.gcp.bigquery_read_internal import _JsonToDictCoder
from apache_beam.io.gcp.bigquery_read_internal import bigquery_export_destination_uri
from apache_beam.io.gcp.bigquery_tools import JSON_COMPLIANCE_ERROR
from apache_beam.io.gcp.bigquery_tools import BigQueryWrapper
from apache_beam.io.gcp.bigquery_tools import RetryStrategy
from apache_beam.io.gcp.internal.clients import bigquery
from apache_beam.io.gcp.pubsub import ReadFromPubSub
from apache_beam.io.gcp.tests import utils
from apache_beam.io.gcp.tests.bigquery_matcher import BigqueryFullResultMatcher
from apache_beam.io.gcp.tests.bigquery_matcher import BigqueryFullResultStreamingMatcher
from apache_beam.io.gcp.tests.bigquery_matcher import BigQueryTableMatcher
from apache_beam.metrics.metric import Lineage
from apache_beam.options import value_provider
from apache_beam.options.pipeline_options import PipelineOptions
from apache_beam.options.pipeline_options import StandardOptions
from apache_beam.options.value_provider import RuntimeValueProvider
from apache_beam.options.value_provider import StaticValueProvider
from apache_beam.runners.dataflow.test_dataflow_runner import TestDataflowRunner
from apache_beam.runners.runner import PipelineState
from apache_beam.testing import test_utils
from apache_beam.testing.pipeline_verifiers import PipelineStateMatcher
from apache_beam.testing.test_pipeline import TestPipeline
from apache_beam.testing.test_stream import TestStream
from apache_beam.testing.util import assert_that
from apache_beam.testing.util import equal_to
from apache_beam.transforms.display import DisplayData
from apache_beam.transforms.display_test import DisplayDataItemMatcher
# Protect against environments where bigquery library is not available.
# pylint: disable=wrong-import-order, wrong-import-position
try:
from apache_beam.io.gcp.internal.clients.bigquery import bigquery_v2_client
from apitools.base.py.exceptions import HttpError
from apitools.base.py.exceptions import HttpForbiddenError
from google.cloud import bigquery as gcp_bigquery
from google.cloud import bigquery_storage_v1 as bq_storage
from google.api_core import exceptions
except ImportError:
gcp_bigquery = None
bq_storage = None
HttpError = None
HttpForbiddenError = None
exceptions = None
# pylint: enable=wrong-import-order, wrong-import-position
_LOGGER = logging.getLogger(__name__)
_ELEMENTS = [
{
'name': 'beam', 'language': 'py'
},
{
'name': 'beam', 'language': 'java'
},
{
'name': 'beam', 'language': 'go'
},
{
'name': 'flink', 'language': 'java'
},
{
'name': 'flink', 'language': 'scala'
},
{
'name': 'spark', 'language': 'scala'
},
{
'name': 'spark', 'language': 'py'
},
{
'name': 'spark', 'language': 'scala'
},
{
'name': 'beam', 'foundation': 'apache'
},
{
'name': 'flink', 'foundation': 'apache'
},
{
'name': 'spark', 'foundation': 'apache'
},
]
def _load_or_default(filename):
try:
with open(filename) as f:
return json.load(f)
except: # pylint: disable=bare-except
return {}
@unittest.skipIf(
HttpError is None or gcp_bigquery is None,
'GCP dependencies are not installed')
class TestTableRowJsonCoder(unittest.TestCase):
def test_row_as_table_row(self):
schema_definition = [('s', 'STRING'), ('i', 'INTEGER'), ('f', 'FLOAT'),
('b', 'BOOLEAN'), ('n', 'NUMERIC'), ('r', 'RECORD'),
('g', 'GEOGRAPHY')]
data_definition = [
'abc',
123,
123.456,
True,
decimal.Decimal('987654321.987654321'), {
'a': 'b'
},
'LINESTRING(1 2, 3 4, 5 6, 7 8)'
]
str_def = (
'{"s": "abc", '
'"i": 123, '
'"f": 123.456, '
'"b": true, '
'"n": "987654321.987654321", '
'"r": {"a": "b"}, '
'"g": "LINESTRING(1 2, 3 4, 5 6, 7 8)"}')
schema = bigquery.TableSchema(
fields=[
bigquery.TableFieldSchema(name=k, type=v) for k,
v in schema_definition
])
coder = TableRowJsonCoder(table_schema=schema)
def value_or_decimal_to_json(val):
if isinstance(val, decimal.Decimal):
return to_json_value(str(val))
else:
return to_json_value(val)
test_row = bigquery.TableRow(
f=[
bigquery.TableCell(v=value_or_decimal_to_json(e))
for e in data_definition
])
self.assertEqual(str_def, coder.encode(test_row))
self.assertEqual(test_row, coder.decode(coder.encode(test_row)))
# A coder without schema can still decode.
self.assertEqual(
test_row, TableRowJsonCoder().decode(coder.encode(test_row)))
def test_row_and_no_schema(self):
coder = TableRowJsonCoder()
test_row = bigquery.TableRow(
f=[
bigquery.TableCell(v=to_json_value(e))
for e in ['abc', 123, 123.456, True]
])
with self.assertRaisesRegex(AttributeError,
r'^The TableRowJsonCoder requires'):
coder.encode(test_row)
def json_compliance_exception(self, value):
with self.assertRaisesRegex(ValueError, re.escape(JSON_COMPLIANCE_ERROR)):
schema_definition = [('f', 'FLOAT')]
schema = bigquery.TableSchema(
fields=[
bigquery.TableFieldSchema(name=k, type=v) for k,
v in schema_definition
])
coder = TableRowJsonCoder(table_schema=schema)
test_row = bigquery.TableRow(
f=[bigquery.TableCell(v=to_json_value(value))])
coder.encode(test_row)
def test_invalid_json_nan(self):
self.json_compliance_exception(float('nan'))
def test_invalid_json_inf(self):
self.json_compliance_exception(float('inf'))
def test_invalid_json_neg_inf(self):
self.json_compliance_exception(float('-inf'))
@unittest.skipIf(HttpError is None, 'GCP dependencies are not installed')
class TestJsonToDictCoder(unittest.TestCase):
@staticmethod
def _make_schema(fields):
def _fill_schema(fields):
for field in fields:
table_field = bigquery.TableFieldSchema()
table_field.name, table_field.type, table_field.mode, nested_fields, \
= field
if nested_fields:
table_field.fields = list(_fill_schema(nested_fields))
yield table_field
schema = bigquery.TableSchema()
schema.fields = list(_fill_schema(fields))
return schema
def test_coder_is_pickable(self):
try:
schema = self._make_schema([
(
'record',
'RECORD',
'NULLABLE', [
('float', 'FLOAT', 'NULLABLE', []),
]),
('integer', 'INTEGER', 'NULLABLE', []),
])
coder = _JsonToDictCoder(schema)
pickler.loads(pickler.dumps(coder))
except pickle.PicklingError:
self.fail('{} is not pickable'.format(coder.__class__.__name__))
def test_values_are_converted(self):
input_row = b'{"float": "10.5", "string": "abc"}'
expected_row = {'float': 10.5, 'string': 'abc'}
schema = self._make_schema([('float', 'FLOAT', 'NULLABLE', []),
('string', 'STRING', 'NULLABLE', [])])
coder = _JsonToDictCoder(schema)
actual = coder.decode(input_row)
self.assertEqual(expected_row, actual)
def test_null_fields_are_preserved(self):
input_row = b'{"float": "10.5"}'
expected_row = {'float': 10.5, 'string': None}
schema = self._make_schema([('float', 'FLOAT', 'NULLABLE', []),
('string', 'STRING', 'NULLABLE', [])])
coder = _JsonToDictCoder(schema)
actual = coder.decode(input_row)
self.assertEqual(expected_row, actual)
def test_record_field_is_properly_converted(self):
input_row = b'{"record": {"float": "55.5"}, "integer": 10}'
expected_row = {'record': {'float': 55.5}, 'integer': 10}
schema = self._make_schema([
(
'record',
'RECORD',
'NULLABLE', [
('float', 'FLOAT', 'NULLABLE', []),
]),
('integer', 'INTEGER', 'NULLABLE', []),
])
coder = _JsonToDictCoder(schema)
actual = coder.decode(input_row)
self.assertEqual(expected_row, actual)
def test_record_and_repeatable_field_is_properly_converted(self):
input_row = b'{"record": [{"float": "55.5"}, {"float": "65.5"}], ' \
b'"integer": 10}'
expected_row = {'record': [{'float': 55.5}, {'float': 65.5}], 'integer': 10}
schema = self._make_schema([
(
'record',
'RECORD',
'REPEATED', [
('float', 'FLOAT', 'NULLABLE', []),
]),
('integer', 'INTEGER', 'NULLABLE', []),
])
coder = _JsonToDictCoder(schema)
actual = coder.decode(input_row)
self.assertEqual(expected_row, actual)
def test_repeatable_field_is_properly_converted(self):
input_row = b'{"repeated": ["55.5", "65.5"], "integer": "10"}'
expected_row = {'repeated': [55.5, 65.5], 'integer': 10}
schema = self._make_schema([
('repeated', 'FLOAT', 'REPEATED', []),
('integer', 'INTEGER', 'NULLABLE', []),
])
coder = _JsonToDictCoder(schema)
actual = coder.decode(input_row)
self.assertEqual(expected_row, actual)
@unittest.skipIf(
HttpError is None or HttpForbiddenError is None,
'GCP dependencies are not installed')
class TestReadFromBigQuery(unittest.TestCase):
@classmethod
def setUpClass(cls):
class UserDefinedOptions(PipelineOptions):
@classmethod
def _add_argparse_args(cls, parser):
parser.add_value_provider_argument('--gcs_location')
cls.UserDefinedOptions = UserDefinedOptions
def tearDown(self):
# Reset runtime options to avoid side-effects caused by other tests.
RuntimeValueProvider.set_runtime_options(None)
@classmethod
def tearDownClass(cls):
# Unset the option added in setupClass to avoid interfere with other tests.
# Force a gc so PipelineOptions.__subclass__() no longer contains it.
del cls.UserDefinedOptions
gc.collect()
def test_get_destination_uri_empty_runtime_vp(self):
with self.assertRaisesRegex(ValueError,
'^ReadFromBigQuery requires a GCS '
'location to be provided'):
# Don't provide any runtime values.
RuntimeValueProvider.set_runtime_options({})
options = self.UserDefinedOptions()
bigquery_export_destination_uri(
options.gcs_location, None, uuid.uuid4().hex)
def test_get_destination_uri_none(self):
with self.assertRaisesRegex(ValueError,
'^ReadFromBigQuery requires a GCS '
'location to be provided'):
bigquery_export_destination_uri(None, None, uuid.uuid4().hex)
def test_get_destination_uri_runtime_vp(self):
# Provide values at job-execution time.
RuntimeValueProvider.set_runtime_options({'gcs_location': 'gs://bucket'})
options = self.UserDefinedOptions()
unique_id = uuid.uuid4().hex
uri = bigquery_export_destination_uri(options.gcs_location, None, unique_id)
self.assertEqual(
uri, 'gs://bucket/' + unique_id + '/bigquery-table-dump-*.json')
def test_get_destination_uri_static_vp(self):
unique_id = uuid.uuid4().hex
uri = bigquery_export_destination_uri(
StaticValueProvider(str, 'gs://bucket'), None, unique_id)
self.assertEqual(
uri, 'gs://bucket/' + unique_id + '/bigquery-table-dump-*.json')
def test_get_destination_uri_fallback_temp_location(self):
# Don't provide any runtime values.
RuntimeValueProvider.set_runtime_options({})
options = self.UserDefinedOptions()
with self.assertLogs('apache_beam.io.gcp.bigquery_read_internal',
level='DEBUG') as context:
bigquery_export_destination_uri(
options.gcs_location, 'gs://bucket', uuid.uuid4().hex)
self.assertEqual(
context.output,
[
'DEBUG:apache_beam.io.gcp.bigquery_read_internal:gcs_location is '
'empty, using temp_location instead'
])
@mock.patch.object(BigQueryWrapper, '_delete_table')
@mock.patch.object(BigQueryWrapper, '_delete_dataset')
@mock.patch('apache_beam.io.gcp.internal.clients.bigquery.BigqueryV2')
def test_temp_dataset_is_configurable(
self, api, delete_dataset, delete_table):
temp_dataset = bigquery.DatasetReference(
projectId='temp-project', datasetId='bq_dataset')
bq = BigQueryWrapper(client=api, temp_dataset_id=temp_dataset.datasetId)
gcs_location = 'gs://gcs_location'
c = beam.io.gcp.bigquery._CustomBigQuerySource(
query='select * from test_table',
gcs_location=gcs_location,
method=beam.io.ReadFromBigQuery.Method.EXPORT,
validate=True,
pipeline_options=beam.options.pipeline_options.PipelineOptions(),
job_name='job_name',
step_name='step_name',
project='execution_project',
**{'temp_dataset': temp_dataset})
c._setup_temporary_dataset(bq)
api.datasets.assert_not_called()
# User provided temporary dataset should not be deleted but the temporary
# table created by Beam should be deleted.
bq.clean_up_temporary_dataset(temp_dataset.projectId)
delete_dataset.assert_not_called()
delete_table.assert_called_with(
temp_dataset.projectId, temp_dataset.datasetId, mock.ANY)
@parameterized.expand([
param(
exception_type=exceptions.Forbidden if exceptions else None,
error_message='accessDenied'),
param(
exception_type=exceptions.ServiceUnavailable if exceptions else None,
error_message='backendError'),
])
def test_create_temp_dataset_exception(self, exception_type, error_message):
with mock.patch.object(bigquery_v2_client.BigqueryV2.JobsService,
'Insert'),\
mock.patch.object(BigQueryWrapper,
'get_or_create_dataset') as mock_insert, \
mock.patch('time.sleep'), \
self.assertRaises(Exception) as exc,\
beam.Pipeline() as p:
mock_insert.side_effect = exception_type(error_message)
_ = p | ReadFromBigQuery(
project='apache-beam-testing',
query='SELECT * FROM `project.dataset.table`',
gcs_location='gs://temp_location')
mock_insert.assert_called()
self.assertIn(error_message, exc.exception.args[0])
@parameterized.expand([
# read without exception
param(responses=[], expected_retries=0),
# first attempt returns a Http 500 blank error and retries
# second attempt returns a Http 408 blank error and retries,
# third attempt passes
param(
responses=[
HttpForbiddenError(
response={'status': 500}, content="something", url="")
if HttpForbiddenError else None,
HttpForbiddenError(
response={'status': 408}, content="blank", url="")
if HttpForbiddenError else None
],
expected_retries=2),
# first attempts returns a 403 rateLimitExceeded error
# second attempt returns a 429 blank error
# third attempt returns a Http 403 rateLimitExceeded error
# fourth attempt passes
param(
responses=[
exceptions.Forbidden(
"some message",
errors=({
"message": "transient", "reason": "rateLimitExceeded"
}, )) if exceptions else None,
exceptions.ResourceExhausted("some message")
if exceptions else None,
HttpForbiddenError(
response={'status': 403},
content={
"error": {
"errors": [{
"message": "transient",
"reason": "rateLimitExceeded"
}]
}
},
url="") if HttpForbiddenError else None,
],
expected_retries=3),
])
def test_get_table_transient_exception(self, responses, expected_retries):
class DummyTable:
class DummySchema:
fields = []
numBytes = 5
schema = DummySchema()
with mock.patch('time.sleep'), \
mock.patch.object(bigquery_v2_client.BigqueryV2.TablesService,
'Get') as mock_get_table, \
mock.patch.object(BigQueryWrapper,
'wait_for_bq_job'), \
mock.patch.object(BigQueryWrapper,
'perform_extract_job'), \
mock.patch.object(FileSystems,
'match'), \
mock.patch.object(FileSystems,
'delete'), \
beam.Pipeline() as p:
call_counter = 0
def store_callback(unused_request):
nonlocal call_counter
if call_counter < len(responses):
exception = responses[call_counter]
call_counter += 1
raise exception
else:
call_counter += 1
return DummyTable()
mock_get_table.side_effect = store_callback
_ = p | beam.io.ReadFromBigQuery(
table="project.dataset.table", gcs_location="gs://some_bucket")
# ReadFromBigQuery export mode calls get_table() twice. Once to get
# metadata (numBytes), and once to retrieve the table's schema
# Any additional calls are retries
self.assertEqual(expected_retries, mock_get_table.call_count - 2)
self.assertSetEqual(
Lineage.query(p.result.metrics(), Lineage.SOURCE),
set(["bigquery:project.dataset.table"]))
@parameterized.expand([
# first attempt returns a Http 429 with transient reason and retries
# second attempt returns a Http 403 with non-transient reason and fails
param(
responses=[
HttpForbiddenError(
response={'status': 429},
content={
"error": {
"errors": [{
"message": "transient",
"reason": "rateLimitExceeded"
}]
}
},
url="") if HttpForbiddenError else None,
HttpForbiddenError(
response={'status': 403},
content={
"error": {
"errors": [{
"message": "transient", "reason": "accessDenied"
}]
}
},
url="") if HttpForbiddenError else None
],
expected_retries=1),
# first attempt returns a transient 403 error and retries
# second attempt returns a 403 error with bad contents and fails
param(
responses=[
HttpForbiddenError(
response={'status': 403},
content={
"error": {
"errors": [{
"message": "transient",
"reason": "rateLimitExceeded"
}]
}
},
url="") if HttpForbiddenError else None,
HttpError(
response={'status': 403}, content="bad contents", url="")
if HttpError else None
],
expected_retries=1),
# first attempt returns a transient 403 error and retries
# second attempt returns a 429 error and retries
# third attempt returns a 403 with non-transient reason and fails
param(
responses=[
exceptions.Forbidden(
"some error",
errors=({
"message": "transient", "reason": "rateLimitExceeded"
}, )) if exceptions else None,
exceptions.ResourceExhausted("some transient error")
if exceptions else None,
exceptions.Forbidden(
"some error",
errors=({
"message": "transient", "reason": "accessDenied"
}, )) if exceptions else None,
],
expected_retries=2),
])
def test_get_table_non_transient_exception(self, responses, expected_retries):
class DummyTable:
class DummySchema:
fields = []
numBytes = 5
schema = DummySchema()
with mock.patch('time.sleep'), \
mock.patch.object(bigquery_v2_client.BigqueryV2.TablesService,
'Get') as mock_get_table, \
mock.patch.object(BigQueryWrapper,
'wait_for_bq_job'), \
mock.patch.object(BigQueryWrapper,
'perform_extract_job'), \
mock.patch.object(FileSystems,
'match'), \
mock.patch.object(FileSystems,
'delete'), \
self.assertRaises(Exception), \
beam.Pipeline() as p:
call_counter = 0
def store_callback(unused_request):
nonlocal call_counter
if call_counter < len(responses):
exception = responses[call_counter]
call_counter += 1
raise exception
else:
call_counter += 1
return DummyTable()
mock_get_table.side_effect = store_callback
_ = p | beam.io.ReadFromBigQuery(
table="project.dataset.table", gcs_location="gs://some_bucket")
# ReadFromBigQuery export mode calls get_table() twice. Once to get
# metadata (numBytes), and once to retrieve the table's schema
# However, the second call is never reached because this test will always
# fail before it does so
# After the first call, any additional calls are retries
self.assertEqual(expected_retries, mock_get_table.call_count - 1)
@parameterized.expand([
param(
exception_type=exceptions.BadRequest if exceptions else None,
error_message='invalidQuery'),
param(
exception_type=exceptions.NotFound if exceptions else None,
error_message='notFound'),
param(
exception_type=exceptions.Forbidden if exceptions else None,
error_message='responseTooLarge')
])
def test_query_job_exception(self, exception_type, error_message):
with mock.patch.object(beam.io.gcp.bigquery._CustomBigQuerySource,
'estimate_size') as mock_estimate,\
mock.patch.object(BigQueryWrapper,
'get_query_location') as mock_query_location,\
mock.patch.object(bigquery_v2_client.BigqueryV2.JobsService,
'Insert') as mock_query_job,\
mock.patch.object(bigquery_v2_client.BigqueryV2.DatasetsService, 'Get'), \
mock.patch('time.sleep'), \
self.assertRaises(Exception) as exc, \
beam.Pipeline() as p:
mock_estimate.return_value = None
mock_query_location.return_value = None
mock_query_job.side_effect = exception_type(error_message)
_ = p | ReadFromBigQuery(
query='SELECT * FROM `project.dataset.table`',
gcs_location='gs://temp_location')
mock_query_job.assert_called()
self.assertIn(error_message, exc.exception.args[0])
@parameterized.expand([
param(
exception_type=exceptions.BadRequest if exceptions else None,
error_message='invalid'),
param(
exception_type=exceptions.Forbidden if exceptions else None,
error_message='accessDenied')
])
def test_read_export_exception(self, exception_type, error_message):
with mock.patch.object(beam.io.gcp.bigquery._CustomBigQuerySource,
'estimate_size') as mock_estimate,\
mock.patch.object(bigquery_v2_client.BigqueryV2.TablesService, 'Get'),\
mock.patch.object(bigquery_v2_client.BigqueryV2.JobsService,
'Insert') as mock_query_job, \
mock.patch('time.sleep'), \
self.assertRaises(Exception) as exc,\
beam.Pipeline() as p:
mock_estimate.return_value = None
mock_query_job.side_effect = exception_type(error_message)
_ = p | ReadFromBigQuery(
project='apache-beam-testing',
method=ReadFromBigQuery.Method.EXPORT,
table='project:dataset.table',
gcs_location="gs://temp_location")
mock_query_job.assert_called()
self.assertIn(error_message, exc.exception.args[0])
def test_read_direct_lineage(self):
with mock.patch.object(bigquery_tools.BigQueryWrapper,
'_bigquery_client'),\
mock.patch.object(bq_storage.BigQueryReadClient,
'create_read_session'),\
beam.Pipeline() as p:
_ = p | ReadFromBigQuery(
method=ReadFromBigQuery.Method.DIRECT_READ,
table='project:dataset.table')
self.assertSetEqual(
Lineage.query(p.result.metrics(), Lineage.SOURCE),
set(["bigquery:project.dataset.table"]))
def test_read_all_lineage(self):
with mock.patch.object(_BigQueryReadSplit, '_export_files') as export, \
beam.Pipeline() as p:
export.return_value = (None, [])
_ = (
p
| beam.Create([
beam.io.ReadFromBigQueryRequest(table='project1:dataset1.table1'),
beam.io.ReadFromBigQueryRequest(table='project2:dataset2.table2')
])
| beam.io.ReadAllFromBigQuery(gcs_location='gs://bucket/tmp'))
self.assertSetEqual(
Lineage.query(p.result.metrics(), Lineage.SOURCE),
set([
'bigquery:project1.dataset1.table1',
'bigquery:project2.dataset2.table2'
]))
@unittest.skipIf(HttpError is None, 'GCP dependencies are not installed')
class TestBigQuerySink(unittest.TestCase):
def test_table_spec_display_data(self):
sink = beam.io.BigQuerySink('dataset.table')
dd = DisplayData.create_from(sink)
expected_items = [
DisplayDataItemMatcher('table', 'dataset.table'),
DisplayDataItemMatcher('validation', False)
]
hc.assert_that(dd.items, hc.contains_inanyorder(*expected_items))
def test_parse_schema_descriptor(self):
sink = beam.io.BigQuerySink('dataset.table', schema='s:STRING, n:INTEGER')
self.assertEqual(sink.table_reference.datasetId, 'dataset')
self.assertEqual(sink.table_reference.tableId, 'table')
result_schema = {
field['name']: field['type']
for field in sink.schema['fields']
}
self.assertEqual({'n': 'INTEGER', 's': 'STRING'}, result_schema)
def test_project_table_display_data(self):
sinkq = beam.io.BigQuerySink('project:dataset.table')
dd = DisplayData.create_from(sinkq)
expected_items = [
DisplayDataItemMatcher('table', 'project:dataset.table'),
DisplayDataItemMatcher('validation', False)
]
hc.assert_that(dd.items, hc.contains_inanyorder(*expected_items))
@unittest.skipIf(HttpError is None, 'GCP dependencies are not installed')
class TestWriteToBigQuery(unittest.TestCase):
def _cleanup_files(self):
if os.path.exists('insert_calls1'):
os.remove('insert_calls1')
if os.path.exists('insert_calls2'):
os.remove('insert_calls2')
def setUp(self):
self._cleanup_files()
def tearDown(self):
self._cleanup_files()
def test_noop_schema_parsing(self):
expected_table_schema = None
table_schema = beam.io.gcp.bigquery.BigQueryWriteFn.get_table_schema(
schema=None)
self.assertEqual(expected_table_schema, table_schema)
def test_dict_schema_parsing(self):
schema = {
'fields': [{
'name': 's', 'type': 'STRING', 'mode': 'NULLABLE'
}, {
'name': 'n', 'type': 'INTEGER', 'mode': 'NULLABLE'
},
{
'name': 'r',
'type': 'RECORD',
'mode': 'NULLABLE',
'fields': [{
'name': 'x', 'type': 'INTEGER', 'mode': 'NULLABLE'
}]
}]
}
table_schema = beam.io.gcp.bigquery.BigQueryWriteFn.get_table_schema(schema)
string_field = bigquery.TableFieldSchema(
name='s', type='STRING', mode='NULLABLE')
nested_field = bigquery.TableFieldSchema(
name='x', type='INTEGER', mode='NULLABLE')
number_field = bigquery.TableFieldSchema(
name='n', type='INTEGER', mode='NULLABLE')
record_field = bigquery.TableFieldSchema(
name='r', type='RECORD', mode='NULLABLE', fields=[nested_field])
expected_table_schema = bigquery.TableSchema(
fields=[string_field, number_field, record_field])
self.assertEqual(expected_table_schema, table_schema)
def test_string_schema_parsing(self):
schema = 's:STRING, n:INTEGER'
expected_dict_schema = {
'fields': [{
'name': 's', 'type': 'STRING', 'mode': 'NULLABLE'
}, {
'name': 'n', 'type': 'INTEGER', 'mode': 'NULLABLE'
}]
}
dict_schema = (
beam.io.gcp.bigquery.WriteToBigQuery.get_dict_table_schema(schema))
self.assertEqual(expected_dict_schema, dict_schema)
def test_table_schema_parsing(self):
string_field = bigquery.TableFieldSchema(
name='s', type='STRING', mode='NULLABLE')
nested_field = bigquery.TableFieldSchema(
name='x', type='INTEGER', mode='NULLABLE')
number_field = bigquery.TableFieldSchema(
name='n', type='INTEGER', mode='NULLABLE')
record_field = bigquery.TableFieldSchema(
name='r', type='RECORD', mode='NULLABLE', fields=[nested_field])
schema = bigquery.TableSchema(
fields=[string_field, number_field, record_field])
expected_dict_schema = {
'fields': [{
'name': 's', 'type': 'STRING', 'mode': 'NULLABLE'
}, {
'name': 'n', 'type': 'INTEGER', 'mode': 'NULLABLE'
},
{
'name': 'r',
'type': 'RECORD',
'mode': 'NULLABLE',
'fields': [{
'name': 'x', 'type': 'INTEGER', 'mode': 'NULLABLE'
}]
}]
}
dict_schema = (
beam.io.gcp.bigquery.WriteToBigQuery.get_dict_table_schema(schema))
self.assertEqual(expected_dict_schema, dict_schema)
def test_table_schema_parsing_end_to_end(self):
string_field = bigquery.TableFieldSchema(
name='s', type='STRING', mode='NULLABLE')
nested_field = bigquery.TableFieldSchema(
name='x', type='INTEGER', mode='NULLABLE')
number_field = bigquery.TableFieldSchema(
name='n', type='INTEGER', mode='NULLABLE')
record_field = bigquery.TableFieldSchema(
name='r', type='RECORD', mode='NULLABLE', fields=[nested_field])
schema = bigquery.TableSchema(
fields=[string_field, number_field, record_field])
table_schema = beam.io.gcp.bigquery.BigQueryWriteFn.get_table_schema(
beam.io.gcp.bigquery.WriteToBigQuery.get_dict_table_schema(schema))
self.assertEqual(table_schema, schema)
def test_none_schema_parsing(self):
schema = None
expected_dict_schema = None
dict_schema = (
beam.io.gcp.bigquery.WriteToBigQuery.get_dict_table_schema(schema))
self.assertEqual(expected_dict_schema, dict_schema)
def test_noop_dict_schema_parsing(self):
schema = {
'fields': [{
'name': 's', 'type': 'STRING', 'mode': 'NULLABLE'
}, {
'name': 'n', 'type': 'INTEGER', 'mode': 'NULLABLE'
}]
}
expected_dict_schema = schema
dict_schema = (
beam.io.gcp.bigquery.WriteToBigQuery.get_dict_table_schema(schema))
self.assertEqual(expected_dict_schema, dict_schema)
def test_schema_autodetect_not_allowed_with_avro_file_loads(self):
with TestPipeline() as p:
pc = p | beam.Impulse()
with self.assertRaisesRegex(ValueError, '^A schema must be provided'):
_ = (
pc
| 'No Schema' >> beam.io.gcp.bigquery.WriteToBigQuery(
"dataset.table",
schema=None,
temp_file_format=bigquery_tools.FileFormat.AVRO))
with self.assertRaisesRegex(ValueError,
'^Schema auto-detection is not supported'):
_ = (
pc
| 'Schema Autodetected' >> beam.io.gcp.bigquery.WriteToBigQuery(
"dataset.table",
schema=beam.io.gcp.bigquery.SCHEMA_AUTODETECT,
temp_file_format=bigquery_tools.FileFormat.AVRO))
def test_to_from_runner_api(self):
"""Tests that serialization of WriteToBigQuery is correct.
This is not intended to be a change-detector test. As such, this only tests
the more complicated serialization logic of parameters: ValueProviders,
callables, and side inputs.
"""
FULL_OUTPUT_TABLE = 'test_project:output_table'
p = TestPipeline()
# Used for testing side input parameters.
table_record_pcv = beam.pvalue.AsDict(
p | "MakeTable" >> beam.Create([('table', FULL_OUTPUT_TABLE)]))
# Used for testing value provider parameters.
schema = value_provider.StaticValueProvider(str, '"a:str"')
original = WriteToBigQuery(
table=lambda _,
side_input: side_input['table'],
table_side_inputs=(table_record_pcv, ),
schema=schema)
# pylint: disable=expression-not-assigned
p | 'MyWriteToBigQuery' >> original
# Run the pipeline through to generate a pipeline proto from an empty
# context. This ensures that the serialization code ran.
pipeline_proto, context = TestPipeline.from_runner_api(
p.to_runner_api(), p.runner, p.get_pipeline_options()).to_runner_api(
return_context=True)
# Find the transform from the context.
write_to_bq_id = [
k for k,
v in pipeline_proto.components.transforms.items()
if v.unique_name == 'MyWriteToBigQuery'
][0]
deserialized_node = context.transforms.get_by_id(write_to_bq_id)
deserialized = deserialized_node.transform
self.assertIsInstance(deserialized, WriteToBigQuery)
# Test that the serialization of a value provider is correct.
self.assertEqual(original.schema, deserialized.schema)
# Test that the serialization of a callable is correct.
self.assertEqual(
deserialized._table(None, {'table': FULL_OUTPUT_TABLE}),
FULL_OUTPUT_TABLE)
# Test that the serialization of a side input is correct.
self.assertEqual(
len(original.table_side_inputs), len(deserialized.table_side_inputs))
original_side_input_data = original.table_side_inputs[0]._side_input_data()