From 20398f4cb3468c89ae901414ba436a38ae02b9c3 Mon Sep 17 00:00:00 2001 From: Chao-Han Tsai Date: Tue, 22 May 2018 13:23:21 -0700 Subject: [PATCH] [AIRFLOW-2510] Introduce new macros: prev_ds and next_ds --- airflow/models.py | 16 +++++++++++++--- docs/code.rst | 6 ++++++ 2 files changed, 19 insertions(+), 3 deletions(-) diff --git a/airflow/models.py b/airflow/models.py index bcd12fb5b1c6a..14fb194276922 100755 --- a/airflow/models.py +++ b/airflow/models.py @@ -1750,14 +1750,22 @@ def get_template_context(self, session=None): if 'tables' in task.params: tables = task.params['tables'] - ds = self.execution_date.isoformat()[:10] + ds = self.execution_date.strftime('%Y-%m-%d') ts = self.execution_date.isoformat() - yesterday_ds = (self.execution_date - timedelta(1)).isoformat()[:10] - tomorrow_ds = (self.execution_date + timedelta(1)).isoformat()[:10] + yesterday_ds = (self.execution_date - timedelta(1)).strftime('%Y-%m-%d') + tomorrow_ds = (self.execution_date + timedelta(1)).strftime('%Y-%m-%d') prev_execution_date = task.dag.previous_schedule(self.execution_date) next_execution_date = task.dag.following_schedule(self.execution_date) + next_ds = None + if next_execution_date: + next_ds = next_execution_date.strftime('%Y-%m-%d') + + prev_ds = None + if prev_execution_date: + prev_ds = prev_execution_date.strftime('%Y-%m-%d') + ds_nodash = ds.replace('-', '') ts_nodash = ts.replace('-', '').replace(':', '') yesterday_ds_nodash = yesterday_ds.replace('-', '') @@ -1819,6 +1827,8 @@ def __repr__(self): return { 'dag': task.dag, 'ds': ds, + 'next_ds': next_ds, + 'prev_ds': prev_ds, 'ds_nodash': ds_nodash, 'ts': ts, 'ts_nodash': ts_nodash, diff --git a/docs/code.rst b/docs/code.rst index 1737d156100e3..53435094e5ac0 100644 --- a/docs/code.rst +++ b/docs/code.rst @@ -231,6 +231,12 @@ Variable Description ================================= ==================================== ``{{ ds }}`` the execution date as ``YYYY-MM-DD`` ``{{ ds_nodash }}`` the execution date as ``YYYYMMDD`` +``{{ prev_ds }}`` the previous execution date as ``YYYY-MM-DD``. + if ``{{ ds }}`` is ``2016-01-08`` and ``schedule_interval`` is ``@weekly``, + ``{{ prev_ds }}`` will be ``2016-01-01``. +``{{ next_ds }}`` the next execution date as ``YYYY-MM-DD``. + if ``{{ ds }}`` is ``2016-01-01`` and ``schedule_interval`` is ``@weekly``, + ``{{ prev_ds }}`` will be ``2016-01-08``. ``{{ yesterday_ds }}`` yesterday's date as ``YYYY-MM-DD`` ``{{ yesterday_ds_nodash }}`` yesterday's date as ``YYYYMMDD`` ``{{ tomorrow_ds }}`` tomorrow's date as ``YYYY-MM-DD``