Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Allow timetable to slightly miss catchup cutoff #33404

Merged
merged 2 commits into from
Aug 18, 2023
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
2 changes: 1 addition & 1 deletion airflow/timetables/trigger.py
Original file line number Diff line number Diff line change
Expand Up @@ -89,7 +89,7 @@ def next_dagrun_info(
else:
next_start_time = self._align_to_next(restriction.earliest)
else:
start_time_candidates = [self._align_to_next(DateTime.utcnow())]
start_time_candidates = [self._align_to_prev(DateTime.utcnow())]
if last_automated_data_interval is not None:
start_time_candidates.append(self._get_next(last_automated_data_interval.end))
if restriction.earliest is not None:
Expand Down
16 changes: 16 additions & 0 deletions newsfragments/33404.significant.rst
Original file line number Diff line number Diff line change
@@ -0,0 +1,16 @@
CronTriggerTimetable is now less aggressive when trying to skip a run

When setting ``catchup=False``, CronTriggerTimetable no longer skips a run if
the scheduler does not query the timetable immediately after the previous run
has been triggered.

This should not affect scheduling in most cases, but can change the behaviour if
a DAG is paused-unpaused to manually skip a run. Previously, the timetable (with
``catchup=False``) would only start a run after a DAG is unpaused, but with this
change, the scheduler would try to look at little bit back to schedule the
previous run that covers a part of the period when the DAG was paused. This
means you will need to keep a DAG paused longer (namely, for the entire cron
period to pass) to really skip a run.

Note that this is also the behaviour exhibited by various other cron-based
scheduling tools, such as anacron.
18 changes: 15 additions & 3 deletions tests/timetables/test_trigger_timetable.py
Original file line number Diff line number Diff line change
Expand Up @@ -48,11 +48,11 @@
[
pytest.param(
None,
CURRENT_TIME + DELTA_FROM_MIDNIGHT,
YESTERDAY + DELTA_FROM_MIDNIGHT,
id="first-run",
),
pytest.param(
PREV_DATA_INTERVAL_EXACT,
DataInterval.exact(YESTERDAY + DELTA_FROM_MIDNIGHT),
CURRENT_TIME + DELTA_FROM_MIDNIGHT,
id="before-now",
),
Expand Down Expand Up @@ -89,9 +89,21 @@ def test_daily_cron_trigger_no_catchup_first_starts_at_next_schedule(
pytest.param(
pendulum.DateTime(2022, 7, 27, 0, 30, 0, tzinfo=TIMEZONE),
START_DATE,
DagRunInfo.exact(pendulum.DateTime(2022, 7, 27, 1, 0, 0, tzinfo=TIMEZONE)),
DagRunInfo.exact(pendulum.DateTime(2022, 7, 27, 0, 0, 0, tzinfo=TIMEZONE)),
id="current_time_not_on_boundary",
),
pytest.param(
pendulum.DateTime(2022, 7, 27, 1, 0, 0, tzinfo=TIMEZONE),
START_DATE,
DagRunInfo.exact(pendulum.DateTime(2022, 7, 27, 1, 0, 0, tzinfo=TIMEZONE)),
id="current_time_miss_one_interval_on_boundary",
),
pytest.param(
pendulum.DateTime(2022, 7, 27, 1, 30, 0, tzinfo=TIMEZONE),
START_DATE,
DagRunInfo.exact(pendulum.DateTime(2022, 7, 27, 1, 0, 0, tzinfo=TIMEZONE)),
id="current_time_miss_one_interval_not_on_boundary",
),
pytest.param(
pendulum.DateTime(2022, 7, 27, 0, 30, 0, tzinfo=TIMEZONE),
pendulum.DateTime(2199, 12, 31, 22, 30, 0, tzinfo=TIMEZONE),
Expand Down