/
gcs_to_bq.py
261 lines (244 loc) · 12.2 KB
/
gcs_to_bq.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
# -*- 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.
import json
from airflow.contrib.hooks.gcs_hook import GoogleCloudStorageHook
from airflow.contrib.hooks.bigquery_hook import BigQueryHook
from airflow.models import BaseOperator
from airflow.utils.decorators import apply_defaults
class GoogleCloudStorageToBigQueryOperator(BaseOperator):
"""
Loads files from Google cloud storage into BigQuery.
The schema to be used for the BigQuery table may be specified in one of
two ways. You may either directly pass the schema fields in, or you may
point the operator to a Google cloud storage object name. The object in
Google cloud storage must be a JSON file with the schema fields in it.
:param bucket: The bucket to load from. (templated)
:type bucket: str
:param source_objects: List of Google cloud storage URIs to load from. (templated)
If source_format is 'DATASTORE_BACKUP', the list must only contain a single URI.
:type source_objects: list of str
:param destination_project_dataset_table: The dotted (<project>.)<dataset>.<table>
BigQuery table to load data into. If <project> is not included,
project will be the project defined in the connection json. (templated)
:type destination_project_dataset_table: str
:param schema_fields: If set, the schema field list as defined here:
https://cloud.google.com/bigquery/docs/reference/v2/jobs#configuration.load
Should not be set when source_format is 'DATASTORE_BACKUP'.
:type schema_fields: list
:param schema_object: If set, a GCS object path pointing to a .json file that
contains the schema for the table. (templated)
:type schema_object: str
:param source_format: File format to export.
:type source_format: str
:param compression: [Optional] The compression type of the data source.
Possible values include GZIP and NONE.
The default value is NONE.
This setting is ignored for Google Cloud Bigtable,
Google Cloud Datastore backups and Avro formats.
:type compression: str
:param create_disposition: The create disposition if the table doesn't exist.
:type create_disposition: str
:param skip_leading_rows: Number of rows to skip when loading from a CSV.
:type skip_leading_rows: int
:param write_disposition: The write disposition if the table already exists.
:type write_disposition: str
:param field_delimiter: The delimiter to use when loading from a CSV.
:type field_delimiter: str
:param max_bad_records: The maximum number of bad records that BigQuery can
ignore when running the job.
:type max_bad_records: int
:param quote_character: The value that is used to quote data sections in a CSV file.
:type quote_character: str
:param ignore_unknown_values: [Optional] Indicates if BigQuery should allow
extra values that are not represented in the table schema.
If true, the extra values are ignored. If false, records with extra columns
are treated as bad records, and if there are too many bad records, an
invalid error is returned in the job result.
:type ignore_unknown_values: bool
:param allow_quoted_newlines: Whether to allow quoted newlines (true) or not (false).
:type allow_quoted_newlines: bool
:param allow_jagged_rows: Accept rows that are missing trailing optional columns.
The missing values are treated as nulls. If false, records with missing trailing
columns are treated as bad records, and if there are too many bad records, an
invalid error is returned in the job result. Only applicable to CSV, ignored
for other formats.
:type allow_jagged_rows: bool
:param max_id_key: If set, the name of a column in the BigQuery table
that's to be loaded. This will be used to select the MAX value from
BigQuery after the load occurs. The results will be returned by the
execute() command, which in turn gets stored in XCom for future
operators to use. This can be helpful with incremental loads--during
future executions, you can pick up from the max ID.
:type max_id_key: str
:param bigquery_conn_id: Reference to a specific BigQuery hook.
:type bigquery_conn_id: str
:param google_cloud_storage_conn_id: Reference to a specific Google
cloud storage hook.
:type google_cloud_storage_conn_id: str
:param delegate_to: The account to impersonate, if any. For this to
work, the service account making the request must have domain-wide
delegation enabled.
:type delegate_to: str
:param schema_update_options: Allows the schema of the destination
table to be updated as a side effect of the load job.
:type schema_update_options: list
:param src_fmt_configs: configure optional fields specific to the source format
:type src_fmt_configs: dict
:param external_table: Flag to specify if the destination table should be
a BigQuery external table. Default Value is False.
:type external_table: bool
:param time_partitioning: configure optional time partitioning fields i.e.
partition by field, type and expiration as per API specifications.
Note that 'field' is not available in concurrency with
dataset.table$partition.
:type time_partitioning: dict
:param cluster_fields: Request that the result of this load be stored sorted
by one or more columns. This is only available in conjunction with
time_partitioning. The order of columns given determines the sort order.
Not applicable for external tables.
:type cluster_fields: list of str
"""
template_fields = ('bucket', 'source_objects',
'schema_object', 'destination_project_dataset_table')
template_ext = ('.sql',)
ui_color = '#f0eee4'
@apply_defaults
def __init__(self,
bucket,
source_objects,
destination_project_dataset_table,
schema_fields=None,
schema_object=None,
source_format='CSV',
compression='NONE',
create_disposition='CREATE_IF_NEEDED',
skip_leading_rows=0,
write_disposition='WRITE_EMPTY',
field_delimiter=',',
max_bad_records=0,
quote_character=None,
ignore_unknown_values=False,
allow_quoted_newlines=False,
allow_jagged_rows=False,
max_id_key=None,
bigquery_conn_id='bigquery_default',
google_cloud_storage_conn_id='google_cloud_default',
delegate_to=None,
schema_update_options=(),
src_fmt_configs=None,
external_table=False,
time_partitioning=None,
cluster_fields=None,
*args, **kwargs):
super(GoogleCloudStorageToBigQueryOperator, self).__init__(*args, **kwargs)
# GCS config
if src_fmt_configs is None:
src_fmt_configs = {}
if time_partitioning is None:
time_partitioning = {}
self.bucket = bucket
self.source_objects = source_objects
self.schema_object = schema_object
# BQ config
self.destination_project_dataset_table = destination_project_dataset_table
self.schema_fields = schema_fields
self.source_format = source_format
self.compression = compression
self.create_disposition = create_disposition
self.skip_leading_rows = skip_leading_rows
self.write_disposition = write_disposition
self.field_delimiter = field_delimiter
self.max_bad_records = max_bad_records
self.quote_character = quote_character
self.ignore_unknown_values = ignore_unknown_values
self.allow_quoted_newlines = allow_quoted_newlines
self.allow_jagged_rows = allow_jagged_rows
self.external_table = external_table
self.max_id_key = max_id_key
self.bigquery_conn_id = bigquery_conn_id
self.google_cloud_storage_conn_id = google_cloud_storage_conn_id
self.delegate_to = delegate_to
self.schema_update_options = schema_update_options
self.src_fmt_configs = src_fmt_configs
self.time_partitioning = time_partitioning
self.cluster_fields = cluster_fields
def execute(self, context):
bq_hook = BigQueryHook(bigquery_conn_id=self.bigquery_conn_id,
delegate_to=self.delegate_to)
if not self.schema_fields and \
self.schema_object and \
self.source_format != 'DATASTORE_BACKUP':
gcs_hook = GoogleCloudStorageHook(
google_cloud_storage_conn_id=self.google_cloud_storage_conn_id,
delegate_to=self.delegate_to)
schema_fields = json.loads(gcs_hook.download(
self.bucket,
self.schema_object).decode("utf-8"))
else:
schema_fields = self.schema_fields
source_uris = ['gs://{}/{}'.format(self.bucket, source_object)
for source_object in self.source_objects]
conn = bq_hook.get_conn()
cursor = conn.cursor()
if self.external_table:
cursor.create_external_table(
external_project_dataset_table=self.destination_project_dataset_table,
schema_fields=schema_fields,
source_uris=source_uris,
source_format=self.source_format,
compression=self.compression,
skip_leading_rows=self.skip_leading_rows,
field_delimiter=self.field_delimiter,
max_bad_records=self.max_bad_records,
quote_character=self.quote_character,
ignore_unknown_values=self.ignore_unknown_values,
allow_quoted_newlines=self.allow_quoted_newlines,
allow_jagged_rows=self.allow_jagged_rows,
src_fmt_configs=self.src_fmt_configs
)
else:
cursor.run_load(
destination_project_dataset_table=self.destination_project_dataset_table,
schema_fields=schema_fields,
source_uris=source_uris,
source_format=self.source_format,
create_disposition=self.create_disposition,
skip_leading_rows=self.skip_leading_rows,
write_disposition=self.write_disposition,
field_delimiter=self.field_delimiter,
max_bad_records=self.max_bad_records,
quote_character=self.quote_character,
ignore_unknown_values=self.ignore_unknown_values,
allow_quoted_newlines=self.allow_quoted_newlines,
allow_jagged_rows=self.allow_jagged_rows,
schema_update_options=self.schema_update_options,
src_fmt_configs=self.src_fmt_configs,
time_partitioning=self.time_partitioning,
cluster_fields=self.cluster_fields)
if self.max_id_key:
cursor.execute('SELECT MAX({}) FROM {}'.format(
self.max_id_key,
self.destination_project_dataset_table))
row = cursor.fetchone()
max_id = row[0] if row[0] else 0
self.log.info(
'Loaded BQ data with max %s.%s=%s',
self.destination_project_dataset_table, self.max_id_key, max_id
)
return max_id