You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by "ASF GitHub Bot (JIRA)" <ji...@apache.org> on 2018/12/18 04:22:01 UTC

[jira] [Commented] (AIRFLOW-1096) Add conn_ids to template_fields in PostgreSQL and MySQL operators

    [ https://issues.apache.org/jira/browse/AIRFLOW-1096?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16723663#comment-16723663 ] 

ASF GitHub Bot commented on AIRFLOW-1096:
-----------------------------------------

stale[bot] closed pull request #2235: [AIRFLOW-1096] Add conn_ids to template_fields
URL: https://github.com/apache/incubator-airflow/pull/2235
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/airflow/operators/mysql_operator.py b/airflow/operators/mysql_operator.py
index 156ada8e90..9c93910385 100644
--- a/airflow/operators/mysql_operator.py
+++ b/airflow/operators/mysql_operator.py
@@ -33,7 +33,7 @@ class MySqlOperator(BaseOperator):
     :type database: string
     """
 
-    template_fields = ('sql',)
+    template_fields = ('sql', 'mysql_conn_id',)
     template_ext = ('.sql',)
     ui_color = '#ededed'
 
diff --git a/airflow/operators/postgres_operator.py b/airflow/operators/postgres_operator.py
index 0de5aa53cd..0b9d5556a4 100644
--- a/airflow/operators/postgres_operator.py
+++ b/airflow/operators/postgres_operator.py
@@ -33,7 +33,7 @@ class PostgresOperator(BaseOperator):
     :type database: string
     """
 
-    template_fields = ('sql',)
+    template_fields = ('sql', 'postgres_conn_id',)
     template_ext = ('.sql',)
     ui_color = '#ededed'
 
diff --git a/tests/operators/operators.py b/tests/operators/operators.py
index 62bc4bf80e..9930f6ef6a 100644
--- a/tests/operators/operators.py
+++ b/tests/operators/operators.py
@@ -89,6 +89,16 @@ def mysql_hook_test_bulk_load(self):
                 results = tuple(result[0] for result in c.fetchall())
                 self.assertEqual(sorted(results), sorted(records))
 
+    def test_mysql_conn_id_template(self):
+        conn = 'airflow_db'
+
+        import airflow.operators.mysql_operator
+        t = operators.mysql_operator.MySqlOperator(
+            task_id='test_mysql_conn_id_template',
+            mysql_conn_id='{{ conn }}',
+            sql='SELECT count(1) FROM INFORMATION_SCHEMA.TABLES',
+            dag=self.dag)
+
     def test_mysql_to_mysql(self):
         sql = "SELECT * FROM INFORMATION_SCHEMA.TABLES LIMIT 100;"
         import airflow.operators.generic_transfer
@@ -174,6 +184,16 @@ def postgres_operator_test_multi(self):
         t = operators.postgres_operator.PostgresOperator(
             task_id='postgres_operator_test_multi', sql=sql, dag=self.dag)
         t.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE, ignore_ti_state=True)
+    
+    def test_postgres_conn_id_template(self):
+        conn = 'postgres_default'
+
+        import airflow.operators.postgres_operator
+        t = operators.postgres_operator.PostgresOperator(
+            task_id='test_postgres_conn_id_template', 
+            postgres_conn_id='{{ conn }}',
+            sql='SELECT count(1) FROM INFORMATION_SCHEMA.TABLES', 
+            dag=self.dag)
 
     def test_postgres_to_postgres(self):
         sql = "SELECT * FROM INFORMATION_SCHEMA.TABLES LIMIT 100;"


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


> Add conn_ids to template_fields in PostgreSQL and MySQL operators
> -----------------------------------------------------------------
>
>                 Key: AIRFLOW-1096
>                 URL: https://issues.apache.org/jira/browse/AIRFLOW-1096
>             Project: Apache Airflow
>          Issue Type: New Feature
>            Reporter: Nicholas Duffy
>            Assignee: Nicholas Duffy
>            Priority: Minor
>
> As an Airflow developer, I would like to have the `postgres_conn_id` field on the PostgresOperator and `mysql_conn_id` field on the MysqlOperator templated, so that I can pass in dynamic connection ID names.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)