Skip to content

Commit

Permalink
Fix AttributeError: datetime.timezone object has no attribute…
Browse files Browse the repository at this point in the history
… ``name`` (#16599)

closes: #16551

Previous implementation tried to force / coerce the provided timezone (from the dag's `start_date`) into a `pendulum.tz.timezone.*` that only worked if the provided timezone was already a pendulum's timezone and it specifically failed when with `datetime.timezone.utc` as timezone.

(cherry picked from commit 86c2091)
  • Loading branch information
ecerulm authored and jhtimmins committed Aug 17, 2021
1 parent 79f78ad commit 42c5101
Show file tree
Hide file tree
Showing 3 changed files with 44 additions and 2 deletions.
4 changes: 2 additions & 2 deletions airflow/models/dag.py
Original file line number Diff line number Diff line change
Expand Up @@ -485,7 +485,7 @@ def following_schedule(self, dttm):
else:
# absolute (e.g. 3 AM)
naive = cron.get_next(datetime)
tz = pendulum.timezone(self.timezone.name)
tz = self.timezone
following = timezone.make_aware(naive, tz)
return timezone.convert_to_utc(following)
elif self.normalized_schedule_interval is not None:
Expand Down Expand Up @@ -513,7 +513,7 @@ def previous_schedule(self, dttm):
else:
# absolute (e.g. 3 AM)
naive = cron.get_prev(datetime)
tz = pendulum.timezone(self.timezone.name)
tz = self.timezone
previous = timezone.make_aware(naive, tz)
return timezone.convert_to_utc(previous)
elif self.normalized_schedule_interval is not None:
Expand Down
40 changes: 40 additions & 0 deletions tests/models/test_dag.py
Original file line number Diff line number Diff line change
Expand Up @@ -621,6 +621,46 @@ def test_following_schedule_relativedelta(self):
_next = dag.following_schedule(_next)
assert _next.isoformat() == "2015-01-02T02:00:00+00:00"

def test_previous_schedule_datetime_timezone(self):
# Check that we don't get an AttributeError 'name' for self.timezone

start = datetime.datetime(2018, 3, 25, 2, tzinfo=datetime.timezone.utc)
dag = DAG('tz_dag', start_date=start, schedule_interval='@hourly')
when = dag.previous_schedule(start)
assert when.isoformat() == "2018-03-25T01:00:00+00:00"

def test_following_schedule_datetime_timezone(self):
# Check that we don't get an AttributeError 'name' for self.timezone

start = datetime.datetime(2018, 3, 25, 2, tzinfo=datetime.timezone.utc)
dag = DAG('tz_dag', start_date=start, schedule_interval='@hourly')
when = dag.following_schedule(start)
assert when.isoformat() == "2018-03-25T03:00:00+00:00"

def test_following_schedule_datetime_timezone_utc0530(self):
# Check that we don't get an AttributeError 'name' for self.timezone
class UTC0530(datetime.tzinfo):
"""tzinfo derived concrete class named "+0530" with offset of 19800"""

# can be configured here
_offset = datetime.timedelta(seconds=19800)
_dst = datetime.timedelta(0)
_name = "+0530"

def utcoffset(self, dt):
return self.__class__._offset

def dst(self, dt):
return self.__class__._dst

def tzname(self, dt):
return self.__class__._name

start = datetime.datetime(2018, 3, 25, 10, tzinfo=UTC0530())
dag = DAG('tz_dag', start_date=start, schedule_interval='@hourly')
when = dag.following_schedule(start)
assert when.isoformat() == "2018-03-25T05:30:00+00:00"

def test_dagtag_repr(self):
clear_db_dags()
dag = DAG('dag-test-dagtag', start_date=DEFAULT_DATE, tags=['tag-1', 'tag-2'])
Expand Down
2 changes: 2 additions & 0 deletions tests/serialization/test_dag_serialization.py
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@
from glob import glob
from unittest import mock

import pendulum
import pytest
from dateutil.relativedelta import FR, relativedelta
from kubernetes.client import models as k8s
Expand Down Expand Up @@ -444,6 +445,7 @@ def validate_deserialized_task(
datetime(2019, 7, 30, tzinfo=timezone.utc),
datetime(2019, 8, 1, tzinfo=timezone.utc),
),
(pendulum.datetime(2019, 8, 1, tz='UTC'), None, pendulum.datetime(2019, 8, 1, tz='UTC')),
]
)
def test_deserialization_start_date(self, dag_start_date, task_start_date, expected_task_start_date):
Expand Down

0 comments on commit 42c5101

Please sign in to comment.