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

Conversation

uranusjr
Copy link
Member

Previously, with catchup=False, CronTriggerTimetable would aggressively cut off a run if the scheduler doesn't ask to schedule the next run immediately. This causes DAGs to seemingly mysteriously "miss" a run from time to time due to the scheduler inevitably having a very slight hiccup.

This change makes the timetable's non-catchup cutoff logic a little more lax, and only activate when the scheduler misses at least an entire interval. For example, for a daily cron, if the previous run happened on midnight of 2nd Jun (to cover 1st Jun), the timetable would still allow scheduling a run covering 2nd Jun if the scheduler asks for it some time during the 2nd, and would skip the 2nd Jun run entirely only if the scheduler fails to ask for a run on the entirety of the 2nd and only asks after midnight on the 3rd.

As discussed in #27399, I feel this is the more reasonable fix than #32921, even though it slightly changes the behaviour (specifically, the first ever run would start one interval earlier than previously). Quoting from the linked issue:

I think a reasonable logic would be to change the catchup=False logic to cover one schedule before the current time instead, so in the above scenario, the timetable would make the next run cover 4am, and only skip the 4am run if the current time is pas 5am.

Sounds good. It's natural to those who are used to Cron and Anacron.

I think we need to document the new behavior. […]

Fix #27339. Close #32921.

Previously, with catchup=False, CronTriggerTimetable would aggressively
cut off a run if the scheduler doesn't ask to schedule the next run
immediately. This causes DAGs to seemingly mysteriously "miss" a run
from time to time due to the scheduler inevitably having a very slight
hiccup.

This change makes the timetable's non-catchup cutoff logic a little more
lax, and only activate when the scheduler misses at least an entire
interval. For example, for a daily cron, if the previous run happened on
midnight of 2nd Jun (to cover 1st Jun), the timetable would still allow
scheduling a run covering 2nd Jun if the scheduler asks for it some time
during the 2nd, and would skip the 2nd Jun run entirely only if the
scheduler fails to ask for a run on the entirety of the 2nd and only
asks after midnight on the 3rd.
@uranusjr uranusjr marked this pull request as ready for review August 15, 2023 09:51
Copy link
Member

@potiuk potiuk left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes. I think it is a very good fix even if it changes the first run behaviour.

@potiuk
Copy link
Member

potiuk commented Aug 17, 2023

Thanks for very detailed explanation @uranusjr !

@uranusjr uranusjr merged commit a6299d4 into apache:main Aug 18, 2023
42 checks passed
@uranusjr uranusjr deleted the cron-trigger-timetable-align-prev branch August 18, 2023 06:22
@ephraimbuddy ephraimbuddy added this to the Airflow 2.7.1 milestone Aug 27, 2023
@ephraimbuddy ephraimbuddy added the type:bug-fix Changelog: Bug Fixes label Aug 27, 2023
ephraimbuddy pushed a commit that referenced this pull request Aug 28, 2023
@o-nikolas
Copy link
Contributor

Regarding this Q&A discussion: #35647

Could this change be the cause? I've also received feedback from MWAA customer that the CronTriggerTimetable is broken in the same way described in that discussion.

CC: @uranusjr @potiuk

@potiuk
Copy link
Member

potiuk commented Dec 30, 2023

I am not sure, but I think it was the opposite problem. And I am not entirely sure what's the CronTimetable expected behaviour it - it's not explained in https://airflow.apache.org/docs/apache-airflow/stable/_api/airflow/timetables/interval/index.html#airflow.timetables.interval.CronDataIntervalTimetable whether the previous schedule should be when not full interval passes in this case.

Ah my bad. This actually IS described as desired behaviour because this one in
#35647 is CronTriggerTimetable not CronDataIntervalTimetable

https://airflow.apache.org/docs/apache-airflow/stable/_api/airflow/timetables/trigger/index.html

This is different from CronDataIntervalTimetable, where the cron expression specifies the data interval of a DAG run. With this timetable, the data intervals are specified independently from the cron expression. Also for the same reason, this timetable kicks off a DAG run immediately at the start of the period (similar to POSIX cron), instead of needing to wait for one data interval to pass.

@potiuk
Copy link
Member

potiuk commented Dec 30, 2023

But the decription quoted by the user contradicts it IMHO https://airflow.apache.org/docs/apache-airflow/stable/authoring-and-scheduling/timetable.html#the-time-when-a-dag-run-is-triggered

Here is an example showing how the first DAG run is triggered. Supposes there is a cron expression @daily or 0 0 * * *, which is aimed to run at 12AM every day. If you enable DAGs using the two timetables at 3PM on January 31st, CronTriggerTimetable will trigger a new DAG run at 12AM on February 1st. CronDataIntervalTimetable, on the other hand, will immediately trigger a new DAG run which is supposed to trigger at 12AM on January 31st if the DAG had been enabled beforehand.

So yeah. I am also confused what are the expectations here - @uranusjr - do you know?

@fatmumuhomer
Copy link
Contributor

@potiuk, @o-nikolas The original behavior of the CronTriggerTimetable was as described in the docs and as described as the expected behavior in #35647. If the DAG was enabled, the first DAG run would occur only once the next schedule interval had occurred - e.g. for schedule "0 13 * * *" if the DAG was enabled on December 30th at 13:01, the DAG would not run until December 31st at 13:00.

However, there was the bug (#27399) where this could lead to Airflow not scheduling the DAG on occasion. The fix implemented altered this behavior so Airflow will now trigger for the previous schedule interval once enabled - e.g. for schedule "0 13 * * *" if the DAG was enabled on December 30th at 13:01, the DAG would run immediately for the schedule interval for December 30th.

The docs are wrong. I have been hoping to make a PR to fix them but hadn't had a chance.

For what it's worth, I think there is still value in a timetable that behaves the way the docs describe it though it seems that is more complicated than originally expected.

@shubham22
Copy link

shubham22 commented Jan 2, 2024

The docs are wrong. I have been hoping to make a PR to fix them but hadn't had a chance.

In this case, I would say the behavior described in the docs should be the actual behavior. The fix made in this PR is great, but it shouldn't alter the bahavior for the 1st triggered run. The flag catchup=False makes it further confusing that 1st run is triggered for ongoing interval.
I am only 6 months late (sorry about that!), but I think what TP suggested here #27399 (comment) about adding a flag is more reasonable. Open to other better suggestions as adding a flag just adds more "read the docs" burden on the users.

@gmuenchCB
Copy link

The docs are wrong. I have been hoping to make a PR to fix them but hadn't had a chance.

In this case, I would say the behavior described in the docs should be the actual behavior. The fix made in this PR is great, but it shouldn't alter the bahavior for the 1st triggered run. The flag catchup=False makes it further confusing that 1st run is triggered for ongoing interval. I am only 6 months late (sorry about that!), but I think what TP suggested here #27399 (comment) about adding a flag is more reasonable. Open to other better suggestions as adding a flag just adds more "read the docs" burden on the users.

I agree with @shubham22 here. A DAG should not be triggered to run for the first time prior to its scheduled run time using CronTriggerTimetable. This is the point of Catchup=False.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
type:bug-fix Changelog: Bug Fixes
Projects
None yet
Development

Successfully merging this pull request may close these issues.

7 participants