[AIRFLOW-1340] Add S3 to Redshift transfer operator
Currently RedshiftToS3Transfer (UNLOAD) exists but the opposite doesn't. This PR adds COPY operation as S3ToRedshiftTransfer. Closes #3161 from sekikn/AIRFLOW-1340
This commit is contained in:
Родитель
687bd03df0
Коммит
d9f22e44c0
|
@ -0,0 +1,90 @@
|
|||
# -*- coding: utf-8 -*-
|
||||
#
|
||||
# Licensed 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.
|
||||
from airflow.hooks.postgres_hook import PostgresHook
|
||||
from airflow.hooks.S3_hook import S3Hook
|
||||
from airflow.models import BaseOperator
|
||||
from airflow.utils.decorators import apply_defaults
|
||||
|
||||
|
||||
class S3ToRedshiftTransfer(BaseOperator):
|
||||
"""
|
||||
Executes an COPY command to load files from s3 to Redshift
|
||||
|
||||
:param schema: reference to a specific schema in redshift database
|
||||
:type schema: string
|
||||
:param table: reference to a specific table in redshift database
|
||||
:type table: string
|
||||
:param s3_bucket: reference to a specific S3 bucket
|
||||
:type s3_bucket: string
|
||||
:param s3_key: reference to a specific S3 key
|
||||
:type s3_key: string
|
||||
:param redshift_conn_id: reference to a specific redshift database
|
||||
:type redshift_conn_id: string
|
||||
:param aws_conn_id: reference to a specific S3 connection
|
||||
:type aws_conn_id: string
|
||||
:param copy_options: reference to a list of COPY options
|
||||
:type copy_options: list
|
||||
"""
|
||||
|
||||
template_fields = ()
|
||||
template_ext = ()
|
||||
ui_color = '#ededed'
|
||||
|
||||
@apply_defaults
|
||||
def __init__(
|
||||
self,
|
||||
schema,
|
||||
table,
|
||||
s3_bucket,
|
||||
s3_key,
|
||||
redshift_conn_id='redshift_default',
|
||||
aws_conn_id='aws_default',
|
||||
copy_options=tuple(),
|
||||
autocommit=False,
|
||||
parameters=None,
|
||||
*args, **kwargs):
|
||||
super(S3ToRedshiftTransfer, self).__init__(*args, **kwargs)
|
||||
self.schema = schema
|
||||
self.table = table
|
||||
self.s3_bucket = s3_bucket
|
||||
self.s3_key = s3_key
|
||||
self.redshift_conn_id = redshift_conn_id
|
||||
self.aws_conn_id = aws_conn_id
|
||||
self.copy_options = copy_options
|
||||
self.autocommit = autocommit
|
||||
self.parameters = parameters
|
||||
|
||||
def execute(self, context):
|
||||
self.hook = PostgresHook(postgres_conn_id=self.redshift_conn_id)
|
||||
self.s3 = S3Hook(aws_conn_id=self.aws_conn_id)
|
||||
credentials = self.s3.get_credentials()
|
||||
copy_options = '\n\t\t\t'.join(self.copy_options)
|
||||
|
||||
copy_query = """
|
||||
COPY {schema}.{table}
|
||||
FROM 's3://{s3_bucket}/{s3_key}/{table}'
|
||||
with credentials
|
||||
'aws_access_key_id={access_key};aws_secret_access_key={secret_key}'
|
||||
{copy_options};
|
||||
""".format(schema=self.schema,
|
||||
table=self.table,
|
||||
s3_bucket=self.s3_bucket,
|
||||
s3_key=self.s3_key,
|
||||
access_key=credentials.access_key,
|
||||
secret_key=credentials.secret_key,
|
||||
copy_options=copy_options)
|
||||
|
||||
self.log.info('Executing COPY command...')
|
||||
self.hook.run(copy_query, self.autocommit)
|
||||
self.log.info("COPY command complete...")
|
|
@ -76,6 +76,7 @@ Operators
|
|||
.. autoclass:: airflow.operators.python_operator.PythonVirtualenvOperator
|
||||
.. autoclass:: airflow.operators.s3_file_transform_operator.S3FileTransformOperator
|
||||
.. autoclass:: airflow.operators.s3_to_hive_operator.S3ToHiveTransfer
|
||||
.. autoclass:: airflow.operators.s3_to_redshift_operator.S3ToRedshiftTransfer
|
||||
.. autoclass:: airflow.operators.python_operator.ShortCircuitOperator
|
||||
.. autoclass:: airflow.operators.http_operator.SimpleHttpOperator
|
||||
.. autoclass:: airflow.operators.slack_operator.SlackAPIOperator
|
||||
|
|
|
@ -0,0 +1,70 @@
|
|||
# -*- coding: utf-8 -*-
|
||||
#
|
||||
# Licensed 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 mock
|
||||
import re
|
||||
import unittest
|
||||
|
||||
from boto3.session import Session
|
||||
from airflow.operators.s3_to_redshift_operator import S3ToRedshiftTransfer
|
||||
|
||||
|
||||
class TestS3ToRedshiftTransfer(unittest.TestCase):
|
||||
|
||||
@mock.patch("boto3.session.Session")
|
||||
@mock.patch("airflow.hooks.postgres_hook.PostgresHook.run")
|
||||
def test_execute(self, mock_run, mock_Session):
|
||||
access_key = "aws_access_key_id"
|
||||
secret_key = "aws_secret_access_key"
|
||||
mock_Session.return_value = Session(access_key, secret_key)
|
||||
|
||||
schema = "schema"
|
||||
table = "table"
|
||||
s3_bucket = "bucket"
|
||||
s3_key = "key"
|
||||
copy_options = ""
|
||||
|
||||
t = S3ToRedshiftTransfer(
|
||||
schema=schema,
|
||||
table=table,
|
||||
s3_bucket=s3_bucket,
|
||||
s3_key=s3_key,
|
||||
copy_options=copy_options,
|
||||
redshift_conn_id="redshift_conn_id",
|
||||
aws_conn_id="aws_conn_id",
|
||||
task_id="task_id",
|
||||
dag=None)
|
||||
t.execute(None)
|
||||
|
||||
copy_query = """
|
||||
COPY {schema}.{table}
|
||||
FROM 's3://{s3_bucket}/{s3_key}/{table}'
|
||||
with credentials
|
||||
'aws_access_key_id={access_key};aws_secret_access_key={secret_key}'
|
||||
{copy_options};
|
||||
""".format(schema=schema,
|
||||
table=table,
|
||||
s3_bucket=s3_bucket,
|
||||
s3_key=s3_key,
|
||||
access_key=access_key,
|
||||
secret_key=secret_key,
|
||||
copy_options=copy_options)
|
||||
|
||||
def _trim(s):
|
||||
return re.sub("\s+", " ", s.strip())
|
||||
|
||||
self.assertEqual(_trim(mock_run.call_args[0][0]),
|
||||
_trim(copy_query))
|
||||
mock_run.assert_called_once()
|
Загрузка…
Ссылка в новой задаче