Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
16 changes: 13 additions & 3 deletions airflow/models.py
Original file line number Diff line number Diff line change
Expand Up @@ -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
Copy link
Member

@feng-tao feng-tao May 24, 2018

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

per discussion, I wonder if we could have a better default value here .

But good to see the travis CI pass :)

Copy link
Contributor Author

@milton0825 milton0825 May 24, 2018

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The idea of {{ next_ds }} is just {{ next_execution_date }} in YYYY-MM-DD format so that we don't need to write something like {{ ds_format(next_execution_date, "%Y-%m-%d")}}

Thus I think we should make sure that the behavior of {{ next_ds }} aligns with
{{ next_execution_date }}.

So I think we should do either:
option 1:
If {{ next_execution_date }} is None, we set {{ next_ds }} to None.

option 2:
We make sure {{ next_execution_date }} is never None. But this will change the fact that
{{ next_execution_date }} is None when schedule_interval="@once".

Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

cool, Default with none is ok in this case.

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('-', '')
Expand Down Expand Up @@ -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,
Expand Down
6 changes: 6 additions & 0 deletions docs/code.rst
Original file line number Diff line number Diff line change
Expand Up @@ -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``
Expand Down