Skip to content

Commit

Permalink
Adding SnowflakeOperator howto-documentation and example DAG (#11975)
Browse files Browse the repository at this point in the history
closes #11921
  • Loading branch information
davido912 committed Oct 31, 2020
1 parent 21350aa commit d363adb
Show file tree
Hide file tree
Showing 4 changed files with 199 additions and 35 deletions.
105 changes: 72 additions & 33 deletions airflow/providers/snowflake/example_dags/example_snowflake.py
Original file line number Diff line number Diff line change
Expand Up @@ -18,29 +18,33 @@
"""
Example use of Snowflake related operators.
"""
import os

from airflow import DAG
from airflow.providers.snowflake.operators.snowflake import SnowflakeOperator
from airflow.providers.snowflake.transfers.s3_to_snowflake import S3ToSnowflakeOperator
from airflow.providers.snowflake.transfers.snowflake_to_slack import SnowflakeToSlackOperator
from airflow.utils.dates import days_ago

SNOWFLAKE_CONN_ID = os.environ.get('SNOWFLAKE_CONN_ID', 'snowflake_default')
SLACK_CONN_ID = os.environ.get('SLACK_CONN_ID', 'slack_default')
SNOWFLAKE_CONN_ID = 'my_snowflake_conn'
SLACK_CONN_ID = 'my_slack_conn'
# TODO: should be able to rely on connection's schema, but currently param required by S3ToSnowflakeTransfer
SNOWFLAKE_SCHEMA = os.environ.get('SNOWFLAKE_SCHEMA', 'public')
SNOWFLAKE_STAGE = os.environ.get('SNOWFLAKE_STAGE', 'airflow')
SNOWFLAKE_SAMPLE_TABLE = os.environ.get('SNOWFLAKE_SAMPLE_TABLE', 'snowflake_sample_data.tpch_sf001.orders')
SNOWFLAKE_LOAD_TABLE = os.environ.get('SNOWFLAKE_LOAD_TABLE', 'airflow_example')
SNOWFLAKE_LOAD_JSON_PATH = os.environ.get('SNOWFLAKE_LOAD_PATH', 'example.json')

SNOWFLAKE_SELECT_SQL = f"SELECT * FROM {SNOWFLAKE_SAMPLE_TABLE} LIMIT 100;"
SNOWFLAKE_SLACK_SQL = f"SELECT O_ORDERKEY, O_CUSTKEY, O_ORDERSTATUS FROM {SNOWFLAKE_SAMPLE_TABLE} LIMIT 10;"
SNOWFLAKE_SCHEMA = 'schema_name'
SNOWFLAKE_STAGE = 'stage_name'
SNOWFLAKE_WAREHOUSE = 'warehouse_name'
SNOWFLAKE_DATABASE = 'database_name'
SNOWFLAKE_ROLE = 'role_name'
SNOWFLAKE_SAMPLE_TABLE = 'sample_table'
S3_FILE_PATH = '</path/to/file/sample_file.csv'

# SQL commands
CREATE_TABLE_SQL_STRING = (
f"CREATE OR REPLACE TRANSIENT TABLE {SNOWFLAKE_SAMPLE_TABLE} (name VARCHAR(250), id INT);"
)
SQL_INSERT_STATEMENT = f"INSERT INTO {SNOWFLAKE_SAMPLE_TABLE} VALUES ('name', %(id)s)"
SQL_LIST = [SQL_INSERT_STATEMENT % {"id": n} for n in range(0, 10)]
SNOWFLAKE_SLACK_SQL = f"SELECT name, id FROM {SNOWFLAKE_SAMPLE_TABLE} LIMIT 10;"
SNOWFLAKE_SLACK_MESSAGE = (
"Results in an ASCII table:\n```{{ results_df | tabulate(tablefmt='pretty', headers='keys') }}```"
)
SNOWFLAKE_CREATE_TABLE_SQL = f"CREATE TRANSIENT TABLE IF NOT EXISTS {SNOWFLAKE_LOAD_TABLE}(data VARIANT);"

default_args = {
'owner': 'airflow',
Expand All @@ -53,40 +57,75 @@
tags=['example'],
)

select = SnowflakeOperator(
task_id='select',
snowflake_conn_id=SNOWFLAKE_CONN_ID,
sql=SNOWFLAKE_SELECT_SQL,
dag=dag,
)
# [START howto_operator_snowflake]

slack_report = SnowflakeToSlackOperator(
task_id="slack_report",
sql=SNOWFLAKE_SLACK_SQL,
slack_message=SNOWFLAKE_SLACK_MESSAGE,
snowflake_conn_id=SNOWFLAKE_CONN_ID,
slack_conn_id=SLACK_CONN_ID,
snowflake_op_sql_str = SnowflakeOperator(
task_id='snowflake_op_sql_str',
dag=dag,
snowflake_conn_id=SNOWFLAKE_CONN_ID,
sql=CREATE_TABLE_SQL_STRING,
warehouse=SNOWFLAKE_WAREHOUSE,
database=SNOWFLAKE_DATABASE,
schema=SNOWFLAKE_SCHEMA,
role=SNOWFLAKE_ROLE,
)

create_table = SnowflakeOperator(
task_id='create_table',
snowflake_op_with_params = SnowflakeOperator(
task_id='snowflake_op_with_params',
dag=dag,
snowflake_conn_id=SNOWFLAKE_CONN_ID,
sql=SNOWFLAKE_CREATE_TABLE_SQL,
sql=SQL_INSERT_STATEMENT,
parameters={"id": 56},
warehouse=SNOWFLAKE_WAREHOUSE,
database=SNOWFLAKE_DATABASE,
schema=SNOWFLAKE_SCHEMA,
role=SNOWFLAKE_ROLE,
)

snowflake_op_sql_list = SnowflakeOperator(
task_id='snowflake_op_sql_list', dag=dag, snowflake_conn_id=SNOWFLAKE_CONN_ID, sql=SQL_LIST
)

snowflake_op_template_file = SnowflakeOperator(
task_id='snowflake_op_template_file',
dag=dag,
snowflake_conn_id=SNOWFLAKE_CONN_ID,
sql='/path/to/sql/<filename>.sql',
)

# [END howto_operator_snowflake]

# [START howto_operator_s3_to_snowflake]

copy_into_table = S3ToSnowflakeOperator(
task_id='copy_into_table',
snowflake_conn_id=SNOWFLAKE_CONN_ID,
s3_keys=[SNOWFLAKE_LOAD_JSON_PATH],
table=SNOWFLAKE_LOAD_TABLE,
s3_keys=[S3_FILE_PATH],
table=SNOWFLAKE_SAMPLE_TABLE,
schema=SNOWFLAKE_SCHEMA,
stage=SNOWFLAKE_STAGE,
file_format="(type = 'JSON')",
file_format="(type = 'CSV'," "field_delimiter = ';')",
dag=dag,
)

select >> slack_report
create_table >> copy_into_table
# [END howto_operator_s3_to_snowflake]

# [START howto_operator_snowflake_to_slack]

slack_report = SnowflakeToSlackOperator(
task_id="slack_report",
sql=SNOWFLAKE_SLACK_SQL,
slack_message=SNOWFLAKE_SLACK_MESSAGE,
snowflake_conn_id=SNOWFLAKE_CONN_ID,
slack_conn_id=SLACK_CONN_ID,
dag=dag,
)

# [END howto_operator_snowflake_to_slack]

snowflake_op_sql_str >> [
snowflake_op_with_params,
snowflake_op_sql_list,
snowflake_op_template_file,
copy_into_table,
] >> slack_report
1 change: 1 addition & 0 deletions docs/howto/operator/index.rst
Original file line number Diff line number Diff line change
Expand Up @@ -42,5 +42,6 @@ determine what actually executes when your DAG runs.
microsoft/index
papermill
python
snowflake
external_task_sensor
yandexcloud
124 changes: 124 additions & 0 deletions docs/howto/operator/snowflake.rst
Original file line number Diff line number Diff line change
@@ -0,0 +1,124 @@
.. 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.
.. _howto/operator:SnowflakeOperator:

SnowflakeOperator
=================

Use the :class:`SnowflakeOperator <airflow.providers.snowflake.operators.snowflake>` to execute
SQL commands in a `Snowflake <https://docs.snowflake.com/en/>`__ database.


Using the Operator
^^^^^^^^^^^^^^^^^^

Use the ``snowflake_conn_id`` argument to connect to your Snowflake instance where
the connection metadata is structured as follows:

.. list-table:: Snowflake Airflow Connection Metadata
:widths: 25 25
:header-rows: 1

* - Parameter
- Input
* - Login: string
- Snowflake user name
* - Password: string
- Password for Snowflake user
* - Schema: string
- Set schema to execute SQL operations on by default
* - Extra: dictionary
- ``warehouse``, ``account``, ``database``, ``region``, ``role``, ``authenticator``

An example usage of the SnowflakeOperator is as follows:

.. exampleinclude:: /../airflow/providers/snowflake/example_dags/example_snowflake.py
:language: python
:start-after: [START howto_operator_snowflake]
:end-before: [END howto_operator_snowflake]

.. note::

Parameters that can be passed onto the operator will be given priority over the parameters already given
in the Airflow connection metadata (such as ``schema``, ``role``, ``database`` and so forth).


.. _howto/operator:S3ToSnowflakeOperator:

S3ToSnowflakeOperator
=====================

Use the :class:`S3ToSnowflakeOperator <airflow.providers.snowflake.transfers.s3_to_snowflake>` to load data stored in `AWS S3 <https://aws.amazon.com/s3/>`__
to a Snowflake table.


Using the Operator
^^^^^^^^^^^^^^^^^^

Similarly to the :class:`SnowflakeOperator <airflow.providers.snowflake.operators.snowflake>`, use the ``snowflake_conn_id`` and
the additional relevant parameters to establish connection with your Snowflake instance.
This operator will allow loading of one or more named files from a specific Snowflake stage (predefined S3 path). In order to do so
pass the relevant file names to the ``s3_keys`` parameter and the relevant Snowflake stage to the ``stage`` parameter.
``file_format`` can be used to either reference an already existing Snowflake file format or a custom string that defines
a file format (see `docs <https://docs.snowflake.com/en/sql-reference/sql/create-file-format.html>`__).

An example usage of the S3ToSnowflakeOperator is as follows: #TODO: currently forces usage of schema parameter

.. exampleinclude:: /../airflow/providers/snowflake/example_dags/example_snowflake.py
:language: python
:start-after: [START howto_operator_s3_to_snowflake]
:end-before: [END howto_operator_s3_to_snowflake]


.. _howto/operator:SnowflakeToSlackOperator:

SnowflakeToSlackOperator
========================

Use the :class:`SnowflakeToSlackOperator <airflow.providers.snowflake.transfers.snowflake_to_slack>` to post messages to predefined Slack
channels.

.. list-table:: Slack Webhook Airflow Connection Metadata
:widths: 25 25
:header-rows: 1

* - Parameter
- Input
* - Host: string
- Slack Webhook URL
* - Extra: dictionary
- ``webhook_token`` (optional)



Using the Operator
^^^^^^^^^^^^^^^^^^

Similarly to the :class:`SnowflakeOperator <airflow.providers.snowflake.operators.snowflake>`, use the ``snowflake_conn_id`` and
the additional relevant parameters to establish connection with your Snowflake instance.
This operator will execute a custom query on a selected Snowflake table and publish a Slack message that can be formatted
and contain the resulting dataset (e.g. ASCII formatted dataframe).

An example usage of the SnowflakeToSlackOperator is as follows:

.. exampleinclude:: /../airflow/providers/snowflake/example_dags/example_snowflake.py
:language: python
:start-after: [START howto_operator_snowflake_to_slack]
:end-before: [END howto_operator_snowflake_to_slack]
4 changes: 2 additions & 2 deletions docs/operators-and-hooks-ref.rst
Original file line number Diff line number Diff line change
Expand Up @@ -626,7 +626,7 @@ These integrations allow you to copy data from/to Amazon Web Services.

* - `Amazon Simple Storage Service (S3) <https://aws.amazon.com/s3/>`_
- `Snowflake <https://snowflake.com/>`__
-
- :doc:`How to use <howto/operator/snowflake>`
- :mod:`airflow.providers.snowflake.transfers.s3_to_snowflake`

* - `Amazon Simple Storage Service (S3) <https://aws.amazon.com/s3/>`_
Expand Down Expand Up @@ -1357,7 +1357,7 @@ These integrations allow you to perform various operations within various servic
-

* - `Snowflake <https://www.snowflake.com/>`__
-
- :doc:`How to use <howto/operator/snowflake>`
- :mod:`airflow.providers.snowflake.hooks.snowflake`
- :mod:`airflow.providers.snowflake.operators.snowflake`,
:mod:`airflow.providers.snowflake.transfers.snowflake_to_slack`
Expand Down

0 comments on commit d363adb

Please sign in to comment.