From cf441990293971f58882b9b7427c0428d7da4f4d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?J=C3=A1n=20Ko=C5=A1=C4=8Do?= <3k.stanley@gmail.com> Date: Fri, 10 Feb 2017 11:30:34 +0700 Subject: [PATCH] [AIRFLOW-858] - Configurable database name for DB operators --- airflow/hooks/mssql_hook.py | 10 ++++-- airflow/hooks/mysql_hook.py | 15 +++++---- airflow/hooks/postgres_hook.py | 10 ++++-- airflow/operators/mssql_operator.py | 11 +++++-- airflow/operators/mysql_operator.py | 8 +++-- airflow/operators/postgres_operator.py | 7 ++++- tests/operators/operators.py | 43 ++++++++++++++++++++++++++ 7 files changed, 86 insertions(+), 18 deletions(-) diff --git a/airflow/hooks/mssql_hook.py b/airflow/hooks/mssql_hook.py index 14509677c6dee..99a4c82f7730d 100644 --- a/airflow/hooks/mssql_hook.py +++ b/airflow/hooks/mssql_hook.py @@ -18,14 +18,18 @@ class MsSqlHook(DbApiHook): - ''' + """ Interact with Microsoft SQL Server. - ''' + """ conn_name_attr = 'mssql_conn_id' default_conn_name = 'mssql_default' supports_autocommit = True + def __init__(self, *args, **kwargs): + super(MsSqlHook, self).__init__(*args, **kwargs) + self.schema = kwargs.pop("schema", None) + def get_conn(self): """ Returns a mssql connection object @@ -35,7 +39,7 @@ def get_conn(self): server=conn.host, user=conn.login, password=conn.password, - database=conn.schema, + database=self.schema or conn.schema, port=conn.port) return conn diff --git a/airflow/hooks/mysql_hook.py b/airflow/hooks/mysql_hook.py index e4f953320105a..bf1a721e0e2e2 100644 --- a/airflow/hooks/mysql_hook.py +++ b/airflow/hooks/mysql_hook.py @@ -19,18 +19,22 @@ class MySqlHook(DbApiHook): - ''' + """ Interact with MySQL. You can specify charset in the extra field of your connection as ``{"charset": "utf8"}``. Also you can choose cursor as ``{"cursor": "SSCursor"}``. Refer to the MySQLdb.cursors for more details. - ''' + """ conn_name_attr = 'mysql_conn_id' default_conn_name = 'mysql_default' supports_autocommit = True + def __init__(self, *args, **kwargs): + super(MySqlHook, self).__init__(*args, **kwargs) + self.schema = kwargs.pop("schema", None) + def get_conn(self): """ Returns a mysql connection object @@ -38,17 +42,16 @@ def get_conn(self): conn = self.get_connection(self.mysql_conn_id) conn_config = { "user": conn.login, - "passwd": conn.password or '' + "passwd": conn.password or '', + "host": conn.host or 'localhost', + "db": self.schema or conn.schema or '' } - conn_config["host"] = conn.host or 'localhost' if not conn.port: conn_config["port"] = 3306 else: conn_config["port"] = int(conn.port) - conn_config["db"] = conn.schema or '' - if conn.extra_dejson.get('charset', False): conn_config["charset"] = conn.extra_dejson["charset"] if (conn_config["charset"]).lower() == 'utf8' or\ diff --git a/airflow/hooks/postgres_hook.py b/airflow/hooks/postgres_hook.py index 75c822602677a..372e4e54847da 100644 --- a/airflow/hooks/postgres_hook.py +++ b/airflow/hooks/postgres_hook.py @@ -19,22 +19,26 @@ class PostgresHook(DbApiHook): - ''' + """ Interact with Postgres. You can specify ssl parameters in the extra field of your connection as ``{"sslmode": "require", "sslcert": "/path/to/cert.pem", etc}``. - ''' + """ conn_name_attr = 'postgres_conn_id' default_conn_name = 'postgres_default' supports_autocommit = True + def __init__(self, *args, **kwargs): + super(PostgresHook, self).__init__(*args, **kwargs) + self.schema = kwargs.pop("schema", None) + def get_conn(self): conn = self.get_connection(self.postgres_conn_id) conn_args = dict( host=conn.host, user=conn.login, password=conn.password, - dbname=conn.schema, + dbname=self.schema or conn.schema, port=conn.port) # check for ssl parameters in conn.extra for arg_name, arg_val in conn.extra_dejson.items(): diff --git a/airflow/operators/mssql_operator.py b/airflow/operators/mssql_operator.py index 0590454e08117..0f0cd6344c384 100644 --- a/airflow/operators/mssql_operator.py +++ b/airflow/operators/mssql_operator.py @@ -27,6 +27,8 @@ class MsSqlOperator(BaseOperator): :param sql: the sql code to be executed :type sql: string or string pointing to a template file. File must have a '.sql' extensions. + :param database: name of database which overwrite defined one in connection + :type database: string """ template_fields = ('sql',) @@ -36,14 +38,17 @@ class MsSqlOperator(BaseOperator): @apply_defaults def __init__( self, sql, mssql_conn_id='mssql_default', parameters=None, - autocommit=False, *args, **kwargs): + autocommit=False, database=None, *args, **kwargs): super(MsSqlOperator, self).__init__(*args, **kwargs) self.mssql_conn_id = mssql_conn_id self.sql = sql self.parameters = parameters self.autocommit = autocommit + self.database = database def execute(self, context): logging.info('Executing: ' + str(self.sql)) - hook = MsSqlHook(mssql_conn_id=self.mssql_conn_id) - hook.run(self.sql, autocommit=self.autocommit, parameters=self.parameters) + hook = MsSqlHook(mssql_conn_id=self.mssql_conn_id, + schema=self.database) + hook.run(self.sql, autocommit=self.autocommit, + parameters=self.parameters) diff --git a/airflow/operators/mysql_operator.py b/airflow/operators/mysql_operator.py index b3a3c73d1bf4c..156ada8e90714 100644 --- a/airflow/operators/mysql_operator.py +++ b/airflow/operators/mysql_operator.py @@ -29,6 +29,8 @@ class MySqlOperator(BaseOperator): :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 database: name of database which overwrite defined one in connection + :type database: string """ template_fields = ('sql',) @@ -38,16 +40,18 @@ class MySqlOperator(BaseOperator): @apply_defaults def __init__( self, sql, mysql_conn_id='mysql_default', parameters=None, - autocommit=False, *args, **kwargs): + autocommit=False, database=None, *args, **kwargs): super(MySqlOperator, self).__init__(*args, **kwargs) self.mysql_conn_id = mysql_conn_id self.sql = sql self.autocommit = autocommit self.parameters = parameters + self.database = database def execute(self, context): logging.info('Executing: ' + str(self.sql)) - hook = MySqlHook(mysql_conn_id=self.mysql_conn_id) + hook = MySqlHook(mysql_conn_id=self.mysql_conn_id, + schema=self.database) hook.run( self.sql, autocommit=self.autocommit, diff --git a/airflow/operators/postgres_operator.py b/airflow/operators/postgres_operator.py index c4f56a49d8324..0de5aa53cd3ab 100644 --- a/airflow/operators/postgres_operator.py +++ b/airflow/operators/postgres_operator.py @@ -29,6 +29,8 @@ class PostgresOperator(BaseOperator): :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 database: name of database which overwrite defined one in connection + :type database: string """ template_fields = ('sql',) @@ -40,14 +42,17 @@ def __init__( self, sql, postgres_conn_id='postgres_default', autocommit=False, parameters=None, + database=None, *args, **kwargs): super(PostgresOperator, self).__init__(*args, **kwargs) self.sql = sql self.postgres_conn_id = postgres_conn_id self.autocommit = autocommit self.parameters = parameters + self.database = database def execute(self, context): logging.info('Executing: ' + str(self.sql)) - self.hook = PostgresHook(postgres_conn_id=self.postgres_conn_id) + self.hook = PostgresHook(postgres_conn_id=self.postgres_conn_id, + schema=self.database) self.hook.run(self.sql, self.autocommit, parameters=self.parameters) diff --git a/tests/operators/operators.py b/tests/operators/operators.py index 6fc2449649c97..62bc4bf80e032 100644 --- a/tests/operators/operators.py +++ b/tests/operators/operators.py @@ -114,6 +114,27 @@ def test_sql_sensor(self): dag=self.dag) t.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE, ignore_ti_state=True) + def test_overwrite_schema(self): + """ + Verifies option to overwrite connection schema + """ + import airflow.operators.mysql_operator + + sql = "SELECT 1;" + t = operators.mysql_operator.MySqlOperator( + task_id='test_mysql_operator_test_schema_overwrite', + sql=sql, + dag=self.dag, + database="foobar", + ) + + from _mysql_exceptions import OperationalError + try: + t.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE, + ignore_ti_state=True) + except OperationalError as e: + assert "Unknown database 'foobar'" in str(e) + @skipUnlessImported('airflow.operators.postgres_operator', 'PostgresOperator') class PostgresTest(unittest.TestCase): @@ -193,6 +214,28 @@ def test_vacuum(self): autocommit=True) t.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE, ignore_ti_state=True) + def test_overwrite_schema(self): + """ + Verifies option to overwrite connection schema + """ + import airflow.operators.postgres_operator + + sql = "SELECT 1;" + t = operators.postgres_operator.PostgresOperator( + task_id='postgres_operator_test_schema_overwrite', + sql=sql, + dag=self.dag, + autocommit=True, + database="foobar", + ) + + from psycopg2._psycopg import OperationalError + try: + t.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE, + ignore_ti_state=True) + except OperationalError as e: + assert 'database "foobar" does not exist' in str(e) + @skipUnlessImported('airflow.operators.hive_operator', 'HiveOperator') @skipUnlessImported('airflow.operators.postgres_operator', 'PostgresOperator')