-
Notifications
You must be signed in to change notification settings - Fork 22
Expand file tree
/
Copy paths3_to_redshift_operator.py
More file actions
409 lines (366 loc) · 17.6 KB
/
s3_to_redshift_operator.py
File metadata and controls
409 lines (366 loc) · 17.6 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
import json
import random
import string
import logging
from airflow.utils.db import provide_session
from airflow.models import Connection
from airflow.utils.decorators import apply_defaults
from airflow.models import BaseOperator
from airflow.hooks.S3_hook import S3Hook
from airflow.hooks.postgres_hook import PostgresHook
class S3ToRedshiftOperator(BaseOperator):
"""
S3 To Redshift Operator
:param redshift_conn_id: The destination redshift connection id.
:type redshift_conn_id: string
:param redshift_schema: The destination redshift schema.
:type redshift_schema: string
:param table: The destination redshift table.
:type table: string
:param s3_conn_id: The source s3 connection id.
:type s3_conn_id: string
:param s3_bucket: The source s3 bucket.
:type s3_bucket: string
:param s3_key: The source s3 key.
:type s3_key: string
:param copy_params: The parameters to be included when issuing
the copy statement in Redshift.
:type copy_params: list
:param origin_schema: The s3 key for the incoming data schema.
Expects a JSON file with an array of
dictionaries specifying name and type.
(e.g. {"name": "_id", "type": "int4"})
:type origin_schema: array of dictionaries
:param schema_location: The location of the origin schema. This
can be set to 'S3' or 'Local'.
If 'S3', it will expect a valid S3 Key. If
'Local', it will expect a dictionary that
is defined in the operator itself. By
default the location is set to 's3'.
:type schema_location: string
:param load_type: The method of loading into Redshift that
should occur. Options:
- "append"
- "rebuild"
- "truncate"
- "upsert"
Defaults to "append."
:type load_type: string
:param primary_key: *(optional)* The primary key for the
destination table. Not enforced by redshift
and only required if using a load_type of
"upsert".
:type primary_key: string
:param incremental_key: *(optional)* The incremental key to compare
new data against the destination table
with. Only required if using a load_type of
"upsert".
:type incremental_key: string
:param foreign_key: *(optional)* This specifies any foreign_keys
in the table and which corresponding table
and key they reference. This may be either
a dictionary or list of dictionaries (for
multiple foreign keys). The fields that are
required in each dictionary are:
- column_name
- reftable
- ref_column
:type foreign_key: dictionary
:param distkey: *(optional)* The distribution key for the
table. Only one key may be specified.
:type distkey: string
:param sortkey: *(optional)* The sort keys for the table.
If more than one key is specified, set this
as a list.
:type sortkey: string
:param sort_type: *(optional)* The style of distribution
to sort the table. Possible values include:
- compound
- interleaved
Defaults to "compound".
:type sort_type: string
"""
template_fields = ('s3_key',
'origin_schema')
@apply_defaults
def __init__(self,
s3_conn_id,
s3_bucket,
s3_key,
redshift_conn_id,
redshift_schema,
table,
copy_params=[],
origin_schema=None,
schema_location='s3',
load_type='append',
primary_key=None,
incremental_key=None,
foreign_key={},
distkey=None,
sortkey='',
sort_type='COMPOUND',
*args,
**kwargs):
super().__init__(*args, **kwargs)
self.s3_conn_id = s3_conn_id
self.s3_bucket = s3_bucket
self.s3_key = s3_key
self.redshift_conn_id = redshift_conn_id
self.redshift_schema = redshift_schema.lower()
self.table = table.lower()
self.copy_params = copy_params
self.origin_schema = origin_schema
self.schema_location = schema_location
self.load_type = load_type
self.primary_key = primary_key
self.incremental_key = incremental_key
self.foreign_key = foreign_key
self.distkey = distkey
self.sortkey = sortkey
self.sort_type = sort_type
if self.load_type.lower() not in ("append", "rebuild", "truncate", "upsert"):
raise Exception('Please choose "append", "rebuild", or "upsert".')
if self.schema_location.lower() not in ('s3', 'local'):
raise Exception('Valid Schema Locations are "s3" or "local".')
if not (isinstance(self.sortkey, str) or isinstance(self.sortkey, list)):
raise Exception('Sort Keys must be specified as either a string or list.')
if not (isinstance(self.foreign_key, dict) or isinstance(self.foreign_key, list)):
raise Exception('Foreign Keys must be specified as either a dictionary or a list of dictionaries.')
if self.distkey and ((',' in self.distkey) or not isinstance(self.distkey, str)):
raise Exception('Only one distribution key may be specified.')
if self.sort_type.lower() not in ('compound', 'interleaved'):
raise Exception('Please choose "compound" or "interleaved" for sort type.')
def execute(self, context):
# Append a random string to the end of the staging table to ensure
# no conflicts if multiple processes running concurrently.
letters = string.ascii_lowercase
random_string = ''.join(random.choice(letters) for _ in range(7))
self.temp_suffix = '_tmp_{0}'.format(random_string)
if self.origin_schema:
schema = self.read_and_format()
pg_hook = PostgresHook(self.redshift_conn_id)
self.create_if_not_exists(schema, pg_hook)
self.reconcile_schemas(schema, pg_hook)
self.copy_data(pg_hook, schema)
def read_and_format(self):
if self.schema_location.lower() == 's3':
hook = S3Hook(self.s3_conn_id)
# NOTE: In retrieving the schema, it is assumed
# that boto3 is being used. If using boto,
# `.get()['Body'].read().decode('utf-8'))`
# should be changed to
# `.get_contents_as_string(encoding='utf-8'))`
schema = (hook.get_key(self.origin_schema,
bucket_name=
'{0}'.format(self.s3_bucket))
.get()['Body'].read().decode('utf-8'))
schema = json.loads(schema.replace("'", '"'))
else:
schema = self.origin_schema
return schema
def reconcile_schemas(self, schema, pg_hook):
pg_query = \
"""
SELECT column_name, udt_name
FROM information_schema.columns
WHERE table_schema = '{0}' AND table_name = '{1}';
""".format(self.redshift_schema, self.table)
pg_schema = dict(pg_hook.get_records(pg_query))
incoming_keys = [column['name'].lower() for column in schema]
diff = list(set(incoming_keys) - set(pg_schema.keys()))
print(diff)
# Check length of column differential to see if any new columns exist
if len(diff):
for i in diff:
for e in schema:
if i == e['name']:
alter_query = \
"""
ALTER TABLE "{0}"."{1}"
ADD COLUMN "{2}" {3}
""".format(self.redshift_schema,
self.table,
e['name'],
e['type'])
pg_hook.run(alter_query)
logging.info('The new columns were:' + str(diff))
else:
logging.info('There were no new columns.')
def copy_data(self, pg_hook, schema=None):
@provide_session
def get_conn(conn_id, session=None):
conn = (
session.query(Connection)
.filter(Connection.conn_id == conn_id)
.first())
return conn
def getS3Conn():
creds = ""
s3_conn = get_conn(self.s3_conn_id)
aws_key = s3_conn.extra_dejson.get('aws_access_key_id', None)
aws_secret = s3_conn.extra_dejson.get('aws_secret_access_key', None)
# support for cross account resource access
aws_role_arn = s3_conn.extra_dejson.get('role_arn', None)
if aws_key and aws_secret:
creds = ("aws_access_key_id={0};aws_secret_access_key={1}"
.format(aws_key, aws_secret))
elif aws_role_arn:
creds = ("aws_iam_role={0}"
.format(aws_role_arn))
return creds
# Delete records from the destination table where the incremental_key
# is greater than or equal to the incremental_key of the source table
# and the primary key is the same.
# (e.g. Source: {"id": 1, "updated_at": "2017-01-02 00:00:00"};
# Destination: {"id": 1, "updated_at": "2017-01-01 00:00:00"})
delete_sql = \
'''
DELETE FROM "{rs_schema}"."{rs_table}"
USING "{rs_schema}"."{rs_table}{rs_suffix}"
WHERE "{rs_schema}"."{rs_table}"."{rs_pk}" =
"{rs_schema}"."{rs_table}{rs_suffix}"."{rs_pk}"
AND "{rs_schema}"."{rs_table}{rs_suffix}"."{rs_ik}" >=
"{rs_schema}"."{rs_table}"."{rs_ik}"
'''.format(rs_schema=self.redshift_schema,
rs_table=self.table,
rs_pk=self.primary_key,
rs_suffix=self.temp_suffix,
rs_ik=self.incremental_key)
# Delete records from the source table where the incremental_key
# is greater than or equal to the incremental_key of the destination
# table and the primary key is the same. This is done in the edge case
# where data is pulled BEFORE it is altered in the source table but
# AFTER a workflow containing an updated version of the record runs.
# In this case, not running this will cause the older record to be
# added as a duplicate to the newer record.
# (e.g. Source: {"id": 1, "updated_at": "2017-01-01 00:00:00"};
# Destination: {"id": 1, "updated_at": "2017-01-02 00:00:00"})
delete_confirm_sql = \
'''
DELETE FROM "{rs_schema}"."{rs_table}{rs_suffix}"
USING "{rs_schema}"."{rs_table}"
WHERE "{rs_schema}"."{rs_table}{rs_suffix}"."{rs_pk}" =
"{rs_schema}"."{rs_table}"."{rs_pk}"
AND "{rs_schema}"."{rs_table}"."{rs_ik}" >=
"{rs_schema}"."{rs_table}{rs_suffix}"."{rs_ik}"
'''.format(rs_schema=self.redshift_schema,
rs_table=self.table,
rs_pk=self.primary_key,
rs_suffix=self.temp_suffix,
rs_ik=self.incremental_key)
append_sql = \
'''
ALTER TABLE "{0}"."{1}"
APPEND FROM "{0}"."{1}{2}"
FILLTARGET
'''.format(self.redshift_schema, self.table, self.temp_suffix)
drop_sql = \
'''
DROP TABLE IF EXISTS "{0}"."{1}"
'''.format(self.redshift_schema, self.table)
drop_temp_sql = \
'''
DROP TABLE IF EXISTS "{0}"."{1}{2}"
'''.format(self.redshift_schema, self.table, self.temp_suffix)
truncate_sql = \
'''
TRUNCATE TABLE "{0}"."{1}"
'''.format(self.redshift_schema, self.table)
params = '\n'.join(self.copy_params)
# Example params for loading json from US-East-1 S3 region
# params = ["COMPUPDATE OFF",
# "STATUPDATE OFF",
# "JSON 'auto'",
# "TIMEFORMAT 'auto'",
# "TRUNCATECOLUMNS",
# "region as 'us-east-1'"]
base_sql = \
"""
FROM 's3://{0}/{1}'
CREDENTIALS '{2}'
{3};
""".format(self.s3_bucket,
self.s3_key,
getS3Conn(),
params)
load_sql = '''COPY "{0}"."{1}" {2}'''.format(self.redshift_schema,
self.table,
base_sql)
if self.load_type == 'append':
pg_hook.run(load_sql)
elif self.load_type == 'rebuild':
pg_hook.run(drop_sql)
self.create_if_not_exists(schema, pg_hook)
pg_hook.run(load_sql)
elif self.load_type == 'truncate':
pg_hook.run(truncate_sql)
pg_hook.run(load_sql)
elif self.load_type == 'upsert':
self.create_if_not_exists(schema, pg_hook, temp=True)
load_temp_sql = \
'''COPY "{0}"."{1}{2}" {3}'''.format(self.redshift_schema,
self.table,
self.temp_suffix,
base_sql)
pg_hook.run(load_temp_sql)
pg_hook.run(delete_sql)
pg_hook.run(delete_confirm_sql)
pg_hook.run(append_sql, autocommit=True)
pg_hook.run(drop_temp_sql)
def create_if_not_exists(self, schema, pg_hook, temp=False):
output = ''
for item in schema:
k = "{quote}{key}{quote}".format(quote='"', key=item['name'])
field = ' '.join([k, item['type']])
if isinstance(self.sortkey, str) and self.sortkey == item['name']:
field += ' sortkey'
output += field
output += ', '
# Remove last comma and space after schema items loop ends
output = output[:-2]
if temp:
copy_table = '{0}{1}'.format(self.table, self.temp_suffix)
else:
copy_table = self.table
create_schema_query = \
'''
CREATE SCHEMA IF NOT EXISTS "{0}";
'''.format(self.redshift_schema)
pk = ''
fk = ''
dk = ''
sk = ''
if self.primary_key:
pk = ', primary key("{0}")'.format(self.primary_key)
if self.foreign_key:
if isinstance(self.foreign_key, list):
fk = ', '
for i, e in enumerate(self.foreign_key):
fk += 'foreign key("{0}") references {1}("{2}")'.format(e['column_name'],
e['reftable'],
e['ref_column'])
if i != (len(self.foreign_key) - 1):
fk += ', '
elif isinstance(self.foreign_key, dict):
fk += ', '
fk += 'foreign key("{0}") references {1}("{2}")'.format(self.foreign_key['column_name'],
self.foreign_key['reftable'],
self.foreign_key['ref_column'])
if self.distkey:
dk = 'distkey({})'.format(self.distkey)
if self.sortkey:
if isinstance(self.sortkey, list):
sk += '{0} sortkey({1})'.format(self.sort_type, ', '.join(["{}".format(e) for e in self.sortkey]))
create_table_query = \
'''
CREATE TABLE IF NOT EXISTS "{schema}"."{table}"
({fields}{primary_key}{foreign_key}) {distkey} {sortkey}
'''.format(schema=self.redshift_schema,
table=copy_table,
fields=output,
primary_key=pk,
foreign_key=fk,
distkey=dk,
sortkey=sk)
pg_hook.run([create_schema_query, create_table_query])