You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by GitBox <gi...@apache.org> on 2019/01/05 02:17:47 UTC

[GitHub] feng-tao closed pull request #4359: [AIRFLOW-3150] Make execution_date templated in TriggerDagRunOperator

feng-tao closed pull request #4359: [AIRFLOW-3150] Make execution_date templated in TriggerDagRunOperator
URL: https://github.com/apache/incubator-airflow/pull/4359
 
 
   

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/dagrun_operator.py b/airflow/operators/dagrun_operator.py
index 8d339894f4..7000c6c90b 100644
--- a/airflow/operators/dagrun_operator.py
+++ b/airflow/operators/dagrun_operator.py
@@ -17,6 +17,8 @@
 # specific language governing permissions and limitations
 # under the License.
 
+import datetime
+import six
 from airflow.models import BaseOperator
 from airflow.utils import timezone
 from airflow.utils.decorators import apply_defaults
@@ -47,10 +49,10 @@ class TriggerDagRunOperator(BaseOperator):
         to your tasks while executing that DAG run. Your function header
         should look like ``def foo(context, dag_run_obj):``
     :type python_callable: python callable
-    :param execution_date: Execution date for the dag
-    :type execution_date: datetime.datetime
+    :param execution_date: Execution date for the dag (templated)
+    :type execution_date: str or datetime.datetime
     """
-    template_fields = ('trigger_dag_id',)
+    template_fields = ('trigger_dag_id', 'execution_date')
     template_ext = tuple()
     ui_color = '#ffefeb'
 
@@ -64,10 +66,26 @@ def __init__(
         super(TriggerDagRunOperator, self).__init__(*args, **kwargs)
         self.python_callable = python_callable
         self.trigger_dag_id = trigger_dag_id
-        self.execution_date = execution_date
+
+        if isinstance(execution_date, datetime.datetime):
+            self.execution_date = execution_date.isoformat()
+        elif isinstance(execution_date, six.string_types):
+            self.execution_date = execution_date
+        elif execution_date is None:
+            self.execution_date = execution_date
+        else:
+            raise TypeError(
+                'Expected str or datetime.datetime type '
+                'for execution_date. Got {}'.format(
+                    type(execution_date)))
 
     def execute(self, context):
-        dro = DagRunOrder(run_id='trig__' + timezone.utcnow().isoformat())
+        if self.execution_date is not None:
+            run_id = 'trig__{}'.format(self.execution_date)
+            self.execution_date = timezone.parse(self.execution_date)
+        else:
+            run_id = 'trig__' + timezone.utcnow().isoformat()
+        dro = DagRunOrder(run_id=run_id)
         if self.python_callable is not None:
             dro = self.python_callable(context, dro)
         if dro:
diff --git a/tests/core.py b/tests/core.py
index 9110c0138a..4d7b4af1ff 100644
--- a/tests/core.py
+++ b/tests/core.py
@@ -44,7 +44,7 @@
 
 from airflow import configuration
 from airflow.executors import SequentialExecutor
-from airflow.models import Variable
+from airflow.models import Variable, TaskInstance
 
 from airflow import jobs, models, DAG, utils, macros, settings, exceptions
 from airflow.models import BaseOperator
@@ -990,6 +990,42 @@ def payload_generator(context, object):
         dag_run = dag_runs[0]
         self.assertEquals(dag_run.execution_date, utc_now)
 
+    def test_trigger_dagrun_with_str_execution_date(self):
+        utc_now_str = timezone.utcnow().isoformat()
+        self.assertIsInstance(utc_now_str, six.string_types)
+        run_id = 'trig__' + utc_now_str
+
+        def payload_generator(context, object):
+            object.run_id = run_id
+            return object
+
+        task = TriggerDagRunOperator(
+            task_id='test_trigger_dagrun_with_str_execution_date',
+            trigger_dag_id='example_bash_operator',
+            python_callable=payload_generator,
+            execution_date=utc_now_str,
+            dag=self.dag)
+        task.run(start_date=DEFAULT_DATE, end_date=DEFAULT_DATE, ignore_ti_state=True)
+        dag_runs = models.DagRun.find(dag_id='example_bash_operator',
+                                      run_id=run_id)
+        self.assertEquals(len(dag_runs), 1)
+        dag_run = dag_runs[0]
+        self.assertEquals(dag_run.execution_date.isoformat(), utc_now_str)
+
+    def test_trigger_dagrun_with_templated_execution_date(self):
+        task = TriggerDagRunOperator(
+            task_id='test_trigger_dagrun_with_str_execution_date',
+            trigger_dag_id='example_bash_operator',
+            execution_date='{{ execution_date }}',
+            dag=self.dag)
+
+        self.assertTrue(isinstance(task.execution_date, six.string_types))
+        self.assertEqual(task.execution_date, '{{ execution_date }}')
+
+        ti = TaskInstance(task=task, execution_date=DEFAULT_DATE)
+        ti.render_templates()
+        self.assertEqual(timezone.parse(task.execution_date), DEFAULT_DATE)
+
     def test_externally_triggered_dagrun(self):
         TI = models.TaskInstance
 


 

----------------------------------------------------------------
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


With regards,
Apache Git Services