Manual DAG runs cause prev_data_interval_start_success
and prev_data_interval_end_success
to behave unexpectedly
#28016
Labels
airflow3.0:breaking
Candidates for Airflow 3.0 that contain breaking changes
airflow3.0:candidate
Potential candidates for Airflow 3.0
area:core
kind:feature
Feature Requests
Apache Airflow version
Other Airflow 2 version (please specify below)
What happened
Using Airflow Version 2.4.2
Manually triggered DAG runs interspersed between scheduled DAG runs cause the scheduled DAG runs' template variables — in our specific use case,
prev_data_interval_start_success
andprev_data_interval_end_success
— to no longer behave as expected. This appears to be due to how Airflow determines a previous DAG run.When we have mixed manually triggered and scheduled DAG runs, the
logical_date
are not equivalent - For manually triggered DAG runs, thelogical_date
is the exact timestamp the DAG was triggered.The following table shows mixed manually triggered and scheduled DAG runs, and some of the templated timestamps. Notice that this table is ordered by the
Actual execution date
, which is the exact timestamp the DAG executed, and that thelogical_date
is out of order for manually triggered DAGs.Bolded italics are the template variables that IMO do not behave as predicted:
logical_date
data_interval
_start
data_interval
_end
prev_data_
interval_
start_success
prev_data_
interval_
end_success
For the bolded scheduled DAG runs, note that the
prev_data_interval_start_success
andprev_data_interval_end_success
both point to the data interval that is technically earlier than the most recent successful DAG run's data interval.Causes
I believe this is due to the way Airflow calculates the previous DAG run. If we order by
logical_date
, this function will prioritize manually triggered DAG runs, because:logical_date
(==execution_date
) are later thandata_interval_end
logical_date
are equal todata_interval_start
, which occurs beforedata_interval_end
for an equivalent "trigger" timestampSee in
airflow/models/dagrun.py
:What you think should happen instead
IMO this is a bug - DAG Run 6 in the table above should identify DAG Run 5 as the most recent successful DAG run, and the template variables should reflect this accordingly.
A deeper issue in how Airflow draws the line between scheduled DAG runs and manually triggered DAG runs.
There have been discussions on keeping manually triggered DAGs and scheduled DAGs entirely separate - best practice being don't ever manually trigger scheduled DAG runs.
In our use case, we want to be able to easily trigger a DAG run when there is a gap in execution, and that DAG run should have a data interval that covers any gaps.
Possible Solutions
null
and not aDataset
), manually triggered DAG runs setlogical_date
todata_interval_start
How to reproduce
Sample DAG that prints template variables every 5 minutes.
To reproduce, manually trigger the DAG between scheduled DAG runs and note the lag in
prev_data_interval_start_success
andprev_data_interval_end_success
.Operating System
debian
Versions of Apache Airflow Providers
No response
Deployment
Docker-Compose
Deployment details
No response
Anything else
No response
Are you willing to submit PR?
Code of Conduct
The text was updated successfully, but these errors were encountered: