-
Notifications
You must be signed in to change notification settings - Fork 4.3k
/
bigquery_tools.py
1875 lines (1641 loc) · 64.3 KB
/
bigquery_tools.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.
#
"""Tools used by BigQuery sources and sinks.
Classes, constants and functions in this file are experimental and have no
backwards compatibility guarantees.
These tools include wrappers and clients to interact with BigQuery APIs.
NOTHING IN THIS FILE HAS BACKWARDS COMPATIBILITY GUARANTEES.
"""
# pytype: skip-file
import datetime
import decimal
import io
import json
import logging
import re
import sys
import time
import uuid
from json.decoder import JSONDecodeError
from typing import Optional
from typing import Sequence
from typing import Tuple
from typing import TypeVar
from typing import Union
import fastavro
import numpy as np
import regex
import apache_beam
from apache_beam import coders
from apache_beam.internal.gcp import auth
from apache_beam.internal.gcp.json_value import from_json_value
from apache_beam.internal.http_client import get_new_http
from apache_beam.internal.metrics.metric import MetricLogger
from apache_beam.internal.metrics.metric import Metrics
from apache_beam.internal.metrics.metric import ServiceCallMetric
from apache_beam.io.gcp import bigquery_avro_tools
from apache_beam.io.gcp import resource_identifiers
from apache_beam.io.gcp.internal.clients import bigquery
from apache_beam.metrics import monitoring_infos
from apache_beam.options import value_provider
from apache_beam.options.pipeline_options import PipelineOptions
from apache_beam.transforms import DoFn
from apache_beam.typehints.row_type import RowTypeConstraint
from apache_beam.typehints.typehints import Any
from apache_beam.utils import retry
from apache_beam.utils.histogram import LinearBucket
# Protect against environments where bigquery library is not available.
# pylint: disable=wrong-import-order, wrong-import-position
try:
from apitools.base.py.transfer import Upload
from apitools.base.py.exceptions import HttpError, HttpForbiddenError
from google.api_core.exceptions import ClientError, GoogleAPICallError
from google.api_core.client_info import ClientInfo
from google.cloud import bigquery as gcp_bigquery
except ImportError:
gcp_bigquery = None
pass
try:
from orjson import dumps as fast_json_dumps
from orjson import loads as fast_json_loads
except ImportError:
fast_json_dumps = json.dumps
fast_json_loads = json.loads
# pylint: enable=wrong-import-order, wrong-import-position
# pylint: disable=wrong-import-order, wrong-import-position, ungrouped-imports
try:
from apache_beam.io.gcp.internal.clients.bigquery import TableReference
except ImportError:
TableReference = None
# pylint: enable=wrong-import-order, wrong-import-position, ungrouped-imports
_LOGGER = logging.getLogger(__name__)
JSON_COMPLIANCE_ERROR = 'NAN, INF and -INF values are not JSON compliant.'
MAX_RETRIES = 3
UNKNOWN_MIME_TYPE = 'application/octet-stream'
# Timeout for a BQ streaming insert RPC. Set to a maximum of 2 minutes.
BQ_STREAMING_INSERT_TIMEOUT_SEC = 120
_PROJECT_PATTERN = r'([a-z0-9.-]+:)?[a-z][a-z0-9-]*[a-z0-9]'
_DATASET_PATTERN = r'\w{1,1024}'
_TABLE_PATTERN = r'[\p{L}\p{M}\p{N}\p{Pc}\p{Pd}\p{Zs}$]{1,1024}'
# TODO(https://github.com/apache/beam/issues/25946): Add support for
# more Beam portable schema types as Python types
BIGQUERY_TYPE_TO_PYTHON_TYPE = {
"STRING": str,
"BOOL": bool,
"BOOLEAN": bool,
"BYTES": bytes,
"INT64": np.int64,
"INTEGER": np.int64,
"FLOAT64": np.float64,
"FLOAT": np.float64,
"NUMERIC": decimal.Decimal,
"TIMESTAMP": apache_beam.utils.timestamp.Timestamp,
}
class FileFormat(object):
CSV = 'CSV'
JSON = 'NEWLINE_DELIMITED_JSON'
AVRO = 'AVRO'
class ExportCompression(object):
GZIP = 'GZIP'
DEFLATE = 'DEFLATE'
SNAPPY = 'SNAPPY'
NONE = 'NONE'
def default_encoder(obj):
if isinstance(obj, decimal.Decimal):
return str(obj)
elif isinstance(obj, bytes):
# on python 3 base64-encoded bytes are decoded to strings
# before being sent to BigQuery
return obj.decode('utf-8')
elif isinstance(obj, (datetime.date, datetime.time)):
return str(obj)
elif isinstance(obj, datetime.datetime):
return obj.isoformat()
_LOGGER.error("Unable to serialize %r to JSON", obj)
raise TypeError(
"Object of type '%s' is not JSON serializable" % type(obj).__name__)
def get_hashable_destination(destination):
"""Parses a table reference into a (project, dataset, table) tuple.
Args:
destination: Either a TableReference object from the bigquery API.
The object has the following attributes: projectId, datasetId, and
tableId. Or a string representing the destination containing
'PROJECT:DATASET.TABLE'.
Returns:
A string representing the destination containing
'PROJECT:DATASET.TABLE'.
"""
if isinstance(destination, TableReference):
return '%s:%s.%s' % (
destination.projectId, destination.datasetId, destination.tableId)
else:
return destination
V = TypeVar('V')
def to_hashable_table_ref(
table_ref_elem_kv: Tuple[Union[str, TableReference], V]) -> Tuple[str, V]:
"""Turns the key of the input tuple to its string representation. The key
should be either a string or a TableReference.
Args:
table_ref_elem_kv: A tuple of table reference and element.
Returns:
A tuple of string representation of input table and input element.
"""
table_ref = table_ref_elem_kv[0]
hashable_table_ref = get_hashable_destination(table_ref)
return (hashable_table_ref, table_ref_elem_kv[1])
def parse_table_schema_from_json(schema_string):
"""Parse the Table Schema provided as string.
Args:
schema_string: String serialized table schema, should be a valid JSON.
Returns:
A TableSchema of the BigQuery export from either the Query or the Table.
"""
try:
json_schema = json.loads(schema_string)
except JSONDecodeError as e:
raise ValueError(
'Unable to parse JSON schema: %s - %r' % (schema_string, e))
def _parse_schema_field(field):
"""Parse a single schema field from dictionary.
Args:
field: Dictionary object containing serialized schema.
Returns:
A TableFieldSchema for a single column in BigQuery.
"""
schema = bigquery.TableFieldSchema()
schema.name = field['name']
schema.type = field['type']
if 'mode' in field:
schema.mode = field['mode']
else:
schema.mode = 'NULLABLE'
if 'description' in field:
schema.description = field['description']
if 'fields' in field:
schema.fields = [_parse_schema_field(x) for x in field['fields']]
return schema
fields = [_parse_schema_field(f) for f in json_schema['fields']]
return bigquery.TableSchema(fields=fields)
def parse_table_reference(table, dataset=None, project=None):
"""Parses a table reference into a (project, dataset, table) tuple.
Args:
table: The ID of the table. The ID must contain only letters
(a-z, A-Z), numbers (0-9), connectors (-_). If dataset argument is None
then the table argument must contain the entire table reference:
'DATASET.TABLE' or 'PROJECT:DATASET.TABLE'. This argument can be a
TableReference instance in which case dataset and project are
ignored and the reference is returned as a result. Additionally, for date
partitioned tables, appending '$YYYYmmdd' to the table name is supported,
e.g. 'DATASET.TABLE$YYYYmmdd'.
dataset: The ID of the dataset containing this table or null if the table
reference is specified entirely by the table argument.
project: The ID of the project containing this table or null if the table
reference is specified entirely by the table (and possibly dataset)
argument.
Returns:
A TableReference object from the bigquery API. The object has the following
attributes: projectId, datasetId, and tableId.
If the input is a TableReference object, a new object will be returned.
Raises:
ValueError: if the table reference as a string does not match the expected
format.
"""
if isinstance(table, TableReference):
return TableReference(
projectId=table.projectId,
datasetId=table.datasetId,
tableId=table.tableId)
elif callable(table):
return table
elif isinstance(table, value_provider.ValueProvider):
return table
table_reference = TableReference()
# If dataset argument is not specified, the expectation is that the
# table argument will contain a full table reference instead of just a
# table name.
if dataset is None:
pattern = (
f'((?P<project>{_PROJECT_PATTERN})[:\\.])?'
f'(?P<dataset>{_DATASET_PATTERN})\\.(?P<table>{_TABLE_PATTERN})')
match = regex.fullmatch(pattern, table)
if not match:
raise ValueError(
'Expected a table reference (PROJECT:DATASET.TABLE or '
'DATASET.TABLE) instead of %s.' % table)
table_reference.projectId = match.group('project')
table_reference.datasetId = match.group('dataset')
table_reference.tableId = match.group('table')
else:
table_reference.projectId = project
table_reference.datasetId = dataset
table_reference.tableId = table
return table_reference
# -----------------------------------------------------------------------------
# BigQueryWrapper.
def _build_job_labels(input_labels):
"""Builds job label protobuf structure."""
input_labels = input_labels or {}
result = bigquery.JobConfiguration.LabelsValue()
for k, v in input_labels.items():
result.additionalProperties.append(
bigquery.JobConfiguration.LabelsValue.AdditionalProperty(
key=k,
value=v,
))
return result
def _build_dataset_labels(input_labels):
"""Builds dataset label protobuf structure."""
input_labels = input_labels or {}
result = bigquery.Dataset.LabelsValue()
for k, v in input_labels.items():
result.additionalProperties.append(
bigquery.Dataset.LabelsValue.AdditionalProperty(
key=k,
value=v,
))
return result
def _build_filter_from_labels(labels):
filter_str = ''
for key, value in labels.items():
filter_str += 'labels.' + key + ':' + value + ' '
return filter_str
class BigQueryWrapper(object):
"""BigQuery client wrapper with utilities for querying.
The wrapper is used to organize all the BigQuery integration points and
offer a common place where retry logic for failures can be controlled.
In addition, it offers various functions used both in sources and sinks
(e.g., find and create tables, query a table, etc.).
Note that client parameter in constructor is only for testing purposes and
should not be used in production code.
"""
# If updating following names, also update the corresponding pydocs in
# bigquery.py.
TEMP_TABLE = 'beam_temp_table_'
TEMP_DATASET = 'beam_temp_dataset_'
HISTOGRAM_METRIC_LOGGER = MetricLogger()
def __init__(self, client=None, temp_dataset_id=None, temp_table_ref=None):
self.client = client or BigQueryWrapper._bigquery_client(PipelineOptions())
self.gcp_bq_client = client or gcp_bigquery.Client(
client_info=ClientInfo(
user_agent="apache-beam-%s" % apache_beam.__version__))
self._unique_row_id = 0
# For testing scenarios where we pass in a client we do not want a
# randomized prefix for row IDs.
self._row_id_prefix = '' if client else uuid.uuid4()
self._latency_histogram_metric = Metrics.histogram(
self.__class__,
'latency_histogram_ms',
LinearBucket(0, 20, 3000),
BigQueryWrapper.HISTOGRAM_METRIC_LOGGER)
if temp_dataset_id is not None and temp_table_ref is not None:
raise ValueError(
'Both a BigQuery temp_dataset_id and a temp_table_ref were specified.'
' Please specify only one of these.')
if temp_dataset_id and temp_dataset_id.startswith(self.TEMP_DATASET):
raise ValueError(
'User provided temp dataset ID cannot start with %r' %
self.TEMP_DATASET)
if temp_table_ref is not None:
self.temp_table_ref = temp_table_ref
self.temp_dataset_id = temp_table_ref.datasetId
else:
self.temp_table_ref = None
self._temporary_table_suffix = uuid.uuid4().hex
self.temp_dataset_id = temp_dataset_id or self._get_temp_dataset()
self.created_temp_dataset = False
@property
def unique_row_id(self):
"""Returns a unique row ID (str) used to avoid multiple insertions.
If the row ID is provided, BigQuery will make a best effort to not insert
the same row multiple times for fail and retry scenarios in which the insert
request may be issued several times. This comes into play for sinks executed
in a local runner.
Returns:
a unique row ID string
"""
self._unique_row_id += 1
return '%s_%d' % (self._row_id_prefix, self._unique_row_id)
def _get_temp_table(self, project_id):
if self.temp_table_ref:
return self.temp_table_ref
return parse_table_reference(
table=BigQueryWrapper.TEMP_TABLE + self._temporary_table_suffix,
dataset=self.temp_dataset_id,
project=project_id)
def _get_temp_dataset(self):
if self.temp_table_ref:
return self.temp_table_ref.datasetId
return BigQueryWrapper.TEMP_DATASET + self._temporary_table_suffix
@retry.with_exponential_backoff(
num_retries=MAX_RETRIES,
retry_filter=retry.retry_on_server_errors_and_timeout_filter)
def get_query_location(self, project_id, query, use_legacy_sql):
"""
Get the location of tables referenced in a query.
This method returns the location of the first available referenced
table for user in the query and depends on the BigQuery service to
provide error handling for queries that reference tables in multiple
locations.
"""
reference = bigquery.JobReference(
jobId=uuid.uuid4().hex, projectId=project_id)
request = bigquery.BigqueryJobsInsertRequest(
projectId=project_id,
job=bigquery.Job(
configuration=bigquery.JobConfiguration(
dryRun=True,
query=bigquery.JobConfigurationQuery(
query=query,
useLegacySql=use_legacy_sql,
)),
jobReference=reference))
response = self.client.jobs.Insert(request)
if response.statistics is None:
# This behavior is only expected in tests
_LOGGER.warning(
"Unable to get location, missing response.statistics. Query: %s",
query)
return None
referenced_tables = response.statistics.query.referencedTables
if referenced_tables: # Guards against both non-empty and non-None
for table in referenced_tables:
try:
location = self.get_table_location(
table.projectId, table.datasetId, table.tableId)
except HttpForbiddenError:
# Permission access for table (i.e. from authorized_view),
# try next one
continue
_LOGGER.info(
"Using location %r from table %r referenced by query %s",
location,
table,
query)
return location
_LOGGER.debug(
"Query %s does not reference any tables or "
"you don't have permission to inspect them.",
query)
return None
@retry.with_exponential_backoff(
num_retries=MAX_RETRIES,
retry_filter=retry.retry_on_server_errors_and_timeout_filter)
def _insert_copy_job(
self,
project_id,
job_id,
from_table_reference,
to_table_reference,
create_disposition=None,
write_disposition=None,
job_labels=None):
reference = bigquery.JobReference()
reference.jobId = job_id
reference.projectId = project_id
request = bigquery.BigqueryJobsInsertRequest(
projectId=project_id,
job=bigquery.Job(
configuration=bigquery.JobConfiguration(
copy=bigquery.JobConfigurationTableCopy(
destinationTable=to_table_reference,
sourceTable=from_table_reference,
createDisposition=create_disposition,
writeDisposition=write_disposition,
),
labels=_build_job_labels(job_labels),
),
jobReference=reference,
))
return self._start_job(request).jobReference
@retry.with_exponential_backoff(
num_retries=MAX_RETRIES,
retry_filter=retry.retry_on_server_errors_and_timeout_filter)
def _insert_load_job(
self,
project_id,
job_id,
table_reference,
source_uris=None,
source_stream=None,
schema=None,
write_disposition=None,
create_disposition=None,
additional_load_parameters=None,
source_format=None,
job_labels=None):
if not source_uris and not source_stream:
_LOGGER.warning(
'Both source URIs and source stream are not provided. BigQuery load '
'job will not load any data.')
if source_uris and source_stream:
raise ValueError(
'Only one of source_uris and source_stream may be specified. '
'Got both.')
if source_uris is None:
source_uris = []
additional_load_parameters = additional_load_parameters or {}
job_schema = None if schema == 'SCHEMA_AUTODETECT' else schema
reference = bigquery.JobReference(jobId=job_id, projectId=project_id)
request = bigquery.BigqueryJobsInsertRequest(
projectId=project_id,
job=bigquery.Job(
configuration=bigquery.JobConfiguration(
load=bigquery.JobConfigurationLoad(
sourceUris=source_uris,
destinationTable=table_reference,
schema=job_schema,
writeDisposition=write_disposition,
createDisposition=create_disposition,
sourceFormat=source_format,
useAvroLogicalTypes=True,
autodetect=schema == 'SCHEMA_AUTODETECT',
**additional_load_parameters),
labels=_build_job_labels(job_labels),
),
jobReference=reference,
))
return self._start_job(request, stream=source_stream).jobReference
@staticmethod
def _parse_location_from_exc(content, job_id):
"""Parse job location from Exception content."""
if isinstance(content, bytes):
content = content.decode('ascii', 'replace')
# search for "Already Exists: Job <project-id>:<location>.<job id>"
m = re.search(r"Already Exists: Job \S+\:(\S+)\." + job_id, content)
if not m:
_LOGGER.warning(
"Not able to parse BigQuery load job location for %s", job_id)
return None
return m.group(1)
def _start_job(
self,
request: 'bigquery.BigqueryJobsInsertRequest',
stream=None,
):
"""Inserts a BigQuery job.
If the job exists already, it returns it.
Args:
request (bigquery.BigqueryJobsInsertRequest): An insert job request.
stream (IO[bytes]): A bytes IO object open for reading.
"""
try:
upload = None
if stream:
upload = Upload.FromStream(stream, mime_type=UNKNOWN_MIME_TYPE)
response = self.client.jobs.Insert(request, upload=upload)
_LOGGER.info(
"Started BigQuery job: %s\n "
"bq show -j --format=prettyjson --project_id=%s %s",
response.jobReference,
response.jobReference.projectId,
response.jobReference.jobId)
return response
except HttpError as exn:
if exn.status_code == 409:
jobId = request.job.jobReference.jobId
_LOGGER.info(
"BigQuery job %s already exists, will not retry inserting it: %s",
request.job.jobReference,
exn)
job_location = self._parse_location_from_exc(exn.content, jobId)
response = request.job
if not response.jobReference.location and job_location:
# Request not constructed with location
response.jobReference.location = job_location
return response
else:
_LOGGER.info(
"Failed to insert job %s: %s", request.job.jobReference, exn)
raise
@retry.with_exponential_backoff(
num_retries=MAX_RETRIES,
retry_filter=retry.retry_on_server_errors_and_timeout_filter)
def _start_query_job(
self,
project_id,
query,
use_legacy_sql,
flatten_results,
job_id,
priority,
dry_run=False,
kms_key=None,
job_labels=None):
reference = bigquery.JobReference(jobId=job_id, projectId=project_id)
request = bigquery.BigqueryJobsInsertRequest(
projectId=project_id,
job=bigquery.Job(
configuration=bigquery.JobConfiguration(
dryRun=dry_run,
query=bigquery.JobConfigurationQuery(
query=query,
useLegacySql=use_legacy_sql,
allowLargeResults=not dry_run,
destinationTable=self._get_temp_table(project_id)
if not dry_run else None,
flattenResults=flatten_results,
priority=priority,
destinationEncryptionConfiguration=bigquery.
EncryptionConfiguration(kmsKeyName=kms_key)),
labels=_build_job_labels(job_labels),
),
jobReference=reference))
return self._start_job(request)
def wait_for_bq_job(self, job_reference, sleep_duration_sec=5, max_retries=0):
"""Poll job until it is DONE.
Args:
job_reference: bigquery.JobReference instance.
sleep_duration_sec: Specifies the delay in seconds between retries.
max_retries: The total number of times to retry. If equals to 0,
the function waits forever.
Raises:
`RuntimeError`: If the job is FAILED or the number of retries has been
reached.
"""
retry = 0
while True:
retry += 1
job = self.get_job(
job_reference.projectId, job_reference.jobId, job_reference.location)
_LOGGER.info('Job %s status: %s', job.id, job.status.state)
if job.status.state == 'DONE' and job.status.errorResult:
raise RuntimeError(
'BigQuery job {} failed. Error Result: {}'.format(
job_reference.jobId, job.status.errorResult))
elif job.status.state == 'DONE':
return True
else:
time.sleep(sleep_duration_sec)
if max_retries != 0 and retry >= max_retries:
raise RuntimeError('The maximum number of retries has been reached')
@retry.with_exponential_backoff(
num_retries=MAX_RETRIES,
retry_filter=retry.retry_on_server_errors_and_timeout_filter)
def _get_query_results(
self,
project_id,
job_id,
page_token=None,
max_results=10000,
location=None):
request = bigquery.BigqueryJobsGetQueryResultsRequest(
jobId=job_id,
pageToken=page_token,
projectId=project_id,
maxResults=max_results,
location=location)
response = self.client.jobs.GetQueryResults(request)
return response
@retry.with_exponential_backoff(
num_retries=MAX_RETRIES,
retry_filter=retry.retry_on_server_errors_timeout_or_quota_issues_filter)
def _insert_all_rows(
self,
project_id,
dataset_id,
table_id,
rows,
insert_ids,
skip_invalid_rows=False,
ignore_unknown_values=False):
"""Calls the insertAll BigQuery API endpoint.
Docs for this BQ call: https://cloud.google.com/bigquery/docs/reference\
/rest/v2/tabledata/insertAll."""
# The rows argument is a list of
# bigquery.TableDataInsertAllRequest.RowsValueListEntry instances as
# required by the InsertAll() method.
resource = resource_identifiers.BigQueryTable(
project_id, dataset_id, table_id)
labels = {
# TODO(ajamato): Add Ptransform label.
monitoring_infos.SERVICE_LABEL: 'BigQuery',
# Refer to any method which writes elements to BigQuery in batches
# as "BigQueryBatchWrite". I.e. storage API's insertAll, or future
# APIs introduced.
monitoring_infos.METHOD_LABEL: 'BigQueryBatchWrite',
monitoring_infos.RESOURCE_LABEL: resource,
monitoring_infos.BIGQUERY_PROJECT_ID_LABEL: project_id,
monitoring_infos.BIGQUERY_DATASET_LABEL: dataset_id,
monitoring_infos.BIGQUERY_TABLE_LABEL: table_id,
}
service_call_metric = ServiceCallMetric(
request_count_urn=monitoring_infos.API_REQUEST_COUNT_URN,
base_labels=labels)
started_millis = int(time.time() * 1000)
try:
table_ref_str = '%s.%s.%s' % (project_id, dataset_id, table_id)
errors = self.gcp_bq_client.insert_rows_json(
table_ref_str,
json_rows=rows,
row_ids=insert_ids,
skip_invalid_rows=skip_invalid_rows,
ignore_unknown_values=ignore_unknown_values,
timeout=BQ_STREAMING_INSERT_TIMEOUT_SEC)
if not errors:
service_call_metric.call('ok')
else:
for insert_error in errors:
service_call_metric.call(insert_error['errors'][0])
except (ClientError, GoogleAPICallError) as e:
# e.code contains the numeric http status code.
service_call_metric.call(e.code)
# Package exception with required fields
error = {'message': e.message, 'reason': e.response.reason}
# Add all rows to the errors list along with the error
errors = [{"index": i, "errors": [error]} for i, _ in enumerate(rows)]
except HttpError as e:
service_call_metric.call(e)
# Re-raise the exception so that we re-try appropriately.
raise
finally:
self._latency_histogram_metric.update(
int(time.time() * 1000) - started_millis)
return not errors, errors
@retry.with_exponential_backoff(
num_retries=MAX_RETRIES,
retry_filter=retry.retry_on_server_errors_timeout_or_quota_issues_filter)
def get_table(self, project_id, dataset_id, table_id):
"""Lookup a table's metadata object.
Args:
client: bigquery.BigqueryV2 instance
project_id: table lookup parameter
dataset_id: table lookup parameter
table_id: table lookup parameter
Returns:
bigquery.Table instance
Raises:
HttpError: if lookup failed.
"""
request = bigquery.BigqueryTablesGetRequest(
projectId=project_id, datasetId=dataset_id, tableId=table_id)
response = self.client.tables.Get(request)
return response
def _create_table(
self,
project_id,
dataset_id,
table_id,
schema,
additional_parameters=None):
valid_tablename = regex.fullmatch(_TABLE_PATTERN, table_id, regex.ASCII)
if not valid_tablename:
raise ValueError(
'Invalid BigQuery table name: %s \n'
'See https://cloud.google.com/bigquery/docs/tables#table_naming' %
table_id)
additional_parameters = additional_parameters or {}
table = bigquery.Table(
tableReference=TableReference(
projectId=project_id, datasetId=dataset_id, tableId=table_id),
schema=schema,
**additional_parameters)
request = bigquery.BigqueryTablesInsertRequest(
projectId=project_id, datasetId=dataset_id, table=table)
response = self.client.tables.Insert(request)
_LOGGER.debug("Created the table with id %s", table_id)
# The response is a bigquery.Table instance.
return response
@retry.with_exponential_backoff(
num_retries=MAX_RETRIES,
retry_filter=retry.retry_on_server_errors_and_timeout_filter)
def get_or_create_dataset(
self, project_id, dataset_id, location=None, labels=None):
# Check if dataset already exists otherwise create it
try:
dataset = self.client.datasets.Get(
bigquery.BigqueryDatasetsGetRequest(
projectId=project_id, datasetId=dataset_id))
self.created_temp_dataset = False
return dataset
except HttpError as exn:
if exn.status_code == 404:
_LOGGER.info(
'Dataset %s:%s does not exist so we will create it as temporary '
'with location=%s',
project_id,
dataset_id,
location)
dataset_reference = bigquery.DatasetReference(
projectId=project_id, datasetId=dataset_id)
dataset = bigquery.Dataset(datasetReference=dataset_reference)
if location is not None:
dataset.location = location
if labels is not None:
dataset.labels = _build_dataset_labels(labels)
request = bigquery.BigqueryDatasetsInsertRequest(
projectId=project_id, dataset=dataset)
response = self.client.datasets.Insert(request)
self.created_temp_dataset = True
# The response is a bigquery.Dataset instance.
return response
else:
raise
@retry.with_exponential_backoff(
num_retries=MAX_RETRIES,
retry_filter=retry.retry_on_server_errors_and_timeout_filter)
def _is_table_empty(self, project_id, dataset_id, table_id):
request = bigquery.BigqueryTabledataListRequest(
projectId=project_id,
datasetId=dataset_id,
tableId=table_id,
maxResults=1)
response = self.client.tabledata.List(request)
# The response is a bigquery.TableDataList instance.
return response.totalRows == 0
@retry.with_exponential_backoff(
num_retries=MAX_RETRIES,
retry_filter=retry.retry_on_server_errors_and_timeout_filter)
def _delete_table(self, project_id, dataset_id, table_id):
request = bigquery.BigqueryTablesDeleteRequest(
projectId=project_id, datasetId=dataset_id, tableId=table_id)
try:
self.client.tables.Delete(request)
except HttpError as exn:
if exn.status_code == 404:
_LOGGER.warning(
'Table %s:%s.%s does not exist', project_id, dataset_id, table_id)
return
else:
raise
@retry.with_exponential_backoff(
num_retries=MAX_RETRIES,
retry_filter=retry.retry_on_server_errors_and_timeout_filter)
def _delete_dataset(self, project_id, dataset_id, delete_contents=True):
request = bigquery.BigqueryDatasetsDeleteRequest(
projectId=project_id,
datasetId=dataset_id,
deleteContents=delete_contents)
try:
self.client.datasets.Delete(request)
except HttpError as exn:
if exn.status_code == 404:
_LOGGER.warning('Dataset %s:%s does not exist', project_id, dataset_id)
return
else:
raise
@retry.with_exponential_backoff(
num_retries=MAX_RETRIES,
retry_filter=retry.retry_on_server_errors_and_timeout_filter)
def get_table_location(self, project_id, dataset_id, table_id):
table = self.get_table(project_id, dataset_id, table_id)
return table.location
# Returns true if the temporary dataset was provided by the user.
def is_user_configured_dataset(self):
return (
self.temp_dataset_id and
not self.temp_dataset_id.startswith(self.TEMP_DATASET))
@retry.with_exponential_backoff(
num_retries=MAX_RETRIES,
retry_filter=retry.retry_on_server_errors_and_timeout_filter)
def create_temporary_dataset(self, project_id, location, labels=None):
self.get_or_create_dataset(
project_id, self.temp_dataset_id, location=location, labels=labels)
if (project_id is not None and not self.is_user_configured_dataset() and
not self.created_temp_dataset):
# Unittests don't pass projectIds so they can be run without error
# User configured datasets are allowed to pre-exist.
raise RuntimeError(
'Dataset %s:%s already exists so cannot be used as temporary.' %
(project_id, self.temp_dataset_id))
@retry.with_exponential_backoff(
num_retries=MAX_RETRIES,
retry_filter=retry.retry_on_server_errors_and_timeout_filter)
def clean_up_temporary_dataset(self, project_id):
temp_table = self._get_temp_table(project_id)
try:
self.client.datasets.Get(
bigquery.BigqueryDatasetsGetRequest(
projectId=project_id, datasetId=temp_table.datasetId))
except HttpError as exn:
if exn.status_code == 404:
_LOGGER.warning(
'Dataset %s:%s does not exist', project_id, temp_table.datasetId)
return
else:
raise
try:
# We do not want to delete temporary datasets configured by the user hence
# we just delete the temporary table in that case.
if not self.is_user_configured_dataset():
self._delete_dataset(temp_table.projectId, temp_table.datasetId, True)
else:
self._delete_table(
temp_table.projectId, temp_table.datasetId, temp_table.tableId)
self.created_temp_dataset = False
except HttpError as exn:
if exn.status_code == 403:
_LOGGER.warning(
'Permission denied to delete temporary dataset %s:%s for clean up',
temp_table.projectId,
temp_table.datasetId)
return
else:
raise
@retry.with_exponential_backoff(
num_retries=MAX_RETRIES,
retry_filter=retry.retry_on_server_errors_and_timeout_filter)
def _clean_up_beam_labelled_temporary_datasets(
self, project_id, dataset_id=None, table_id=None, labels=None):
if isinstance(labels, dict):
filter_str = _build_filter_from_labels(labels)
if not self.is_user_configured_dataset() and labels is not None:
response = (
self.client.datasets.List(
bigquery.BigqueryDatasetsListRequest(
projectId=project_id, filter=filter_str)))
for dataset in response.datasets:
try:
dataset_id = dataset.datasetReference.datasetId
self._delete_dataset(project_id, dataset_id, True)
except HttpError as exn:
if exn.status_code == 403:
_LOGGER.warning(
'Permission denied to delete temporary dataset %s:%s for '
'clean up.',
project_id,
dataset_id)
return
else:
raise
else:
try:
self._delete_table(project_id, dataset_id, table_id)
except HttpError as exn:
if exn.status_code == 403:
_LOGGER.warning(
'Permission denied to delete temporary table %s:%s.%s for '