Skip to content

ArgNotSet.__repr__ includes memory address, causing non-deterministic DAG serialization and infinite dag_version increases #65220

@necmo

Description

@necmo

Apache Airflow version

3.2.0

What happened

DAG versions increase every ~30 seconds (on every scheduler parse cycle) even though the DAG file has not changed. The dag_version table accumulates hundreds of entries during a single DAG run.

By diffing two consecutive serialized DAG versions, I found the only difference is the memory address in the ArgNotSet repr:

- "logical_date": "<airflow.serialization.definitions.notset.ArgNotSet object at 0x7fe833fa8f20>"
+ "logical_date": "<airflow.serialization.definitions.notset.ArgNotSet object at 0x7f115effa180>"

What you think should happen instead

DAG version should remain stable when the DAG file and its dependencies have not changed.

How to reproduce

  1. Create a DAG using TriggerDagRunOperator without explicitly setting logical_date
  2. Register the DAG with default_args
  3. Observe the dag_version table growing every parse cycle (~30 seconds)
from airflow.providers.standard.operators.trigger_dagrun import TriggerDagRunOperator

default_args = {
    "email_on_failure": True,
    "email_on_retry": False,
}

@dag(default_args=default_args, ...)
def my_dag():
    trigger = TriggerDagRunOperator(
        task_id="trigger_other_dag",
        trigger_dag_id="other_dag",
        wait_for_completion=True,
        # logical_date is NOT set → defaults to ArgNotSet
    )

Verify with this query:

SELECT dv.version_number, sd.data
FROM dag_version dv
JOIN serialized_dag sd ON sd.dag_id = dv.dag_id AND sd.dag_version_id = dv.id
WHERE dv.dag_id = '<dag_id>'
ORDER BY dv.version_number DESC
LIMIT 2;

Diff the two data JSON values — the only difference is the ArgNotSet memory address.

Root cause

ArgNotSet (airflow/serialization/definitions/notset.py) has no custom __repr__:

class ArgNotSet:
    """Sentinel type for annotations, useful when None is not viable."""

While the primary serialize() path handles ArgNotSet via is_arg_set() check, there are code paths (such as default_args dict values or certain operator parameters) where ArgNotSet bypasses proper encoding and falls through to str()/repr(), producing a memory-address-dependent string.

Since the scheduler runs DagFileProcessor as separate subprocesses, each process allocates ArgNotSet at a different address → different serialized JSON → new dag_version on every parse.

Suggested fix

Add a stable __repr__ to ArgNotSet:

class ArgNotSet:
    """Sentinel type for annotations, useful when None is not viable."""
    def __repr__(self) -> str:
        return "NOTSET"

Alternatively (or additionally), ensure all serialization code paths handle ArgNotSet via is_arg_set() before falling through to str().

Workaround

Explicitly set logical_date=None in default_args or on the operator to prevent the ArgNotSet sentinel from being serialized.

Related issues

Are you willing to submit PR?

Yes

Operating System

Linux (Kubernetes)

Versions of Apache Airflow Providers

apache-airflow-providers-standard

Deployment

Kubernetes

Deployment details

Airflow 3.2.0 on Kubernetes with default scheduler configuration.

Metadata

Metadata

Assignees

No one assigned

    Labels

    area:corearea:serializationkind:bugThis is a clearly a bugpriority:highHigh priority bug that should be patched quickly but does not require immediate new release

    Type

    No type
    No fields configured for issues without a type.

    Projects

    No projects

    Milestone

    No milestone

    Relationships

    None yet

    Development

    No branches or pull requests

    Issue actions