From 829f90ad337c2ea94db7cd58ccdd71dd680ad419 Mon Sep 17 00:00:00 2001 From: Robin Edwards Date: Fri, 22 Oct 2021 02:03:59 +0100 Subject: [PATCH] Fix queued dag runs changes catchup=False behaviour (#19130) --- airflow/timetables/interval.py | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/airflow/timetables/interval.py b/airflow/timetables/interval.py index 8f5c3f1032a29..0dd7ffdbf136c 100644 --- a/airflow/timetables/interval.py +++ b/airflow/timetables/interval.py @@ -81,9 +81,15 @@ def next_dagrun_info( return None start = self._align(earliest) else: - # There's a previous run. Create a data interval starting from when - # the end of the previous interval. - start = last_automated_data_interval.end + # There's a previous run. + if earliest is not None: + # Catchup is False or DAG has new start date in the future. + # Make sure we get the latest start date + start = max(last_automated_data_interval.end, earliest) + else: + # Create a data interval starting from when the end of the previous interval. + start = last_automated_data_interval.end + if restriction.latest is not None and start > restriction.latest: return None end = self._get_next(start)