Versions Compared

Key

  • This line was added.
  • This line was removed.
  • Formatting was changed.
Comment: Acceptance

Status

Page properties


StateVoteAccepted
Discussion Thread
Vote Threadhttps://lists.apache.org/thread/wkjl9prj274psjqm32pqvgmns73d6dnx
Vote Result Threadhttps://lists.apache.org/thread/rlwdxmz16sy9p5bolhkmhy2hplo36zj4
Progress Tracking (PR/GitHub Project/Issue Label)
Date Created

2024-07-30

Version Released
Authors



Motivation

Traditionally, Airflow uses execution_date to distinguish different runs of a DAG. More accurately, different instances of a task, and a run consists of tasks of the same DAG and execution date—the technical differences are beyond the topic. This subtle distinction is why Airflow designed things in this way.

Now that we have established a separate DAG run concept, however, using the execution date as the identifier starts introducing unnecessarily limits to what Airflow can do. For example, TriggerDagRunOperator cannot be trivially mapped dynamically.

TriggerDagRunOperator.partial(task_id="trigger_two").expand(
    conf=[{"foo": 1}, {"foo": 2}],
)

This trivial implementation will not work because both runs will use the same date (inherited from the parent run). It is of course possible to work around this (by manually adding fractions of a second to distinguish between each run), but it not only is awkward, but also makes subsequent lookups more difficult since the parent run now has a different date from its children.

Rationale

Airflow 2.2 introduced a series of changes to move various concepts away from execution date. Task instances are now associated with the DAG run by an integer primary key, and DAG runs (of a DAG) are now identified and looked up by the run_id field both in the web UI, the REST API, and public Python functions. Using the execution_date field in these interfaces still carries a significant semantic to a DAG run, using it to look up a run now emits a deprecated warning, and is not essential for Airflow to correctly function.

I believe Airflow 3 is the right time for us to remove those deprecated arguments to look up a DAG run. And with those functionalities being removed, we should also be able to remove the unique constraint on execution_date in the database.

Proposal

A database migration should be created in Airflow 3 to remove the unique constraint at the database level.

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 execution date value may still be shown to the user when useful. It must be under another name, such as logical date. This should already be the case in recent Airflow 2 versions. They are still considered acceptable in Airflow 3 since the value is still useful for a human to quickly distinguish DAG runs, although not technically unique. When the value 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.

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.