This is an automated email from the ASF dual-hosted git repository. ephraimanierobi pushed a commit to branch v2-7-test in repository https://gitbox.apache.org/repos/asf/airflow.git
commit f0e81adf8ba1345a05cd385dfa3406bfbe9ffb70 Author: Cassandra <cassandra+...@beelen.one> AuthorDate: Thu Oct 12 05:27:19 2023 +0200 fix(TimeSensorAsync): use DAG timezone (#33406) * fix(TimeSensorAsync): use DAG timezone Fixes: #33256 (cherry picked from commit 6c50ef59cc4f739f126e5b123775340a3351a3e8) --- airflow/sensors/time_sensor.py | 2 +- tests/sensors/test_time_sensor.py | 13 +++++++++++++ 2 files changed, 14 insertions(+), 1 deletion(-) diff --git a/airflow/sensors/time_sensor.py b/airflow/sensors/time_sensor.py index 0d7d0a088f..aba2fab2d0 100644 --- a/airflow/sensors/time_sensor.py +++ b/airflow/sensors/time_sensor.py @@ -64,7 +64,7 @@ class TimeSensorAsync(BaseSensorOperator): self.target_time = target_time aware_time = timezone.coerce_datetime( - datetime.datetime.combine(datetime.datetime.today(), self.target_time) + datetime.datetime.combine(datetime.datetime.today(), self.target_time, self.dag.timezone) ) self.target_datetime = timezone.convert_to_utc(aware_time) diff --git a/tests/sensors/test_time_sensor.py b/tests/sensors/test_time_sensor.py index 2ccfdd2c42..935d1cb128 100644 --- a/tests/sensors/test_time_sensor.py +++ b/tests/sensors/test_time_sensor.py @@ -23,6 +23,7 @@ from unittest.mock import patch import pendulum import pytest import time_machine +from pendulum.tz.timezone import UTC from airflow.exceptions import TaskDeferred from airflow.models.dag import DAG @@ -73,3 +74,15 @@ class TestTimeSensorAsync: op = TimeSensorAsync(task_id="test", target_time=aware_time) assert hasattr(op.target_datetime.tzinfo, "offset") assert op.target_datetime.tzinfo.offset == 0 + + def test_target_time_naive_dag_timezone(self): + """ + Tests that naive target_time gets converted correctly using the DAG's timezone. + """ + with DAG( + "test_target_time_naive_dag_timezone", + start_date=pendulum.datetime(2020, 1, 1, 0, 0, tz=DEFAULT_TIMEZONE), + ): + op = TimeSensorAsync(task_id="test", target_time=pendulum.time(9, 0)) + assert op.target_datetime.time() == pendulum.time(1, 0) + assert op.target_datetime.tzinfo == UTC