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-6296] add ODBC hook & deprecation warning for pymssql #6850

Merged
merged 1 commit into from Jan 19, 2020
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
2 changes: 2 additions & 0 deletions Dockerfile
Expand Up @@ -74,6 +74,8 @@ RUN curl -L https://deb.nodesource.com/setup_10.x | bash - \
rsync \
sasl2-bin \
sudo \
unixodbc \
unixodbc-dev \
yarn \
&& apt-get autoremove -yqq --purge \
&& apt-get clean \
Expand Down
36 changes: 35 additions & 1 deletion airflow/hooks/mssql_hook.py
Expand Up @@ -16,14 +16,38 @@
# KIND, either express or implied. See the License for the
# specific language governing permissions and limitations
# under the License.
"""
This module is deprecated due to the discontinuation of the pymssql project.
See https://github.com/pymssql/pymssql/issues/668.
Support for pymssql will be removed in Airflow 2.0.
Please use :mod:`~airflow.providers.odbc.hooks.odbc`.
"""

import warnings

import pymssql

from airflow.hooks.dbapi_hook import DbApiHook

warnings.warn(
(
"This module is deprecated due to the discontinuation of the pymssql project.\n"
"See https://github.com/pymssql/pymssql/issues/668.\n"
"Support for pymssql will be removed in airflow 2.0.\n"
"Please use `airflow.providers.odbc.hooks.odbc`.\n"
),
DeprecationWarning,
stacklevel=2,
)


class MsSqlHook(DbApiHook):
"""
.. warning::

This class is deprecated.
Please use :py:class:`~airflow.providers.odbc.hooks.odbc.OdbcHook`.

Interact with Microsoft SQL Server.
"""

Expand All @@ -32,6 +56,15 @@ class MsSqlHook(DbApiHook):
supports_autocommit = True

def __init__(self, *args, **kwargs):
warnings.warn(
(
"This class is deprecated and will be removed in Airflow 2.0.\n"
"pymssql is discontinued. See https://github.com/pymssql/pymssql/issues/668.\n"
"Please use `airflow.providers.odbc.hooks.odbc.OdbcHook`"
),
DeprecationWarning,
stacklevel=2,
)
super().__init__(*args, **kwargs)
self.schema = kwargs.pop("schema", None)

Expand All @@ -45,7 +78,8 @@ def get_conn(self):
user=conn.login,
password=conn.password,
database=self.schema or conn.schema,
port=conn.port)
port=conn.port,
)
return conn

def set_autocommit(self, conn, autocommit):
Expand Down
3 changes: 3 additions & 0 deletions airflow/models/connection.py
Expand Up @@ -265,6 +265,9 @@ def get_hook(self):
elif self.conn_type == 'mssql':
from airflow.hooks.mssql_hook import MsSqlHook
return MsSqlHook(mssql_conn_id=self.conn_id)
elif self.conn_type == 'odbc':
from airflow.providers.odbc.hooks.odbc import OdbcHook
return OdbcHook(odbc_conn_id=self.conn_id)
elif self.conn_type == 'oracle':
from airflow.hooks.oracle_hook import OracleHook
return OracleHook(oracle_conn_id=self.conn_id)
Expand Down
31 changes: 27 additions & 4 deletions airflow/operators/mssql_operator.py
Expand Up @@ -18,15 +18,22 @@
# under the License.
from typing import Iterable, Mapping, Optional, Union

from airflow import AirflowException
from airflow.hooks.mssql_hook import MsSqlHook
from airflow.models import BaseOperator
from airflow.providers.odbc.hooks.odbc import OdbcHook
from airflow.utils.decorators import apply_defaults


class MsSqlOperator(BaseOperator):
"""
Executes sql code in a specific Microsoft SQL database

This operator may use one of two hooks, depending on the ``conn_type`` of the connection.

If conn_type is ``'odbc'``, then :py:class:`~airflow.providers.odbc.hooks.odbc.OdbcHook`
is used. Otherwise, :py:class:`~airflow.hooks.mssql_hook.MsSqlHook` is used.

:param sql: the sql code to be executed
:type sql: str or string pointing to a template file with .sql
extension. (templated)
Expand Down Expand Up @@ -60,10 +67,26 @@ def __init__(
self.parameters = parameters
self.autocommit = autocommit
self.database = database
self._hook = None

def get_hook(self):
"""
Will retrieve hook as determined by Connection.

If conn_type is ``'odbc'``, will use
:py:class:`~airflow.providers.odbc.hooks.odbc.OdbcHook`.
Otherwise, :py:class:`~airflow.hooks.mssql_hook.MsSqlHook` will be used.
"""
if not self._hook:
conn = MsSqlHook.get_connection(conn_id=self.mssql_conn_id)
try:
self._hook = conn.get_hook() # type: Union[MsSqlHook, OdbcHook]
self._hook.schema = self.database
except AirflowException:
self._hook = MsSqlHook(mssql_conn_id=self.mssql_conn_id, schema=self.database)
return self._hook

def execute(self, context):
self.log.info('Executing: %s', self.sql)
hook = MsSqlHook(mssql_conn_id=self.mssql_conn_id,
schema=self.database)
hook.run(self.sql, autocommit=self.autocommit,
parameters=self.parameters)
hook = self.get_hook()
hook.run(sql=self.sql, autocommit=self.autocommit, parameters=self.parameters)
16 changes: 16 additions & 0 deletions airflow/providers/odbc/__init__.py
@@ -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/odbc/hooks/__init__.py
@@ -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.
220 changes: 220 additions & 0 deletions airflow/providers/odbc/hooks/odbc.py
@@ -0,0 +1,220 @@
# 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 ODBC hook.
"""
from typing import Optional
from urllib.parse import quote_plus

import pyodbc

from airflow.hooks.dbapi_hook import DbApiHook
from airflow.utils.helpers import merge_dicts


class OdbcHook(DbApiHook):
"""
Interact with odbc data sources using pyodbc.

See :ref:`howto/connection/odbc` for full documentation.
"""

DEFAULT_SQLALCHEMY_SCHEME = 'mssql+pyodbc'
conn_name_attr = 'odbc_conn_id'
default_conn_name = 'odbc_default'
supports_autocommit = True

def __init__(
self,
*args,
database: Optional[str] = None,
driver: Optional[str] = None,
dsn: Optional[str] = None,
connect_kwargs: Optional[dict] = None,
sqlalchemy_scheme: Optional[str] = None,
**kwargs,
):
"""
:param args: passed to DbApiHook
:param database: database to use -- overrides connection ``schema``
:param driver: name of driver or path to driver. overrides driver supplied in connection ``extra``
:param dsn: name of DSN to use. overrides DSN supplied in connection ``extra``
:param connect_kwargs: keyword arguments passed to ``pyodbc.connect``
:param sqlalchemy_scheme: Scheme sqlalchemy connection. Default is ``mssql+pyodbc`` Only used for
``get_sqlalchemy_engine`` and ``get_sqlalchemy_connection`` methods.
:param kwargs: passed to DbApiHook
"""
super().__init__(*args, **kwargs)
self._database = database
self._driver = driver
self._dsn = dsn
self._conn_str = None
self._sqlalchemy_scheme = sqlalchemy_scheme
self._connection = None
self._connect_kwargs = connect_kwargs

@property
def connection(self):
"""
``airflow.Connection`` object with connection id ``odbc_conn_id``
"""
if not self._connection:
self._connection = self.get_connection(getattr(self, self.conn_name_attr))
return self._connection

@property
def database(self):
"""
Database provided in init if exists; otherwise, ``schema`` from ``Connection`` object.
"""
return self._database or self.connection.schema

@property
def sqlalchemy_scheme(self):
"""
Database provided in init if exists; otherwise, ``schema`` from ``Connection`` object.
"""
return (
self._sqlalchemy_scheme or
self.connection_extra_lower.get('sqlalchemy_scheme') or
self.DEFAULT_SQLALCHEMY_SCHEME
)

@property
def connection_extra_lower(self):
"""
``connection.extra_dejson`` but where keys are converted to lower case.

This is used internally for case-insensitive access of odbc params.
"""
return {k.lower(): v for k, v in self.connection.extra_dejson.items()}

@property
def driver(self):
"""
Driver from init param if given; else try to find one in connection extra.
"""
if not self._driver:
driver = self.connection_extra_lower.get('driver')
if driver:
self._driver = driver
return self._driver and self._driver.strip().lstrip('{').rstrip('}').strip()
dstandish marked this conversation as resolved.
Show resolved Hide resolved

@property
def dsn(self):
"""
DSN from init param if given; else try to find one in connection extra.
"""
if not self._dsn:
dsn = self.connection_extra_lower.get('dsn')
if dsn:
self._dsn = dsn.strip()
return self._dsn

@property
def odbc_connection_string(self):
"""
ODBC connection string
We build connection string instead of using ``pyodbc.connect`` params because, for example, there is
no param representing ``ApplicationIntent=ReadOnly``. Any key-value pairs provided in
``Connection.extra`` will be added to the connection string.
"""
if not self._conn_str:
conn_str = ''
if self.driver:
conn_str += f"DRIVER={{{self.driver}}};"
if self.dsn:
conn_str += f"DSN={self.dsn};"
if self.connection.host:
conn_str += f"SERVER={self.connection.host};"
database = self.database or self.connection.schema
if database:
conn_str += f"DATABASE={database};"
if self.connection.login:
conn_str += f"UID={self.connection.login};"
if self.connection.password:
conn_str += f"PWD={self.connection.password};"
if self.connection.port:
f"PORT={self.connection.port};"

extra_exclude = {'driver', 'dsn', 'connect_kwargs', 'sqlalchemy_scheme'}
extra_params = {
k: v
for k, v in self.connection.extra_dejson.items()
if not k.lower() in extra_exclude
}
for k, v in extra_params.items():
conn_str += f"{k}={v};"

self._conn_str = conn_str
return self._conn_str

@property
def connect_kwargs(self):
"""
Returns effective kwargs to be passed to ``pyodbc.connect`` after merging between conn extra,
``connect_kwargs`` and hook init.

Hook ``connect_kwargs`` precedes ``connect_kwargs`` from conn extra.

String values for 'true' and 'false' are converted to bool type.

If ``attrs_before`` provided, keys and values are converted to int, as required by pyodbc.
"""

def clean_bool(val): # pylint: disable=inconsistent-return-statements
if hasattr(val, 'lower'):
if val.lower() == 'true':
return True
elif val.lower() == 'false':
return False
else:
return val

conn_connect_kwargs = self.connection_extra_lower.get('connect_kwargs', {})
hook_connect_kwargs = self._connect_kwargs or {}
merged_connect_kwargs = merge_dicts(conn_connect_kwargs, hook_connect_kwargs)

if 'attrs_before' in merged_connect_kwargs:
merged_connect_kwargs['attrs_before'] = {
int(k): int(v) for k, v in merged_connect_kwargs['attrs_before'].items()
}

return {k: clean_bool(v) for k, v in merged_connect_kwargs.items()}

def get_conn(self) -> pyodbc.Connection:
dstandish marked this conversation as resolved.
Show resolved Hide resolved
dstandish marked this conversation as resolved.
Show resolved Hide resolved
"""
Returns a pyodbc connection object.
"""
conn = pyodbc.connect(self.odbc_connection_string, **self.connect_kwargs)
return conn

def get_uri(self):
"""
URI invoked in :py:meth:`~airflow.hooks.dbapi_hook.DbApiHook.get_sqlalchemy_engine` method
"""
quoted_conn_str = quote_plus(self.odbc_connection_string)
uri = f"{self.sqlalchemy_scheme}:///?odbc_connect={quoted_conn_str}"
return uri

def get_sqlalchemy_connection(self, connect_kwargs=None, engine_kwargs=None):
"""
Sqlalchemy connection object
"""
engine = self.get_sqlalchemy_engine(engine_kwargs=engine_kwargs)
cnx = engine.connect(**(connect_kwargs or {}))
return cnx