Repository: incubator-airflow Updated Branches: refs/heads/master 97680d85f -> bf8c8b247
[AIRFLOW-1480] Render template attributes for ExternalTaskSensor fields Closes #2926 from pdambrauskas/feature/external_ta sk_sensor_templates Project: http://git-wip-us.apache.org/repos/asf/incubator-airflow/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-airflow/commit/bf8c8b24 Tree: http://git-wip-us.apache.org/repos/asf/incubator-airflow/tree/bf8c8b24 Diff: http://git-wip-us.apache.org/repos/asf/incubator-airflow/diff/bf8c8b24 Branch: refs/heads/master Commit: bf8c8b24751d5b1016f2a063e69a81b317c4db47 Parents: 97680d8 Author: Paulius <paul...@exacaster.com> Authored: Wed Jan 10 19:56:23 2018 +0100 Committer: Bolke de Bruin <bo...@xs4all.nl> Committed: Wed Jan 10 19:56:23 2018 +0100 ---------------------------------------------------------------------- airflow/operators/sensors.py | 1 + tests/operators/sensors.py | 19 +++++++++++++++++++ 2 files changed, 20 insertions(+) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/bf8c8b24/airflow/operators/sensors.py ---------------------------------------------------------------------- diff --git a/airflow/operators/sensors.py b/airflow/operators/sensors.py index c8a8df6..e9bf7ff 100644 --- a/airflow/operators/sensors.py +++ b/airflow/operators/sensors.py @@ -202,6 +202,7 @@ class ExternalTaskSensor(BaseSensorOperator): or execution_date_fn can be passed to ExternalTaskSensor, but not both. :type execution_date_fn: callable """ + template_fields = ['external_dag_id', 'external_task_id'] ui_color = '#19647e' @apply_defaults http://git-wip-us.apache.org/repos/asf/incubator-airflow/blob/bf8c8b24/tests/operators/sensors.py ---------------------------------------------------------------------- diff --git a/tests/operators/sensors.py b/tests/operators/sensors.py index fccc9c3..d60f15c 100644 --- a/tests/operators/sensors.py +++ b/tests/operators/sensors.py @@ -272,6 +272,25 @@ class ExternalTaskSensorTests(unittest.TestCase): 'start_date': DEFAULT_DATE, 'depends_on_past': False} + def test_templated_sensor(self): + dag = DAG(TEST_DAG_ID, self.args) + + with dag: + sensor = ExternalTaskSensor( + task_id='templated_task', + external_dag_id='dag_{{ ds }}', + external_task_id='task_{{ ds }}', + start_date=DEFAULT_DATE + ) + + instance = TaskInstance(sensor, DEFAULT_DATE) + instance.render_templates() + + self.assertEqual(sensor.external_dag_id, + "dag_{}".format(DEFAULT_DATE.date())) + self.assertEqual(sensor.external_task_id, + "task_{}".format(DEFAULT_DATE.date())) + def test_external_task_sensor_fn_multiple_execution_dates(self): bash_command_code = """ {% set s=execution_date.time().second %}