Status

StateCompleted
Discussion Thread
Vote Threadhttps://lists.apache.org/thread/wkjl9prj274psjqm32pqvgmns73d6dnx
Vote Result Threadhttps://lists.apache.org/thread/rlwdxmz16sy9p5bolhkmhy2hplo36zj4
Progress Tracking (PR/GitHub Project/Issue Label)https://github.com/orgs/apache/projects/410
GitHub Projectlink
Date Created

2024-07-30

Version Released3.0.0
Authors

Motivation

On execution_date uniqueness / requiredness

Since inception, Airflow has used execution_date to distinguish different runs of a DAG.  It's essentially what defined the run of a dag.  Presumably this came from one of Airlfow's early use cases, moving hive partitions around.  In that example, the dag's behavior is driven by execution date.  And in this style of dag, backfill makes sense, and execution_date serves as a sort of data completeness tracker.

But a substantial portion of dags in the wild are not execution-date driven, in that the execution_date of the dag run does not affect behavior of the the dag.  Perhaps the dag looks to an external source to determine its state and behavior.  If a dag has catchup=False, this is a sign that the dag may not really care about execution date.  For a dag that does not care about execution date, the association of the dag run with an execution date is actively misleading.  Even more so with the rename to "logical date".

To more fully embrace dags that don't care about execution date, in this AIP we propose to make it nullable.

This also solves a problem for folks who generate many dag runs in quick succession via API or via TriggerDagRunOperator.  Previously they may have had to add milliseconds to the logical date of each run.

On the question of logical_date vs execution_date

Execution date has always been confusing.  Does it mean "when it ran"?  Or something else?  If you can't answer that, you're not alone.  Some time ago, the initiative to replace execution_date with logical_date was... initiated.  This AIP proposes to complete that initiative, and finally rename execution_date to logical_date.  Logical date is not a bad name because, it implies that it is a date that carries meaning for the dag, that drives the behavior of the dag, as distinguished from execution_date which, could be misunderstood as "when it ran".

Proposal

Generally speaking, references to execution_date should be replaced with logical_date, and logical date should be made nullable.

Logical_date should be a required but nullable param for triggering a dag.

Arguments in Python functions and the REST API using execution_date to look up a DAG run should be removed. The arguments should all have already been deprecated.

The execution_date key shall be removed from the task execution context.

The logical_date may still be shown to the user when useful.

New date run_after is added, and should be preferred to logical_date generally speaking.  Every dag run will have a run_after date, but not all will have a logical_date.

When logical_date is used, the run ID value should be displayed alongside it for reference if needed. In the DAGs View, for example, while the Last Run column may continue to display a date, the run ID should be shown with e.g. a tooltip when the user mouse-over the cell.

In cases where the execution date value is displayed both under execution_date and another name, the execution_date field should be removed.

The logic to generate a run_id to write a DAG run to the database should be slightly modified so it does not only rely on the logical_date field. One simple solution would be to use the ON CONFLICT (or equivalent) clause to add a short hash to the run ID when needed.

When a dag run has no logical date, if a task tries to access the logical_date (or derivative) task context attribute, KeyError should be raised.

Compatibility and Migration

As mentioned before, all current usages of execution_date are either deprecated, or already available under another name. A properly migrated code base should therefore not have any compatibility issues, and can be modified to be compatible before upgrading.

One notable possibility to incompatibility is log templates, specifically the task_log_prefix_template and log_filename_template configurations. The default Airflow configuration only uses run_id exclusively in log_filename_template, but a custom configured template may still uses execution_date (which will break) or the value under another name. These configurations need to modified to either use run_id, or add another value (dag_run.id, for example) so two DAG runs don’t accidentally write to the same log file.

Airflow currently uses execution_date in task_log_prefix_template. This value is only used for display purposes in Airflow, but may be critical to identify DAG runs uniquely for third-party tools and services that consume and parse Airflow logs. The default configuration will be changed to use run_id instead. The same guidelines above apply for custom values of this configuration key.


Note: Amended 3/6/2025 to reflect the amendments as agreed on dev list.