Skip to content

Commit

Permalink
[AIRFLOW-5816] Add S3 to snowflake operator (#6469)
Browse files Browse the repository at this point in the history
- move Snowflake from contrib to providers
- fix pylint issues

Co-authored-by: Felix Uellendall <feluelle@users.noreply.github.com>
  • Loading branch information
shaikshakeel and feluelle committed Jan 21, 2020
1 parent af14d92 commit 17af3be
Show file tree
Hide file tree
Showing 17 changed files with 574 additions and 177 deletions.
121 changes: 9 additions & 112 deletions airflow/contrib/hooks/snowflake_hook.py
Original file line number Diff line number Diff line change
Expand Up @@ -17,118 +17,15 @@
# specific language governing permissions and limitations
# under the License.

import snowflake.connector
from cryptography.hazmat.backends import default_backend
from cryptography.hazmat.primitives import serialization
"""This module is deprecated. Please use `airflow.providers.snowflake.hooks.snowflake`."""

from airflow.hooks.dbapi_hook import DbApiHook
import warnings

# pylint: disable=unused-import
from airflow.providers.snowflake.hooks.snowflake import SnowflakeHook # noqa

class SnowflakeHook(DbApiHook):
"""
Interact with Snowflake.
get_sqlalchemy_engine() depends on snowflake-sqlalchemy
"""

conn_name_attr = 'snowflake_conn_id'
default_conn_name = 'snowflake_default'
supports_autocommit = True

def __init__(self, *args, **kwargs):
super().__init__(*args, **kwargs)
self.account = kwargs.pop("account", None)
self.warehouse = kwargs.pop("warehouse", None)
self.database = kwargs.pop("database", None)
self.region = kwargs.pop("region", None)
self.role = kwargs.pop("role", None)
self.schema = kwargs.pop("schema", None)

def _get_conn_params(self):
"""
one method to fetch connection params as a dict
used in get_uri() and get_connection()
"""
conn = self.get_connection(self.snowflake_conn_id)
account = conn.extra_dejson.get('account', None)
warehouse = conn.extra_dejson.get('warehouse', None)
database = conn.extra_dejson.get('database', None)
region = conn.extra_dejson.get("region", None)
role = conn.extra_dejson.get('role', None)

conn_config = {
"user": conn.login,
"password": conn.password or '',
"schema": self.schema or conn.schema or '',
"database": self.database or database or '',
"account": self.account or account or '',
"warehouse": self.warehouse or warehouse or '',
"region": self.region or region or '',
"role": self.role or role or ''

}

"""
If private_key_file is specified in the extra json, load the contents of the file as a private
key and specify that in the connection configuration. The connection password then becomes the
passphrase for the private key. If your private key file is not encrypted (not recommended), then
leave the password empty.
"""
private_key_file = conn.extra_dejson.get('private_key_file', None)
if private_key_file:
with open(private_key_file, "rb") as key:
passphrase = None
if conn.password:
passphrase = conn.password.strip().encode()

p_key = serialization.load_pem_private_key(
key.read(),
password=passphrase,
backend=default_backend()
)

pkb = p_key.private_bytes(encoding=serialization.Encoding.DER,
format=serialization.PrivateFormat.PKCS8,
encryption_algorithm=serialization.NoEncryption())

conn_config['private_key'] = pkb
conn_config.pop('password', None)

return conn_config

def get_uri(self):
"""
override DbApiHook get_uri method for get_sqlalchemy_engine()
"""
conn_config = self._get_conn_params()
uri = 'snowflake://{user}:{password}@{account}/{database}/'
uri += '{schema}?warehouse={warehouse}&role={role}'
return uri.format(**conn_config)

def get_conn(self):
"""
Returns a snowflake.connection object
"""
conn_config = self._get_conn_params()
conn = snowflake.connector.connect(**conn_config)
return conn

def _get_aws_credentials(self):
"""
returns aws_access_key_id, aws_secret_access_key
from extra
intended to be used by external import and export statements
"""
if self.snowflake_conn_id:
connection_object = self.get_connection(self.snowflake_conn_id)
if 'aws_secret_access_key' in connection_object.extra_dejson:
aws_access_key_id = connection_object.extra_dejson.get(
'aws_access_key_id')
aws_secret_access_key = connection_object.extra_dejson.get(
'aws_secret_access_key')
return aws_access_key_id, aws_secret_access_key

def set_autocommit(self, conn, autocommit):
conn.autocommit(autocommit)
warnings.warn(
"This module is deprecated. Please use `airflow.providers.snowflake.hooks.snowflake`.",
DeprecationWarning,
stacklevel=2,
)
64 changes: 8 additions & 56 deletions airflow/contrib/operators/snowflake_operator.py
Original file line number Diff line number Diff line change
Expand Up @@ -16,62 +16,14 @@
# KIND, either express or implied. See the License for the
# specific language governing permissions and limitations
# under the License.
from airflow.contrib.hooks.snowflake_hook import SnowflakeHook
from airflow.models import BaseOperator
from airflow.utils.decorators import apply_defaults
"""This module is deprecated. Please use `airflow.providers.snowflake.operators.snowflake`."""

import warnings

class SnowflakeOperator(BaseOperator):
"""
Executes sql code in a Snowflake database
# pylint: disable=unused-import
from airflow.providers.snowflake.operators.snowflake import SnowflakeOperator # noqa

:param snowflake_conn_id: reference to specific snowflake connection id
:type snowflake_conn_id: str
:param sql: the sql code to be executed. (templated)
:type sql: Can receive a str representing a sql statement,
a list of str (sql statements), or reference to a template file.
Template reference are recognized by str ending in '.sql'
:param warehouse: name of warehouse (will overwrite any warehouse
defined in the connection's extra JSON)
:type warehouse: str
:param database: name of database (will overwrite database defined
in connection)
:type database: str
:param schema: name of schema (will overwrite schema defined in
connection)
:type schema: str
:param role: name of role (will overwrite any role defined in
connection's extra JSON)
"""

template_fields = ('sql',)
template_ext = ('.sql',)
ui_color = '#ededed'

@apply_defaults
def __init__(
self, sql, snowflake_conn_id='snowflake_default', parameters=None,
autocommit=True, warehouse=None, database=None, role=None,
schema=None, *args, **kwargs):
super().__init__(*args, **kwargs)
self.snowflake_conn_id = snowflake_conn_id
self.sql = sql
self.autocommit = autocommit
self.parameters = parameters
self.warehouse = warehouse
self.database = database
self.role = role
self.schema = schema

def get_hook(self):
return SnowflakeHook(snowflake_conn_id=self.snowflake_conn_id,
warehouse=self.warehouse, database=self.database,
role=self.role, schema=self.schema)

def execute(self, context):
self.log.info('Executing: %s', self.sql)
hook = self.get_hook()
hook.run(
self.sql,
autocommit=self.autocommit,
parameters=self.parameters)
warnings.warn(
"This module is deprecated. Please use `airflow.providers.snowflake.operators.snowflake`.",
DeprecationWarning, stacklevel=2
)
16 changes: 16 additions & 0 deletions airflow/providers/snowflake/__init__.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,16 @@
# 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.
16 changes: 16 additions & 0 deletions airflow/providers/snowflake/hooks/__init__.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,16 @@
# 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.
132 changes: 132 additions & 0 deletions airflow/providers/snowflake/hooks/snowflake.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,132 @@
# -*- 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.

from cryptography.hazmat.backends import default_backend
from cryptography.hazmat.primitives import serialization
# pylint: disable=no-name-in-module
from snowflake import connector

from airflow.hooks.dbapi_hook import DbApiHook


class SnowflakeHook(DbApiHook):
"""
Interact with Snowflake.
get_sqlalchemy_engine() depends on snowflake-sqlalchemy
"""
default_conn_name = 'snowflake_default'
supports_autocommit = True

def __init__(self, *args, **kwargs):
super().__init__(*args, **kwargs)
self.account = kwargs.pop("account", None)
self.warehouse = kwargs.pop("warehouse", None)
self.database = kwargs.pop("database", None)
self.region = kwargs.pop("region", None)
self.role = kwargs.pop("role", None)
self.schema = kwargs.pop("schema", None)
self.snowflake_conn_id = 'snowflake_conn_id'

def _get_conn_params(self):
"""
one method to fetch connection params as a dict
used in get_uri() and get_connection()
"""
conn = self.get_connection(self.snowflake_conn_id)
account = conn.extra_dejson.get('account', '')
warehouse = conn.extra_dejson.get('warehouse', '')
database = conn.extra_dejson.get('database', '')
region = conn.extra_dejson.get("region", '')
role = conn.extra_dejson.get('role', '')
schema = conn.schema or ''

conn_config = {
"user": conn.login,
"password": conn.password or '',
"schema": self.schema or schema,
"database": self.database or database,
"account": self.account or account,
"warehouse": self.warehouse or warehouse,
"region": self.region or region,
"role": self.role or role

}

# If private_key_file is specified in the extra json, load the contents of the file as a private
# key and specify that in the connection configuration. The connection password then becomes the
# passphrase for the private key. If your private key file is not encrypted (not recommended), then
# leave the password empty.

private_key_file = conn.extra_dejson.get('private_key_file', None)
if private_key_file:
with open(private_key_file, "rb") as key:
passphrase = None
if conn.password:
passphrase = conn.password.strip().encode()

p_key = serialization.load_pem_private_key(
key.read(),
password=passphrase,
backend=default_backend()
)

pkb = p_key.private_bytes(encoding=serialization.Encoding.DER,
format=serialization.PrivateFormat.PKCS8,
encryption_algorithm=serialization.NoEncryption())

conn_config['private_key'] = pkb
conn_config.pop('password', None)

return conn_config

def get_uri(self):
"""
override DbApiHook get_uri method for get_sqlalchemy_engine()
"""
conn_config = self._get_conn_params()
uri = 'snowflake://{user}:{password}@{account}/{database}/{schema}' \
'?warehouse={warehouse}&role={role}'
return uri.format(**conn_config)

def get_conn(self):
"""
Returns a snowflake.connection object
"""
conn_config = self._get_conn_params()
conn = connector.connect(**conn_config)
return conn

def _get_aws_credentials(self):
"""
returns aws_access_key_id, aws_secret_access_key
from extra
intended to be used by external import and export statements
"""
if self.snowflake_conn_id:
connection_object = self.get_connection(self.snowflake_conn_id)
if 'aws_secret_access_key' in connection_object.extra_dejson:
aws_access_key_id = connection_object.extra_dejson.get(
'aws_access_key_id')
aws_secret_access_key = connection_object.extra_dejson.get(
'aws_secret_access_key')
return aws_access_key_id, aws_secret_access_key

def set_autocommit(self, conn, autocommit):
conn.autocommit(autocommit)
16 changes: 16 additions & 0 deletions airflow/providers/snowflake/operators/__init__.py
Original file line number Diff line number Diff line change
@@ -0,0 +1,16 @@
# 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.
Loading

0 comments on commit 17af3be

Please sign in to comment.