You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by GitBox <gi...@apache.org> on 2018/08/30 12:26:13 UTC

[GitHub] bolkedebruin closed pull request #3822: [AIRFLOW-2984] Convert operator dates to UTC

bolkedebruin closed pull request #3822: [AIRFLOW-2984] Convert operator dates to UTC
URL: https://github.com/apache/incubator-airflow/pull/3822
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/airflow/models.py b/airflow/models.py
index 55badf4828..94e18794d6 100755
--- a/airflow/models.py
+++ b/airflow/models.py
@@ -2413,10 +2413,17 @@ def __init__(
         self.email = email
         self.email_on_retry = email_on_retry
         self.email_on_failure = email_on_failure
+
         self.start_date = start_date
         if start_date and not isinstance(start_date, datetime):
             self.log.warning("start_date for %s isn't datetime.datetime", self)
+        elif start_date:
+            self.start_date = timezone.convert_to_utc(start_date)
+
         self.end_date = end_date
+        if end_date:
+            self.end_date = timezone.convert_to_utc(end_date)
+
         if not TriggerRule.is_valid(trigger_rule):
             raise AirflowException(
                 "The trigger_rule must be one of {all_triggers},"
diff --git a/docs/timezone.rst b/docs/timezone.rst
index 9e8598e2ed..fe44ecfbb9 100644
--- a/docs/timezone.rst
+++ b/docs/timezone.rst
@@ -2,23 +2,23 @@ Time zones
 ==========
 
 Support for time zones is enabled by default. Airflow stores datetime information in UTC internally and in the database.
-It allows you to run your DAGs with time zone dependent schedules. At the moment Airflow does not convert them to the 
-end user’s time zone in the user interface. There it will always be displayed in UTC. Also templates used in Operators 
+It allows you to run your DAGs with time zone dependent schedules. At the moment Airflow does not convert them to the
+end user’s time zone in the user interface. There it will always be displayed in UTC. Also templates used in Operators
 are not converted. Time zone information is exposed and it is up to the writer of DAG what do with it.
 
-This is handy if your users live in more than one time zone and you want to display datetime information according to 
+This is handy if your users live in more than one time zone and you want to display datetime information according to
 each user’s wall clock.
 
-Even if you are running Airflow in only one time zone it is still good practice to store data in UTC in your database 
-(also before Airflow became time zone aware this was also to recommended or even required setup). The main reason is 
-Daylight Saving Time (DST). Many countries have a system of DST, where clocks are moved forward in spring and backward 
-in autumn. If you’re working in local time, you’re likely to encounter errors twice a year, when the transitions 
-happen. (The pendulum and pytz documentation discusses these issues in greater detail.) This probably doesn’t matter 
-for a simple DAG, but it’s a problem if you are in, for example, financial services where you have end of day 
-deadlines to meet. 
+Even if you are running Airflow in only one time zone it is still good practice to store data in UTC in your database
+(also before Airflow became time zone aware this was also to recommended or even required setup). The main reason is
+Daylight Saving Time (DST). Many countries have a system of DST, where clocks are moved forward in spring and backward
+in autumn. If you’re working in local time, you’re likely to encounter errors twice a year, when the transitions
+happen. (The pendulum and pytz documentation discusses these issues in greater detail.) This probably doesn’t matter
+for a simple DAG, but it’s a problem if you are in, for example, financial services where you have end of day
+deadlines to meet.
 
-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 
-an arbitrary IANA time zone, e.g. `Europe/Amsterdam`. It is dependent on `pendulum`, which is more accurate than `pytz`. 
+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
+an arbitrary IANA time zone, e.g. `Europe/Amsterdam`. It is dependent on `pendulum`, which is more accurate than `pytz`.
 Pendulum is installed when you install Airflow.
 
 Please note that the Web UI currently only runs in UTC.
@@ -28,8 +28,8 @@ Concepts
 Naïve and aware datetime objects
 ''''''''''''''''''''''''''''''''
 
-Python’s datetime.datetime objects have a tzinfo attribute that can be used to store time zone information, 
-represented as an instance of a subclass of datetime.tzinfo. When this attribute is set and describes an offset, 
+Python’s datetime.datetime objects have a tzinfo attribute that can be used to store time zone information,
+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.
@@ -39,7 +39,7 @@ Because Airflow uses time-zone-aware datetime objects. If your code creates date
 .. code:: python
 
     from airflow.utils import timezone
-    
+
     now = timezone.utcnow()
     a_date = timezone.datetime(2017,1,1)
 
@@ -49,9 +49,9 @@ Interpretation of naive datetime objects
 
 Although Airflow operates fully time zone aware, it still accepts naive date time objects for `start_dates`
 and `end_dates` in your DAG definitions. This is mostly in order to preserve backwards compatibility. In
-case a naive `start_date` or `end_date` is encountered the default time zone is applied. It is applied 
+case a naive `start_date` or `end_date` is encountered the default time zone is applied. It is applied
 in such a way that it is assumed that the naive date time is already in the default time zone. In other
-words if you have a default time zone setting of `Europe/Amsterdam` and create a naive datetime `start_date` of 
+words if you have a default time zone setting of `Europe/Amsterdam` and create a naive datetime `start_date` of
 `datetime(2017,1,1)` it is assumed to be a `start_date` of Jan 1, 2017 Amsterdam time.
 
 .. code:: python
@@ -65,16 +65,16 @@ words if you have a default time zone setting of `Europe/Amsterdam` and create a
     op = DummyOperator(task_id='dummy', dag=dag)
     print(op.owner) # Airflow
 
-Unfortunately, during DST transitions, some datetimes don’t exist or are ambiguous. 
-In such situations, pendulum raises an exception. That’s why you should always create aware 
+Unfortunately, during DST transitions, some datetimes don’t exist or are ambiguous.
+In such situations, pendulum raises an exception. That’s why you should always create aware
 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 
+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.
 
 
-Default time zone 
+Default time zone
 '''''''''''''''''
 
 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
 Time zone aware DAGs
 --------------------
 
-Creating a time zone aware DAG is quite simple. Just make sure to supply a time zone aware `start_date`. It is 
+Creating a time zone aware DAG is quite simple. Just make sure to supply a time zone aware `start_date`. It is
 recommended to use `pendulum` for this, but `pytz` (to be installed manually) can also be used for this.
 
 .. code:: python
 
     import pendulum
-    
+
     local_tz = pendulum.timezone("Europe/Amsterdam")
-    
+
     default_args=dict(
         start_date=datetime(2016, 1, 1, tzinfo=local_tz),
         owner='Airflow'
@@ -110,18 +110,21 @@ recommended to use `pendulum` for this, but `pytz` (to be installed manually) ca
     op = DummyOperator(task_id='dummy', dag=dag)
     print(dag.timezone) # <Timezone [Europe/Amsterdam]>
 
-
+Please note that while it is possible to set a `start_date` and `end_date` for Tasks always the DAG timezone
+or global timezone (in that order) will be used to calculate the next execution date. Upon first encounter
+the start date or end date will be converted to UTC using the timezone associated with start_date or end_date,
+then for calculations this timezone information will be disregarded.
 
 Templates
 '''''''''
 
-Airflow returns time zone aware datetimes in templates, but does not convert them to local time so they remain in UTC. 
+Airflow returns time zone aware datetimes in templates, but does not convert them to local time so they remain in UTC.
 It is left up to the DAG to handle this.
 
 .. code:: python
 
     import pendulum
-    
+
     local_tz = pendulum.timezone("Europe/Amsterdam")
     local_tz.convert(execution_date)
 
@@ -129,10 +132,10 @@ It is left up to the DAG to handle this.
 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. 
+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 deltas
diff --git a/tests/models.py b/tests/models.py
index a1fd1e9912..22340b05a2 100644
--- a/tests/models.py
+++ b/tests/models.py
@@ -1521,6 +1521,16 @@ def test_timezone_awareness(self):
         ti = TI(task=op1, execution_date=execution_date)
         self.assertEquals(ti.execution_date, utc_date)
 
+    def test_task_naive_datetime(self):
+        NAIVE_DATETIME = DEFAULT_DATE.replace(tzinfo=None)
+
+        op_no_dag = DummyOperator(task_id='test_task_naive_datetime',
+                                  start_date=NAIVE_DATETIME,
+                                  end_date=NAIVE_DATETIME)
+
+        self.assertTrue(op_no_dag.start_date.tzinfo)
+        self.assertTrue(op_no_dag.end_date.tzinfo)
+
     def test_set_dag(self):
         """
         Test assigning Operators to Dags, including deferred assignment


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services