Skip to content

Commit

Permalink
[AIRFLOW-3783] Speed up Redshift to S3 UNload with HEADERs (#6309)
Browse files Browse the repository at this point in the history
replace the default unload_option 'PARALLEL OFF' by 'HEADER' as Redshift handles
header with parallel mode now

Co-authored-by: Shreya Chakraborty <[email protected]>
  • Loading branch information
2 people authored and ashb committed Oct 14, 2019
1 parent 55facad commit 3ed7b32
Show file tree
Hide file tree
Showing 3 changed files with 26 additions and 83 deletions.
62 changes: 17 additions & 45 deletions airflow/operators/redshift_to_s3_operator.py
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,9 @@
# KIND, either express or implied. See the License for the
# specific language governing permissions and limitations
# under the License.
"""
Transfers data from AWS Redshift into a S3 Bucket.
"""
from typing import List, Optional, Union

from airflow.hooks.postgres_hook import PostgresHook
Expand Down Expand Up @@ -53,14 +56,19 @@ class RedshiftToS3Transfer(BaseOperator):
:type verify: bool or str
:param unload_options: reference to a list of UNLOAD options
:type unload_options: list
:param autocommit: If set to True it will automatically commit the UNLOAD statement.
Otherwise it will be committed right before the redshift connection gets closed.
:type autocommit: bool
:param include_header: If set to True the s3 file contains the header columns.
:type include_header: bool
"""

template_fields = ()
template_ext = ()
ui_color = '#ededed'

@apply_defaults
def __init__(
def __init__( # pylint: disable=too-many-arguments
self,
schema: str,
table: str,
Expand All @@ -85,52 +93,16 @@ def __init__(
self.autocommit = autocommit
self.include_header = include_header

if self.include_header and 'PARALLEL OFF' not in [uo.upper().strip() for uo in self.unload_options]:
self.unload_options = list(self.unload_options) + ['PARALLEL OFF', ]
if self.include_header and 'HEADER' not in [uo.upper().strip() for uo in self.unload_options]:
self.unload_options = list(self.unload_options) + ['HEADER', ]

def execute(self, context):
self.hook = PostgresHook(postgres_conn_id=self.redshift_conn_id)
self.s3 = S3Hook(aws_conn_id=self.aws_conn_id, verify=self.verify)
credentials = self.s3.get_credentials()
unload_options = '\n\t\t\t'.join(self.unload_options)

if self.include_header:
self.log.info("Retrieving headers from %s.%s...",
self.schema, self.table)

columns_query = """SELECT column_name
FROM information_schema.columns
WHERE table_schema = '{schema}'
AND table_name = '{table}'
ORDER BY ordinal_position
""".format(schema=self.schema,
table=self.table)

cursor = self.hook.get_conn().cursor()
cursor.execute(columns_query)
rows = cursor.fetchall()
columns = [row[0] for row in rows]
column_names = ', '.join("{0}".format(c) for c in columns)
column_headers = ', '.join("\\'{0}\\'".format(c) for c in columns)
column_castings = ', '.join("CAST({0} AS text) AS {0}".format(c)
for c in columns)

select_query = """SELECT {column_names} FROM
(SELECT 2 sort_order, {column_castings}
FROM {schema}.{table}
UNION ALL
SELECT 1 sort_order, {column_headers})
ORDER BY sort_order"""\
.format(column_names=column_names,
column_castings=column_castings,
column_headers=column_headers,
schema=self.schema,
table=self.table)
else:
select_query = "SELECT * FROM {schema}.{table}"\
.format(schema=self.schema,
table=self.table)
postgres_hook = PostgresHook(postgres_conn_id=self.redshift_conn_id)
s3_hook = S3Hook(aws_conn_id=self.aws_conn_id, verify=self.verify)

credentials = s3_hook.get_credentials()
unload_options = '\n\t\t\t'.join(self.unload_options)
select_query = "SELECT * FROM {schema}.{table}".format(schema=self.schema, table=self.table)
unload_query = """
UNLOAD ('{select_query}')
TO 's3://{s3_bucket}/{s3_key}/{table}_'
Expand All @@ -146,5 +118,5 @@ def execute(self, context):
unload_options=unload_options)

self.log.info('Executing UNLOAD command...')
self.hook.run(unload_query, self.autocommit)
postgres_hook.run(unload_query, self.autocommit)
self.log.info("UNLOAD command complete...")
2 changes: 0 additions & 2 deletions scripts/ci/pylint_todo.txt
Original file line number Diff line number Diff line change
Expand Up @@ -196,7 +196,6 @@
./airflow/operators/presto_check_operator.py
./airflow/operators/presto_to_mysql.py
./airflow/operators/python_operator.py
./airflow/operators/redshift_to_s3_operator.py
./airflow/operators/s3_file_transform_operator.py
./airflow/operators/s3_to_hive_operator.py
./airflow/operators/s3_to_redshift_operator.py
Expand Down Expand Up @@ -356,7 +355,6 @@
./tests/operators/test_operators.py
./tests/operators/test_papermill_operator.py
./tests/operators/test_python_operator.py
./tests/operators/test_redshift_to_s3_operator.py
./tests/operators/test_s3_file_transform_operator.py
./tests/operators/test_s3_to_hive_operator.py
./tests/operators/test_s3_to_redshift_operator.py
Expand Down
45 changes: 9 additions & 36 deletions tests/operators/test_redshift_to_s3_operator.py
Original file line number Diff line number Diff line change
Expand Up @@ -31,24 +31,17 @@ class TestRedshiftToS3Transfer(unittest.TestCase):

@mock.patch("boto3.session.Session")
@mock.patch("airflow.hooks.postgres_hook.PostgresHook.run")
@mock.patch("airflow.hooks.postgres_hook.PostgresHook.get_conn")
def test_execute(self, mock_get_conn, mock_run, mock_Session):
column_name = "col"
cur = mock.MagicMock()
cur.fetchall.return_value = [(column_name, )]
mock_get_conn.return_value.cursor.return_value = cur

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)

mock_session.return_value = Session(access_key, secret_key)
schema = "schema"
table = "table"
s3_bucket = "bucket"
s3_key = "key"
unload_options = ('PARALLEL OFF',)
unload_options = ['HEADER', ]

t = RedshiftToS3Transfer(
RedshiftToS3Transfer(
schema=schema,
table=table,
s3_bucket=s3_bucket,
Expand All @@ -58,43 +51,23 @@ def test_execute(self, mock_get_conn, mock_run, mock_Session):
redshift_conn_id="redshift_conn_id",
aws_conn_id="aws_conn_id",
task_id="task_id",
dag=None)
t.execute(None)
dag=None
).execute(None)

unload_options = '\n\t\t\t'.join(unload_options)

columns_query = """
SELECT column_name
FROM information_schema.columns
WHERE table_schema = '{schema}'
AND table_name = '{table}'
ORDER BY ordinal_position
""".format(schema=schema,
table=table)

select_query = "SELECT * FROM {schema}.{table}".format(schema=schema, table=table)
unload_query = """
UNLOAD ('SELECT {column_name} FROM
(SELECT 2 sort_order,
CAST({column_name} AS text) AS {column_name}
FROM {schema}.{table}
UNION ALL
SELECT 1 sort_order, \\'{column_name}\\')
ORDER BY sort_order')
UNLOAD ('{select_query}')
TO 's3://{s3_bucket}/{s3_key}/{table}_'
with credentials
'aws_access_key_id={access_key};aws_secret_access_key={secret_key}'
{unload_options};
""".format(column_name=column_name,
schema=schema,
""".format(select_query=select_query,
table=table,
s3_bucket=s3_bucket,
s3_key=s3_key,
access_key=access_key,
secret_key=secret_key,
unload_options=unload_options)

assert cur.execute.call_count == 1
assertEqualIgnoreMultipleSpaces(self, cur.execute.call_args[0][0], columns_query)

assert mock_run.call_count == 1
assertEqualIgnoreMultipleSpaces(self, mock_run.call_args[0][0], unload_query)

0 comments on commit 3ed7b32

Please sign in to comment.