Skip to content

Commit

Permalink
Fix queued dag runs changes catchup=False behaviour (apache#19130)
Browse files Browse the repository at this point in the history
  • Loading branch information
robinedwards authored Oct 22, 2021
1 parent 3a93ad1 commit 829f90a
Showing 1 changed file with 9 additions and 3 deletions.
12 changes: 9 additions & 3 deletions airflow/timetables/interval.py
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down

0 comments on commit 829f90a

Please sign in to comment.