Skip to content

Commit 2b407c7

Browse files
bolkedebruinAlice Berard
authored and
Alice Berard
committed
[AIRFLOW-2984] Convert operator dates to UTC (apache#3822)
Tasks can have start_dates or end_dates separately from the DAG. These need to be converted to UTC otherwise we cannot use them for calculation the next execution date.
1 parent 2d1ed5a commit 2b407c7

File tree

3 files changed

+52
-32
lines changed

3 files changed

+52
-32
lines changed

airflow/models.py

+7
Original file line numberDiff line numberDiff line change
@@ -2413,10 +2413,17 @@ def __init__(
24132413
self.email = email
24142414
self.email_on_retry = email_on_retry
24152415
self.email_on_failure = email_on_failure
2416+
24162417
self.start_date = start_date
24172418
if start_date and not isinstance(start_date, datetime):
24182419
self.log.warning("start_date for %s isn't datetime.datetime", self)
2420+
elif start_date:
2421+
self.start_date = timezone.convert_to_utc(start_date)
2422+
24192423
self.end_date = end_date
2424+
if end_date:
2425+
self.end_date = timezone.convert_to_utc(end_date)
2426+
24202427
if not TriggerRule.is_valid(trigger_rule):
24212428
raise AirflowException(
24222429
"The trigger_rule must be one of {all_triggers},"

docs/timezone.rst

+35-32
Original file line numberDiff line numberDiff line change
@@ -2,23 +2,23 @@ Time zones
22
==========
33

44
Support for time zones is enabled by default. Airflow stores datetime information in UTC internally and in the database.
5-
It allows you to run your DAGs with time zone dependent schedules. At the moment Airflow does not convert them to the
6-
end user’s time zone in the user interface. There it will always be displayed in UTC. Also templates used in Operators
5+
It allows you to run your DAGs with time zone dependent schedules. At the moment Airflow does not convert them to the
6+
end user’s time zone in the user interface. There it will always be displayed in UTC. Also templates used in Operators
77
are not converted. Time zone information is exposed and it is up to the writer of DAG what do with it.
88

9-
This is handy if your users live in more than one time zone and you want to display datetime information according to
9+
This is handy if your users live in more than one time zone and you want to display datetime information according to
1010
each user’s wall clock.
1111

12-
Even if you are running Airflow in only one time zone it is still good practice to store data in UTC in your database
13-
(also before Airflow became time zone aware this was also to recommended or even required setup). The main reason is
14-
Daylight Saving Time (DST). Many countries have a system of DST, where clocks are moved forward in spring and backward
15-
in autumn. If you’re working in local time, you’re likely to encounter errors twice a year, when the transitions
16-
happen. (The pendulum and pytz documentation discusses these issues in greater detail.) This probably doesn’t matter
17-
for a simple DAG, but it’s a problem if you are in, for example, financial services where you have end of day
18-
deadlines to meet.
12+
Even if you are running Airflow in only one time zone it is still good practice to store data in UTC in your database
13+
(also before Airflow became time zone aware this was also to recommended or even required setup). The main reason is
14+
Daylight Saving Time (DST). Many countries have a system of DST, where clocks are moved forward in spring and backward
15+
in autumn. If you’re working in local time, you’re likely to encounter errors twice a year, when the transitions
16+
happen. (The pendulum and pytz documentation discusses these issues in greater detail.) This probably doesn’t matter
17+
for a simple DAG, but it’s a problem if you are in, for example, financial services where you have end of day
18+
deadlines to meet.
1919

20-
The time zone is set in `airflow.cfg`. By default it is set to utc, but you change it to use the system’s settings or
21-
an arbitrary IANA time zone, e.g. `Europe/Amsterdam`. It is dependent on `pendulum`, which is more accurate than `pytz`.
20+
The time zone is set in `airflow.cfg`. By default it is set to utc, but you change it to use the system’s settings or
21+
an arbitrary IANA time zone, e.g. `Europe/Amsterdam`. It is dependent on `pendulum`, which is more accurate than `pytz`.
2222
Pendulum is installed when you install Airflow.
2323

2424
Please note that the Web UI currently only runs in UTC.
@@ -28,8 +28,8 @@ Concepts
2828
Naïve and aware datetime objects
2929
''''''''''''''''''''''''''''''''
3030

31-
Python’s datetime.datetime objects have a tzinfo attribute that can be used to store time zone information,
32-
represented as an instance of a subclass of datetime.tzinfo. When this attribute is set and describes an offset,
31+
Python’s datetime.datetime objects have a tzinfo attribute that can be used to store time zone information,
32+
represented as an instance of a subclass of datetime.tzinfo. When this attribute is set and describes an offset,
3333
a datetime object is aware. Otherwise, it’s naive.
3434

3535
You can use timezone.is_aware() and timezone.is_naive() to determine whether datetimes are aware or naive.
@@ -39,7 +39,7 @@ Because Airflow uses time-zone-aware datetime objects. If your code creates date
3939
.. code:: python
4040
4141
from airflow.utils import timezone
42-
42+
4343
now = timezone.utcnow()
4444
a_date = timezone.datetime(2017,1,1)
4545
@@ -49,9 +49,9 @@ Interpretation of naive datetime objects
4949

5050
Although Airflow operates fully time zone aware, it still accepts naive date time objects for `start_dates`
5151
and `end_dates` in your DAG definitions. This is mostly in order to preserve backwards compatibility. In
52-
case a naive `start_date` or `end_date` is encountered the default time zone is applied. It is applied
52+
case a naive `start_date` or `end_date` is encountered the default time zone is applied. It is applied
5353
in such a way that it is assumed that the naive date time is already in the default time zone. In other
54-
words if you have a default time zone setting of `Europe/Amsterdam` and create a naive datetime `start_date` of
54+
words if you have a default time zone setting of `Europe/Amsterdam` and create a naive datetime `start_date` of
5555
`datetime(2017,1,1)` it is assumed to be a `start_date` of Jan 1, 2017 Amsterdam time.
5656

5757
.. code:: python
@@ -65,16 +65,16 @@ words if you have a default time zone setting of `Europe/Amsterdam` and create a
6565
op = DummyOperator(task_id='dummy', dag=dag)
6666
print(op.owner) # Airflow
6767
68-
Unfortunately, during DST transitions, some datetimes don’t exist or are ambiguous.
69-
In such situations, pendulum raises an exception. That’s why you should always create aware
68+
Unfortunately, during DST transitions, some datetimes don’t exist or are ambiguous.
69+
In such situations, pendulum raises an exception. That’s why you should always create aware
7070
datetime objects when time zone support is enabled.
7171

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

7676

77-
Default time zone
77+
Default time zone
7878
'''''''''''''''''
7979

8080
The default time zone is the time zone defined by the `default_timezone` setting under `[core]`. If
@@ -92,15 +92,15 @@ it is therefore important to make sure this setting is equal on all Airflow node
9292
Time zone aware DAGs
9393
--------------------
9494

95-
Creating a time zone aware DAG is quite simple. Just make sure to supply a time zone aware `start_date`. It is
95+
Creating a time zone aware DAG is quite simple. Just make sure to supply a time zone aware `start_date`. It is
9696
recommended to use `pendulum` for this, but `pytz` (to be installed manually) can also be used for this.
9797

9898
.. code:: python
9999
100100
import pendulum
101-
101+
102102
local_tz = pendulum.timezone("Europe/Amsterdam")
103-
103+
104104
default_args=dict(
105105
start_date=datetime(2016, 1, 1, tzinfo=local_tz),
106106
owner='Airflow'
@@ -110,29 +110,32 @@ recommended to use `pendulum` for this, but `pytz` (to be installed manually) ca
110110
op = DummyOperator(task_id='dummy', dag=dag)
111111
print(dag.timezone) # <Timezone [Europe/Amsterdam]>
112112
113-
113+
Please note that while it is possible to set a `start_date` and `end_date` for Tasks always the DAG timezone
114+
or global timezone (in that order) will be used to calculate the next execution date. Upon first encounter
115+
the start date or end date will be converted to UTC using the timezone associated with start_date or end_date,
116+
then for calculations this timezone information will be disregarded.
114117

115118
Templates
116119
'''''''''
117120

118-
Airflow returns time zone aware datetimes in templates, but does not convert them to local time so they remain in UTC.
121+
Airflow returns time zone aware datetimes in templates, but does not convert them to local time so they remain in UTC.
119122
It is left up to the DAG to handle this.
120123

121124
.. code:: python
122125
123126
import pendulum
124-
127+
125128
local_tz = pendulum.timezone("Europe/Amsterdam")
126129
local_tz.convert(execution_date)
127130
128131
129132
Cron schedules
130133
''''''''''''''
131134

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

137140

138141
Time deltas

tests/models.py

+10
Original file line numberDiff line numberDiff line change
@@ -1521,6 +1521,16 @@ def test_timezone_awareness(self):
15211521
ti = TI(task=op1, execution_date=execution_date)
15221522
self.assertEquals(ti.execution_date, utc_date)
15231523

1524+
def test_task_naive_datetime(self):
1525+
NAIVE_DATETIME = DEFAULT_DATE.replace(tzinfo=None)
1526+
1527+
op_no_dag = DummyOperator(task_id='test_task_naive_datetime',
1528+
start_date=NAIVE_DATETIME,
1529+
end_date=NAIVE_DATETIME)
1530+
1531+
self.assertTrue(op_no_dag.start_date.tzinfo)
1532+
self.assertTrue(op_no_dag.end_date.tzinfo)
1533+
15241534
def test_set_dag(self):
15251535
"""
15261536
Test assigning Operators to Dags, including deferred assignment

0 commit comments

Comments
 (0)