Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[AIRFLOW-4161] BigQuery to Mysql Operator #5711

Merged
merged 3 commits into from Aug 7, 2019
Merged
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Empty file added __init__.py
Empty file.
143 changes: 143 additions & 0 deletions airflow/contrib/operators/bigquery_to_mysql_operator.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,143 @@
# -*- 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.
"""
This module contains a Google BigQuery to MySQL operator.
"""

from airflow.contrib.hooks.bigquery_hook import BigQueryHook
from airflow.models import BaseOperator
from airflow.utils.decorators import apply_defaults
from airflow.hooks.mysql_hook import MySqlHook


class BigQueryToMySqlOperator(BaseOperator):
"""
Fetches the data from a BigQuery table (alternatively fetch data for selected columns)
and insert that data into a MySQL table.


.. note::
If you pass fields to ``selected_fields`` which are in different order than the
order of columns already in
BQ table, the data will still be in the order of BQ table.
For example if the BQ table has 3 columns as
``[A,B,C]`` and you pass 'B,A' in the ``selected_fields``
the data would still be of the form ``'A,B'`` and passed through this form
to MySQL

**Example**: ::

transfer_data = BigQueryToMySqlOperator(
task_id='task_id',
dataset_id='dataset_id',
table_id='table_name',
mysql_table='dest_table_name',
replace=True,
)

:param dataset_id: The dataset ID of the requested table. (templated)
:type dataset_id: string
:param table_id: The table ID of the requested table. (templated)
:type table_id: string
:param max_results: The maximum number of records (rows) to be fetched
from the table. (templated)
:type max_results: string
:param selected_fields: List of fields to return (comma-separated). If
unspecified, all fields are returned.
:type selected_fields: string
:param bigquery_conn_id: reference to a specific BigQuery hook.
:type bigquery_conn_id: string
: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: string
:param mysql_conn_id: reference to a specific mysql hook
:type mysql_conn_id: string
:param database: name of database which overwrite defined one in connection
:type database: string
:param replace: Whether to replace instead of insert
:type replace: bool
"""
template_fields = ('dataset_id', 'table_id', 'mysql_table')

@apply_defaults
def __init__(self,
dataset_id,
table_id,
mysql_table,
selected_fields=None,
bigquery_conn_id='bigquery_default',
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is it better to use 'google_cloud_default' as conn_id here because we've already unified all the GCP conn_id.
Reference: #4818

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It is better to use gcp_conn_id as parameter name here because it is written in integration recommendations. In last night, I started working on unification
PolideaInternal#201

mysql_conn_id='mysql_default',
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Is it better to use 'google_cloud_default' as conn_id here because we've already unified all the GCP conn_id?
Reference: #4818

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It is better to use gcp_conn_id as parameter name here because it is written in integration recommendations. In last night, I started working on unification
PolideaInternal#201

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for the input, this is fixed!

database=None,
delegate_to=None,
replace=False,
*args,
**kwargs):
super(BigQueryToMySqlOperator, self).__init__(*args, **kwargs)
self.dataset_id = dataset_id
self.table_id = table_id
self.selected_fields = selected_fields
self.bigquery_conn_id = bigquery_conn_id
self.mysql_conn_id = mysql_conn_id
self.database = database
self.mysql_table = mysql_table
self.replace = replace
self.delegate_to = delegate_to

def _bq_get_data(self):
self.log.info('Fetching Data from:')
self.log.info('Dataset: %s ; Table: %s',
self.dataset_id, self.table_id)

hook = BigQueryHook(bigquery_conn_id=self.bigquery_conn_id,
delegate_to=self.delegate_to)

conn = hook.get_conn()
cursor = conn.cursor()
i = 0
while True:
# Max results is set to 1000 because bq job has an hardcoded limit to 1300.
response = cursor.get_tabledata(dataset_id=self.dataset_id,
table_id=self.table_id,
max_results=1000,
Copy link
Member

@potiuk potiuk Aug 5, 2019

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

NIT: It would be nice to have it extracted as constant or even better provide it as parameters with default. I imagine you want to run it in smaller batches - for various reasons.

selected_fields=self.selected_fields,
start_index=i * 1000)

if 'rows' in response:
rows = response['rows']
else:
self.log.info('Job Finished')
return

self.log.info('Total Extracted rows: %s', len(rows) + i * 1000)

table_data = []
for dict_row in rows:
single_row = []
for fields in dict_row['f']:
single_row.append(fields['v'])
table_data.append(single_row)

yield table_data
i += 1

def execute(self, context):
mysql_hook = MySqlHook(schema=self.database, mysql_conn_id=self.mysql_conn_id)
for rows in self._bq_get_data():
mysql_hook.insert_rows(self.mysql_table, rows, replace=self.replace)
27 changes: 27 additions & 0 deletions tests/contrib/operators/test_bigquery_operator.py
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@
from airflow.contrib.operators.bigquery_to_bigquery import \
BigQueryToBigQueryOperator
from airflow.contrib.operators.bigquery_to_gcs import BigQueryToCloudStorageOperator
from airflow.contrib.operators.bigquery_to_mysql_operator import BigQueryToMySqlOperator
from airflow.exceptions import AirflowException
from airflow.models import DAG, TaskFail, TaskInstance
from airflow.settings import Session
Expand Down Expand Up @@ -514,3 +515,29 @@ def test_execute(self, mock_hook):
print_header=print_header,
labels=labels
)


class BigQueryToMySqlOperatorTest(unittest.TestCase):
@mock.patch('airflow.contrib.operators.bigquery_to_mysql_operator.BigQueryHook')
def test_execute_good_request_to_bq(self, mock_hook):
destination_table = 'table'
operator = BigQueryToMySqlOperator(
task_id=TASK_ID,
dataset_id=TEST_DATASET,
table_id=TEST_TABLE_ID,
mysql_table=destination_table,
replace=False,
)

operator.execute(None)
mock_hook.return_value \
.get_conn.return_value \
.cursor.return_value \
.get_tabledata \
.assert_called_once_with(
dataset_id=TEST_DATASET,
table_id=TEST_TABLE_ID,
max_results=1000,
selected_fields=None,
start_index=0
)