Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -1587,6 +1587,14 @@ def _deserialize_field_value(cls, field_name: str, value: Any) -> Any:
elif field_name == "resources":
return Resources.from_dict(value) if value is not None else None
elif field_name.endswith("_date"):
# Handle ARG_NOT_SET (NOTSET singleton) before trying to parse as datetime.
# Without this, deserializing operators that store a date field as ``NOTSET``
# (e.g. ``TriggerDagRunOperator.logical_date``) would fail because
# ``_deserialize_datetime`` does not understand the ARG_NOT_SET encoding.
if isinstance(value, dict) and value.get(Encoding.TYPE) == DAT.ARG_NOT_SET:
from airflow.serialization.definitions.notset import NOTSET

return NOTSET
return cls._deserialize_datetime(value) if value is not None else None
else:
# For all other fields, return as-is (strings, ints, bools, etc.)
Expand Down
17 changes: 17 additions & 0 deletions airflow-core/tests/unit/serialization/test_serialized_objects.py
Original file line number Diff line number Diff line change
Expand Up @@ -1011,6 +1011,23 @@ def test_deserialize_datetime_with_timestamp(self):
assert isinstance(result, datetime)
assert result.timestamp() == timestamp

def test_deserialize_field_value_with_arg_not_set_for_date_fields(self):
"""``_deserialize_field_value`` returns ``NOTSET`` for ARG_NOT_SET date fields.

Operators may store ``NOTSET`` (an ``ArgNotSet`` instance) on date-suffixed
fields such as ``logical_date`` to mean "use the default at runtime". When
such a field is round-tripped through serialization, the ARG_NOT_SET
encoding must be restored to ``NOTSET`` instead of being parsed as a
datetime.
"""
from airflow.serialization.definitions.notset import NOTSET
from airflow.serialization.serialized_objects import OperatorSerialization

value = {Encoding.TYPE: DAT.ARG_NOT_SET}

for field_name in ("logical_date", "start_date", "end_date"):
assert OperatorSerialization._deserialize_field_value(field_name, value) is NOTSET


class TestRetryPolicySerialization:
"""Test that retry_policy is serialized as a boolean flag (has_retry_policy)."""
Expand Down
Loading