Skip to content

Commit

Permalink
[AIRFLOW-XXX] Clarify daylight savings time behavior (#6324)
Browse files Browse the repository at this point in the history
(cherry picked from commit 52c2be5)
  • Loading branch information
jmcarp authored and kaxil committed Dec 12, 2019
1 parent 1218bd8 commit 10a4392
Showing 1 changed file with 14 additions and 12 deletions.
26 changes: 14 additions & 12 deletions docs/timezone.rst
Original file line number Diff line number Diff line change
Expand Up @@ -51,7 +51,7 @@ Python’s datetime.datetime objects have a tzinfo attribute that can be used to
represented as an instance of a subclass of datetime.tzinfo. When this attribute is set and describes an offset,
a datetime object is aware. Otherwise, it’s naive.

You can use timezone.is_aware() and timezone.is_naive() to determine whether datetimes are aware or naive.
You can use ``timezone.is_localized()`` and ``timezone.is_naive()`` to determine whether datetimes are aware or naive.

Because Airflow uses time-zone-aware datetime objects. If your code creates datetime objects they need to be aware too.

Expand Down Expand Up @@ -90,15 +90,15 @@ datetime objects when time zone support is enabled.

In practice, this is rarely an issue. Airflow gives you aware datetime objects in the models and DAGs, and most often,
new datetime objects are created from existing ones through timedelta arithmetic. The only datetime that’s often
created in application code is the current time, and timezone.utcnow() automatically does the right thing.
created in application code is the current time, and ``timezone.utcnow()`` automatically does the right thing.


Default time zone
'''''''''''''''''

The default time zone is the time zone defined by the ``default_timezone`` setting under ``[core]``. If
you just installed Airflow it will be set to ``utc``, which is recommended. You can also set it to
`system` or an IANA time zone (e.g.``Europe/Amsterdam``). DAGs are also evaluated on Airflow workers,
``system`` or an IANA time zone (e.g.``Europe/Amsterdam``). DAGs are also evaluated on Airflow workers,
it is therefore important to make sure this setting is equal on all Airflow nodes.


Expand Down Expand Up @@ -147,18 +147,20 @@ It is left up to the DAG to handle this.
local_tz = pendulum.timezone("Europe/Amsterdam")
local_tz.convert(execution_date)
Cron schedules
''''''''''''''

In case you set a cron schedule, Airflow assumes you will always want to run at the exact same time. It will
then ignore day light savings time. Thus, if you have a schedule that says
run at end of interval every day at 08:00 GMT+1 it will always run end of interval 08:00 GMT+1,
regardless if day light savings time is in place.

Time zone aware DAGs that use cron schedules respect daylight savings
time. For example, a DAG with a start date in the ``US/Eastern`` time zone
with a schedule of ``0 0 * * *`` will run daily at 04:00 UTC during
daylight savings time and at 05:00 otherwise.

Time deltas
'''''''''''
For schedules with time deltas Airflow assumes you always will want to run with the specified interval. So if you
specify a timedelta(hours=2) you will always want to run to hours later. In this case day light savings time will
be taken into account.

Time zone aware DAGs that use ``timedelta`` or ``relativedelta`` schedules
respect daylight savings time for the start date but do not adjust for
daylight savings time when scheduling subsequent runs. For example, a
DAG with a start date of ``pendulum.create(2020, 1, 1, tz="US/Eastern")``
and a schedule interval of ``timedelta(days=1)`` will run daily at 05:00
UTC regardless of daylight savings time.

0 comments on commit 10a4392

Please sign in to comment.