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 2021/06/27 12:25:41 UTC

[GitHub] [airflow] uranusjr commented on a change in pull request #16678: WIP: Solve timezone serialization by adding a schedule_timezone parameter to DAG

uranusjr commented on a change in pull request #16678:
URL: https://github.com/apache/airflow/pull/16678#discussion_r659282284



##########
File path: airflow/models/dag.py
##########
@@ -422,6 +428,38 @@ def __exit__(self, _type, _value, _tb):
 
     # /Context Manager ----------------------------------------------
 
+    def _get_tzname_or_offset(self, obj):

Review comment:
       This function (and `_is_valid_pendulum_tz_id`) does not use anything on the DAG (i.e. `self`) and should be free functions instead. Maybe put them at the top of the module or in `airflow.utils.dates`?

##########
File path: airflow/models/dag.py
##########
@@ -302,16 +304,20 @@ def __init__(
         self.fileloc = back.f_code.co_filename if back else ""
         self.task_dict: Dict[str, BaseOperator] = {}
 
-        # set timezone from start_date
+        # set timezone from schedule_timezone or start_date
+        if schedule_timezone:
+            self.schedule_timezone = self._get_tzname_or_offset(schedule_timezone)
         if start_date and start_date.tzinfo:
-            self.timezone = start_date.tzinfo
+            self.schedule_timezone = self._get_tzname_or_offset(start_date.tzinfo)
+            # self.timezone = start_date.tzinfo
         elif 'start_date' in self.default_args and self.default_args['start_date']:
             if isinstance(self.default_args['start_date'], str):
                 self.default_args['start_date'] = timezone.parse(self.default_args['start_date'])
-            self.timezone = self.default_args['start_date'].tzinfo
+            if self.default_args['start_date'] and self.default_args['start_date'].tzinfo:
+                self.schedule_timezone = self._get_tzname_or_offset(self.default_args['start_date'].tzinfo)
 
-        if not hasattr(self, 'timezone') or not self.timezone:
-            self.timezone = settings.TIMEZONE
+        if not hasattr(self, 'schedule_timezone'):
+            self.schedule_timezone = self._get_tzname_or_offset(settings.TIMEZONE)

Review comment:
       This `hasattr` call is super weird. This is the initialiser, we should have a way to know whether `self.schedule_timezone` is set or not without introspection. (Granted the existing code already does the same, but that does not make it less awkward.)

##########
File path: airflow/models/dag.py
##########
@@ -422,6 +428,38 @@ def __exit__(self, _type, _value, _tb):
 
     # /Context Manager ----------------------------------------------
 
+    def _get_tzname_or_offset(self, obj):
+        if self._is_valid_pendulum_tz_id(obj):
+            return obj
+        if hasattr(obj, 'name'):
+            candidate = obj.name
+            if self._is_valid_pendulum_tz_id(candidate):
+                return candidate
+        try:
+            candidate = obj.tzname(datetime.now(timezone.utc))
+            if self._is_valid_pendulum_tz_id(candidate):
+                return candidate
+        except (NotImplementedError, ValueError):
+            pass
+
+        try:
+            candidate = int(obj.utcoffset(datetime.now(timezone.utc)).total_seconds())
+            if self._is_valid_pendulum_tz_id(candidate):
+                return candidate
+        except (NotImplementedError, ValueError):
+            pass
+
+        raise ValueError(f"Can't get a timezone name or offset from {obj}")
+
+    def _is_valid_pendulum_tz_id(self, id):

Review comment:
       Dom’t name a variable `id` (linters will be unhappy). “Tz id” is also a bit confusing to me (that value does not actually identify anything—id is short for identifier). Maybe call it `name` or something?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org