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

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

Fokko closed pull request #2334: [AIRFLOW-1252] API - Fix when conf is in JSON 
body
URL: https://github.com/apache/incubator-airflow/pull/2334
 
 
   

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/api/common/experimental/trigger_dag.py 
b/airflow/api/common/experimental/trigger_dag.py
index d7353f66d1..0268752565 100644
--- a/airflow/api/common/experimental/trigger_dag.py
+++ b/airflow/api/common/experimental/trigger_dag.py
@@ -59,7 +59,10 @@ def _trigger_dag(
 
     run_conf = None
     if conf:
-        run_conf = json.loads(conf)
+        if type(conf) is dict:
+            run_conf = conf
+        else:
+            run_conf = json.loads(conf)
 
     triggers = list()
     dags_to_trigger = list()
diff --git a/tests/api/common/experimental/trigger_dag_tests.py 
b/tests/api/common/experimental/trigger_dag_tests.py
index d6354840e2..fc1d7cda8e 100644
--- a/tests/api/common/experimental/trigger_dag_tests.py
+++ b/tests/api/common/experimental/trigger_dag_tests.py
@@ -19,6 +19,7 @@
 
 import mock
 import unittest
+import json
 
 from airflow.exceptions import AirflowException
 from airflow.models import DAG, DagRun
@@ -88,6 +89,44 @@ def test_trigger_dag_include_subdags(self, dag_bag_mock, 
dag_run_mock, dag_mock)
 
         self.assertEqual(3, len(triggers))
 
+    @mock.patch('airflow.models.DagBag')
+    def test_trigger_dag_with_str_conf(self, dag_bag_mock):
+        dag_id = "trigger_dag_with_str_conf"
+        dag = DAG(dag_id)
+        dag_bag_mock.dags = [dag_id]
+        dag_bag_mock.get_dag.return_value = dag
+        conf = "{\"foo\": \"bar\"}"
+        dag_run = DagRun()
+        triggers = _trigger_dag(
+            dag_id,
+            dag_bag_mock,
+            dag_run,
+            run_id=None,
+            conf=conf,
+            execution_date=None,
+            replace_microseconds=True)
+
+        self.assertEquals(triggers[0].conf, json.loads(conf))
+
+    @mock.patch('airflow.models.DagBag')
+    def test_trigger_dag_with_dict_conf(self, dag_bag_mock):
+        dag_id = "trigger_dag_with_dict_conf"
+        dag = DAG(dag_id)
+        dag_bag_mock.dags = [dag_id]
+        dag_bag_mock.get_dag.return_value = dag
+        conf = dict(foo="bar")
+        dag_run = DagRun()
+        triggers = _trigger_dag(
+            dag_id,
+            dag_bag_mock,
+            dag_run,
+            run_id=None,
+            conf=conf,
+            execution_date=None,
+            replace_microseconds=True)
+
+        self.assertEquals(triggers[0].conf, conf)
+
 
 if __name__ == '__main__':
     unittest.main()


 

----------------------------------------------------------------
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:
us...@infra.apache.org


> Experimental API - exception when conf is present in JSON body
> --------------------------------------------------------------
>
>                 Key: AIRFLOW-1252
>                 URL: https://issues.apache.org/jira/browse/AIRFLOW-1252
>             Project: Apache Airflow
>          Issue Type: Bug
>          Components: api
>    Affects Versions: 1.8.0, 1.8.1, 1.9.0, 1.10.0, 2.0.0
>            Reporter: Sergio Herrera
>            Assignee: Sergio Herrera
>            Priority: Major
>              Labels: api
>             Fix For: 2.0.0
>
>
> When someones calls to the endpoint _POST 
> <host>:<port>/api/experimental/dags/<dag_id>/dag_runs {}_, Airflow never run 
> that request if the body of that contains _conf_.
>  This occurs due to a mismatch between types when calling function 
> _trigger_dag()_, which is also used by *CLI*. That function perform a 
> _json.loads(conf)_ because from CLI the type of conf is _string_, but, in the 
> other side, from *experimental API*, that type is _dict_ (because _Json_ is 
> processed before to get all data, such as execution_date).
> There are two possibilities:
>  1. Look for every use of _trigger_dag()_ function and put _Json_ formatting 
> from outside the function.
>  2. In the *experimental API*, put the conf in a string (with _json.dumps()_) 
> to allow _trigger_dag()_ transform into _dict_.
> I have implemented the second option, so I can make a PR with that if you 
> want.
> Thank you a lot
> EDIT: Also, there are no tests which uses conf in the Json passed through 
> request currently.
> Examples:
>  - Before fix (escaped json):
> {noformat}
> POST /api/experimental/dags/test_conf/dag_runs HTTP/1.1
> Content-Type: application/json
> {
>   "conf": "{
>     \"k1\": \"v1\",
>     \"k2\": \"v2\",
>     \"k3\": [\"av1\", \"av2\", \"av3\"],
>     \"k4\": {
>       \"sk1\": \"sv1\",
>       \"sk2\": \"sv2\"
>     }
>   }"  
> }
> {noformat}
>  - After fix (pure json):
> {noformat}
> POST /api/experimental/dags/test_conf/dag_runs HTTP/1.1
> Content-Type: application/json
> {
>   "conf": {
>     "k1": "v1",
>     "k2": "v2",
>     "k3": ["av1", "av2", "av3"],
>     "k4": {
>       "sk1": "sv1",
>       "sk2": "sv2"
>     }
>   }
> }
> {noformat}
>  



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

Reply via email to