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

CronTriggerTimetable lost one task occasionally #27399

Closed
1 of 2 tasks
potiuk opened this issue Oct 31, 2022 Discussed in #27398 · 40 comments
Closed
1 of 2 tasks

CronTriggerTimetable lost one task occasionally #27399

potiuk opened this issue Oct 31, 2022 Discussed in #27398 · 40 comments
Assignees
Labels
AIP-39 Timetables kind:bug This is a clearly a bug

Comments

@potiuk
Copy link
Member

potiuk commented Oct 31, 2022

Discussed in #27398

Originally posted by AndrewTsao October 25, 2022

Apache Airflow version

Other Airflow 2 version (please specify below)

What happened

My Airflow version 2.4.0. I use CronTriggerTimetable in my DAG. I find it lost one task occasionally. I inspected schedule log,

It should be run at 2022-10-24T23:10:00.000+0800. BUT not run.

[2022-10-24T23:08:59.661+0800] {processor.py:768} INFO - DAG(s) dict_keys(['AFlow.ajob']) retrieved from /home/bob/airflow/dags/dag.py
[2022-10-24T23:08:59.685+0800] {logging_mixin.py:117} INFO - [2022-10-24T23:08:59.684+0800] {dag.py:2573} INFO - Sync 1 DAGs
[2022-10-24T23:08:59.710+0800] {logging_mixin.py:117} INFO - [2022-10-24T23:08:59.710+0800] {dag.py:3324} INFO - Setting next_dagrun for AFlow.ajob to 2022-10-24T15:10:00+00:00, run_after=2022-10-24T15:10:00+00:00
[2022-10-24T23:08:59.734+0800] {processor.py:178} INFO - Processing /home/bob/airflow/dags/dag.py took 0.351 seconds
[2022-10-24T23:09:29.855+0800] {processor.py:156} INFO - Started process (PID=16329) to work on /home/bob/airflow/dags/dag.py
[2022-10-24T23:09:29.856+0800] {processor.py:758} INFO - Processing file /home/bob/airflow/dags/dag.py for tasks to queue
[2022-10-24T23:09:29.857+0800] {logging_mixin.py:117} INFO - [2022-10-24T23:09:29.857+0800] {dagbag.py:525} INFO - Filling up the DagBag from /home/bob/airflow/dags/dag.py
[2022-10-24T23:09:29.946+0800] {processor.py:768} INFO - DAG(s) dict_keys(['AFlow.ajob']) retrieved from /home/bob/airflow/dags/dag.py
[2022-10-24T23:09:29.969+0800] {logging_mixin.py:117} INFO - [2022-10-24T23:09:29.968+0800] {dag.py:2573} INFO - Sync 1 DAGs
[2022-10-24T23:09:29.994+0800] {logging_mixin.py:117} INFO - [2022-10-24T23:09:29.994+0800] {dag.py:3324} INFO - Setting next_dagrun for AFlow.ajob to 2022-10-24T15:10:00+00:00, run_after=2022-10-24T15:10:00+00:00
[2022-10-24T23:09:30.178+0800] {processor.py:178} INFO - Processing /home/bob/airflow/dags/dag.py took 0.327 seconds
[2022-10-24T23:10:00.368+0800] {processor.py:156} INFO - Started process (PID=16562) to work on /home/bob/airflow/dags/dag.py
[2022-10-24T23:10:00.369+0800] {processor.py:758} INFO - Processing file /home/bob/airflow/dags/dag.py for tasks to queue
[2022-10-24T23:10:00.370+0800] {logging_mixin.py:117} INFO - [2022-10-24T23:10:00.370+0800] {dagbag.py:525} INFO - Filling up the DagBag from /home/bob/airflow/dags/dag.py
[2022-10-24T23:10:00.628+0800] {processor.py:768} INFO - DAG(s) dict_keys(['AFlow.ajob']) retrieved from /home/bob/airflow/dags/dag.py
[2022-10-24T23:10:00.648+0800] {logging_mixin.py:117} INFO - [2022-10-24T23:10:00.647+0800] {dag.py:2573} INFO - Sync 1 DAGs
[2022-10-24T23:10:00.670+0800] {logging_mixin.py:117} INFO - [2022-10-24T23:10:00.670+0800] {dag.py:3324} INFO - Setting next_dagrun for AFlow.ajob to 2022-10-25T15:10:00+00:00, run_after=2022-10-25T15:10:00+00:00
[2022-10-24T23:10:00.693+0800] {processor.py:178} INFO - Processing /home/bob/airflow/dags/dag.py took 0.331 seconds

What you think should happen instead

I think it lost schedule infomation when reload dag file and reschedule dag.

How to reproduce

  1. Change airflow.cfg ag_dir_list_interval = 10.
  2. Add DAG as follow,
from airflow import DAG
from airflow.timetables.trigger import CronTriggerTimetable
from airflow.decorators import task
import pendulum as pl

with DAG(
    dag_id = 'test-cron-2',
    # schedule_interval='10 23 * * 2-3',
    timetable=CronTriggerTimetable('* * * * *', timezone=pl.tz.get_local_timezone()),  # At 01:00 on Wednesday
    start_date=pl.today().add(days=-21),
    tags=['example'],
) as dag1:
    @task
    def test():
        print("run...")

    test()

20221025132642

In scheduler log, We can find on that point, scheduler is reloading dag file.

20221025133158

Operating System

Centos 7.9

Versions of Apache Airflow Providers

airflow-code-editor                      7.0.0
apache-airflow                           2.4.0
apache-airflow-providers-celery          3.0.0
apache-airflow-providers-common-sql      1.1.0
apache-airflow-providers-ftp             3.1.0
apache-airflow-providers-http            4.0.0
apache-airflow-providers-imap            3.0.0
apache-airflow-providers-microsoft-psrp  2.0.0
apache-airflow-providers-microsoft-winrm 3.0.0
apache-airflow-providers-mysql           3.2.0
apache-airflow-providers-redis           3.0.0
apache-airflow-providers-samba           4.0.0
apache-airflow-providers-sftp            4.0.0
apache-airflow-providers-sqlite          3.2.0
apache-airflow-providers-ssh             3.1.0

Deployment

Virtualenv installation

Deployment details

No response

Anything else

No response

Are you willing to submit PR?

  • Yes I am willing to submit a PR!

Code of Conduct

@potiuk potiuk added kind:bug This is a clearly a bug area:core labels Oct 31, 2022
@potiuk
Copy link
Member Author

potiuk commented Oct 31, 2022

Mistakenly converted to discussion. Seems legit. This is likely only happening in 1 minute interval, so not really critical but woudl be nice to find out when it happens. - thanks @andi for detailed repot.

Could you please check @andi if it appears in the latest version of Airflow and maybe pinpoint some other cicumstances when it happens? How often? Do you see some usage spikes (memory/cpu etc. around that time? do you see any unsual task restarts etc. - we need to try to get some really reproducible case in order to be able to track tha one - otherwise it is a pure quess.

Can you try to see much more information @andi ?

@uranusjr
Copy link
Member

Hmm, this and #27582 seems to be the same issue?

@potiuk
Copy link
Member Author

potiuk commented Nov 10, 2022

I think the other issue is "first" schedule - and this one is "missing one in the middle" - it might be the same but it might be different.

@bluek1te
Copy link
Contributor

bluek1te commented Nov 15, 2022

Hey all, we had an issue in our configuration where our DAGs were reloading by the order of milliseconds (looking at last_parsed_time in the database) and we observed this issue happening a lot. The only DAG that failed to run in this circumstance were the ones that used CronTriggerTimetable. (We were using a test that should run every minute). I believe that if you set the DAG reload time very, very low, you will see this problem occur much more frequently.

When we fixed the reload issue in the configuration, we have noticed this problem much less frequently.

@bluek1te
Copy link
Contributor

bluek1te commented Nov 15, 2022

Steps to reproduce this (with other bug that reloads DAGs in order of milliseconds):

  • Set environment variable TZ to UTC. (Or leave unset if UTC by default)
  • Set environment variable AIRFLOW__CORE__DEFAULT_TIMEZONE=America/Chicago
  • Create new DAG (this DAG must not currently be in the database) that uses CronTriggerTimetable, we used this:
with DAG(
    'cron-trigger-test-repeat-new',
    description='cron-trigger-reproduce-bug',
    catchup=False,
    schedule=CronTriggerTimetable(f'* * * * *', 
        timezone='America/Chicago'),
    default_args = {
        'start_date': datetime(2022, 1, 1),
        'depends_on_past': False,
        'timezone': "America/Chicago"
    },
    is_paused_upon_creation=False
) as echo:
    hello = BashOperator(
        task_id="echo",
        bash_command="echo Hello World!"
    )
    echo
  • Wait for DAG to be picked up by the scheduler
  • Observe that the last_parsed_time in the dag table in airflow database (we use postgres) is updating wayyyy faster than it needs to (multiple times per second).
  • Observe that the DAG has a lot of missed runs every minute (we get less than 50% to trigger on schedule when we do this)

@johnwarburton
Copy link

Can confirm when trying to schedule a DAG exactly one with CronTriggerTimetable airflow [v2.4.2] in standalone mode (where dag_dir_list_interval = 300 seems to be ignored)

import subprocess
from airflow import DAG
from airflow.decorators import dag, task
from airflow.timetables.trigger import CronTriggerTimetable
from datetime import datetime

@task
def mytask():
  bash_command=f'/bin/echo run once'
  EXEC_OUT=(subprocess.check_output(bash_command, shell=True)).decode('utf-8')
  
@dag(
  dag_id=f'run_once',
  start_date=datetime(2022,1,1),
  timetable=CronTriggerTimetable('03 18 16 11 *', timezone='Australia/Sydney'),
  catchup=False,
)
def run_once_dag():
  mytask()
  
run_once_dag()  

In the minute before 18:03, the DAG was processed 22 times

The first processing at 18:03, the next_dagrun was set to the same time one year later, without executing the DAG

[2022-11-16T18:02:50.448+1100] {processor.py:154} INFO - Started process (PID=11018) to work on /export/home/jwarburt/airflow-venv/dags/run-once.py
[2022-11-16T18:02:50.506+1100] {logging_mixin.py:120} INFO - [2022-11-16T18:02:50.506+1100] {dag.py:3336} INFO - Setting next_dagrun for run_once to 2022-11-16T07:03:00+00:00, run_after=2022-11-16T07:03:00+00:00
[2022-11-16T18:02:50.609+1100] {processor.py:176} INFO - Processing /export/home/jwarburt/airflow-venv/dags/run-once.py took 0.164 seconds
[2022-11-16T18:02:53.836+1100] {processor.py:154} INFO - Started process (PID=11026) to work on /export/home/jwarburt/airflow-venv/dags/run-once.py
[2022-11-16T18:02:53.894+1100] {logging_mixin.py:120} INFO - [2022-11-16T18:02:53.894+1100] {dag.py:3336} INFO - Setting next_dagrun for run_once to 2022-11-16T07:03:00+00:00, run_after=2022-11-16T07:03:00+00:00
[2022-11-16T18:02:54.004+1100] {processor.py:176} INFO - Processing /export/home/jwarburt/airflow-venv/dags/run-once.py took 0.171 seconds
[2022-11-16T18:02:59.105+1100] {processor.py:154} INFO - Started process (PID=11032) to work on /export/home/jwarburt/airflow-venv/dags/run-once.py
[2022-11-16T18:02:59.163+1100] {logging_mixin.py:120} INFO - [2022-11-16T18:02:59.163+1100] {dag.py:3336} INFO - Setting next_dagrun for run_once to 2022-11-16T07:03:00+00:00, run_after=2022-11-16T07:03:00+00:00
[2022-11-16T18:02:59.272+1100] {processor.py:176} INFO - Processing /export/home/jwarburt/airflow-venv/dags/run-once.py took 0.171 seconds
[2022-11-16T18:03:00.699+1100] {processor.py:154} INFO - Started process (PID=11034) to work on /export/home/jwarburt/airflow-venv/dags/run-once.py
[2022-11-16T18:03:00.845+1100] {logging_mixin.py:120} INFO - [2022-11-16T18:03:00.845+1100] {dag.py:3336} INFO - Setting next_dagrun for run_once to 2023-11-16T07:03:00+00:00, run_after=2023-11-16T07:03:00+00:00
[2022-11-16T18:03:00.865+1100] {processor.py:176} INFO - Processing /export/home/jwarburt/airflow-venv/dags/run-once.py took 0.169 seconds

If I could slow down the processing interval, that might help?

@potiuk potiuk added this to the Airflow 2.4.4 milestone Nov 16, 2022
@ephraimbuddy ephraimbuddy modified the milestones: Airflow 2.4.4, Airflow 2.5.0, Airflow 2.5.1 Nov 23, 2022
@RNHTTR
Copy link
Contributor

RNHTTR commented Nov 30, 2022

@johnwarburton Have you had any luck with this? Seeing the same issue in 2.4.2 after switching from a plain cron expression (i.e. schedule_interval=0 15 * * *) to using CronTriggerTimetable directly with a timezone (i.e. timetable=CronTriggerTimetable("0 15 * * *", timezone="America/Los_Angeles")). I noticed that all of the examples in this thread have a timezone attached.

Some more context:

  • Kubernetes Executor
  • This isn't a case of a single DAG with a very frequently running cron expression like * * * * *, but it does have a lot of DAGs that all run at the same time.

@johnwarburton
Copy link

Hi @RNHTTR - no, I am waiting to see if there is a fix in the milestones

CronTriggerTimetable must have a timezone attached:

   |     timetable=CronTriggerTimetable('45 4 2 12 *'),
   | TypeError: __init__() missing 1 required keyword-only argument: 'timezone'

@RNHTTR
Copy link
Contributor

RNHTTR commented Dec 2, 2022

Apologies -- I thought that using a cron expression such as schedule_interval=0 15 * * * uses CronTriggerTimetable under the hood, but it looks like it actually users CronDataIntervalTimetable

@fatmumuhomer
Copy link
Contributor

fatmumuhomer commented Dec 5, 2022

I was chatting with Jarek earlier today on Slack about a similar issue I saw over the weekend and he pointed me to this open one.

schedule=CronTriggerTimetable('15 12 * * *', timezone=pendulum.timezone('America/New York'))

In my scenario I had deployed the DAG late on Friday and manually triggered it since the next run wasn't scheduled until Saturday. It then successfully scheduled on Saturday but not on Sunday or today (Monday). I am going through logs and will see if it runs on Tuesday.

@RNHTTR
Copy link
Contributor

RNHTTR commented Dec 6, 2022

One commonality is all timezones are non-utc. Not sure if that's indicative of something, but I think it's worth noting.

@scodesido-at-proton
Copy link

We have observed the same issue with a UTC timezone, so I don't think it is the issue, e.g.

timetable=CronTriggerTimetable("*/15 * * * *", timezone="UTC")

Are there any tips on how to deal with this reliably, or should we just expect CronTriggerTimetable to silently fail every now and then? In that case, maybe it would be good to update the docs here to warn about the inconsistent behaviour?

@potiuk
Copy link
Member Author

potiuk commented Dec 27, 2022

Also observed in #28293

@isaac-florence
Copy link

@potiuk do you have any advice on how to proceed with managing this bug?

@potiuk
Copy link
Member Author

potiuk commented Dec 29, 2022

No. Not until it gets investigated and fixed.

@Gollum999
Copy link
Contributor

For what it's worth, prior to 2.4 I wrote my own timetable that acts almost exactly like CronTriggerTimetable, and I experienced a similar issue, so I'll mention my findings here.

In my case the problem was that there was a race condition between the DagProcessor and the Scheduler when calling Timetable.next_dagrun_info. Depending on when each thread evaluated the timetable, the DagProcessor could end up "bumping" the DAG's next_dagrun_create_after timestamp in the DB before the Scheduler had a chance to schedule the current interval. Then when the Scheduler would do it's query to determine which DagRuns to schedule, it wouldn't find the run that got "skipped". In my case, I believe the bug was a call to DateTime.utcnow() that I had to align to the previous interval boundary.

Also worth noting that my bug only happened with catchup=False, but that could have just been a quirk with my implementation.

@potiuk
Copy link
Member Author

potiuk commented Mar 4, 2023

cc: @uranusjr ? WDYT this looks plausibke I guess ^^

@mai-nakagawa
Copy link
Contributor

mai-nakagawa commented Apr 28, 2023

My gut feeling is also that the root cause is somewhere in DagProcessor and/or Scheduler.

Anyway this problem doesn't happen in the CronDataIntervalTimetable. I think the reason is because CronDataIntervalTimetable runs a last DAG run if it missed even when catchup=False.

Maybe we need to implement the similar functionality in CronTriggerTimetable, like a anacron. What do you think? @uranusjr @potiuk

@potiuk
Copy link
Member Author

potiuk commented Apr 29, 2023

Hard to say.

@LMnet
Copy link
Contributor

LMnet commented May 2, 2023

May it be somehow related to the fact that CronTriggerTimetable by default has an empty date interval? As far as I understand most (if not all) built-in timetables have non-empty date intervals.

To test this hypothesis I set up small (10 minutes) date intervals for my daily jobs. Will report how it goes when I have some results.

@uranusjr
Copy link
Member

uranusjr commented May 3, 2023

I don’t think the interval is relevant; the scheduler never looks at it, but only pass it directly into a run’s context.

@uranusjr
Copy link
Member

uranusjr commented May 3, 2023

This is the logic to calculate the next run for catchup=False, for reference:

start_time_candidates = [self._align_to_next(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:
    start_time_candidates.append(self._align_to_next(restriction.earliest))
next_start_time = max(start_time_candidates)

The two if blocks should be fine since they match the logic for catchup=True. So I wonder if the problem is from the max call—maybe under some edge cases that max would wrongly select the next(utcnow) result (first line) instead of the actual schedule it should use—when the scheduler took too much time to schedule the previous run, perhaps?

Also worth noting that CronDataIntervalTimetable (the old one) uses slightly different logic to account for the current time, and that timetable seems to be bug-free.

@LMnet
Copy link
Contributor

LMnet commented May 21, 2023

May it be somehow related to the fact that CronTriggerTimetable by default has an empty date interval? As far as I understand most (if not all) built-in timetables have non-empty date intervals.

To test this hypothesis I set up small (10 minutes) date intervals for my daily jobs. Will report how it goes when I have some results.

FYI: even with the non-zero date interval I still faced the issue. So, this is not the root cause of this problem.

Overall it looks like the only real workaround for now is to not use CronTriggerTimetable at all.

But there is a bigger problem with this bug in my opinion. Since we are not 100% sure what is the root cause, all other timetables may be affected by this bug too. And since the main purpose of Airflow is to reliably schedule jobs, I think this is a major problem: with this bug we can't say that Airflow guarantees that user's jobs will be scheduled as requested. They can be randomly skipped. And even if only CronTriggerTimetable is affected, users can create custom timetables, and they could be affected by this bug too.

@uranusjr
Copy link
Member

Contributions are always welcomed.

@uranusjr
Copy link
Member

I spent some time to take a closer look at the implementation. The problem with

I think the reason is because CronDataIntervalTimetable runs a last DAG run if it missed even when catchup=False

is that CronDataIntervalTimetable actually does not do that! The reason it seems to be more resillient is that catchup in that time table relies on the data interval start, not the trigger time. So say

  • You have a cron
  • Last run on 3am
  • Current time 4:05

For CronDataIntervalTimetable, since the last run covered 2–3am, the next run should cover 3–4am, which can be achieved without catchup (the further next covers 4–5am and is not due yet). Everything is fine. But for CronTriggerTimetable, the last run covered 3am, but the 4am should be skipped since that’s already in the past.

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.

I can prepare a PR if that sounds reasonable (or anyone can, it’s just one line in airflow.timetables and fixing the corresponding test case).

@uranusjr
Copy link
Member

uranusjr commented Jul 17, 2023

Forgot to mentioned the one-line change. This line

start_time_candidates = [self._align_to_next(DateTime.utcnow())]

should be changed to use _align_to_prev instead.

@uranusjr uranusjr self-assigned this Jul 18, 2023
@uranusjr uranusjr added AIP-39 Timetables and removed area:core labels Jul 18, 2023
@mai-nakagawa
Copy link
Contributor

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. The new behavior might be unnatural to those who are not familiar with Anacron. Say

  • You have an hourly cron
  • Last run on 3am
  • You paused the cron DAG at 3:55 to skip the next run on 4am
  • You unpaused it at 4:30
  • The skipped run on 4am resumes (This might be unexpected)

@uranusjr
Copy link
Member

Yeah. Or maybe we can addd a flag for this? Not sure what it should be called though.

@shmcsensei
Copy link

I've been testing the following change to fix this issue seems to be working. Its similar to what has been suggested:

def next_dagrun_info(
    self,
    *,
    last_automated_data_interval: Optional[DataInterval],
    restriction: TimeRestriction,
) -> Optional[DagRunInfo]:
    if restriction.catchup:
        if last_automated_data_interval is None:
            if restriction.earliest is None:
                return None
            next_start_time = self._align_to_next(restriction.earliest)
        else:
            next_start_time = self._get_next(last_automated_data_interval.end)
    else:
        current_time = DateTime.utcnow()
        if restriction.earliest is not None and current_time < restriction.earliest:
            next_start_time = self._align_to_next(restriction.earliest)
        else:
            if last_automated_data_interval is None:
                # The DAG has never run before.
                next_start_time = self._align_to_next(current_time)
            elif last_automated_data_interval.end == self._align_to_prev(current_time):
                # The last run is the same as the previously scheduled run. Everything is working fine.
                next_start_time = self._align_to_next(current_time)
            else:
                # The last run is NOT the same as the previously scheduled run, we must have skipped it.
                next_start_time = self._align_to_prev(current_time)
                LOGGER.warn('DAG skipped previous schedule, rescheduling. last_automated_data_interval.end:"%s" next_start_time:"%s"', last_automated_data_interval.end, next_start_time)

    if restriction.latest is not None and restriction.latest < next_start_time:
        return None
    return DagRunInfo.exact(next_start_time)

Specifically this addition:

 if last_automated_data_interval is None:
    # The DAG has never run before.
    next_start_time = self._align_to_next(current_time)
elif last_automated_data_interval.end == self._align_to_prev(current_time):
    # The last run is the same as the previously scheduled run. Everything is working fine.
    next_start_time = self._align_to_next(current_time)
else:
    # The last run is NOT the same as the previously scheduled run, we must have skipped it.
    next_start_time = self._align_to_prev(current_time)
    LOGGER.warn('DAG skipped previous schedule, rescheduling. last_automated_data_interval.end:"%s" next_start_time:"%s"', last_automated_data_interval.end, next_start_time)

I added the logging so I could monitor it and alert if it happens, just in case. Not sure if this could be done better though, I'm open to suggestions.

@uranusjr
Copy link
Member

elif last_automated_data_interval.end == self._align_to_prev(current_time):
    # The last run is the same as the previously scheduled run. Everything is working fine.
    next_start_time = self._align_to_next(current_time)

This part has a bug when current_time is exactly last_automated_data_interval.end, next_start_time would be set to current_time again (because the align functions will try to not move the time if it already lands on the cron). Very unlikely but still. I think it can be avoided with self._get_next(last_automated_data_interval.end) instead, to ensure the logic progresses one interval ahead.

The else block is essentially the same as the change discused above and looks good to me.

@shmcsensei
Copy link

Thanks @uranusjr for the feedback! I'll look into it, good points.

@BohdanSemonov
Copy link

BohdanSemonov commented Jul 26, 2023

The bug occurs due to the fact that seconds and microseconds are not reset here: start_time_candidates = [self._align_to_next(DateTime.utcnow())] and a condition if self._get_prev(next_time) != current: in the method _align_to_next chooses next_time instead current, when the current DateTime matches with the next_dagrun before the DagProcessor will process the DAG. Accordingly, the date switching occurs earlier than the DAG launching.

The bug fix is only one line:
start_time_candidates = [self._align_to_next(DateTime.utcnow().replace(second=0, microsecond=0))]
I've already tested the fix - it works properly.

@snippins
Copy link

@BohdanSemonov I tried to applied your fix and do the following test.

  • current time is 10:00
  • switch the cron expression to '2 10 * * *'
  • wait for scheduler to update (done around 10:01)
  • pause the dag
  • wait until 10:03
  • resume the dag

The dag start to run, maybe I misunderstand something but I thought the entire point of CronTriggerTimetable is to prevent this?

@BohdanSemonov
Copy link

@snippins
The situation you describe is the Airflow feature when catchup=True.
Try with catchup=False.

@Tindarid
Copy link

Hey @BohdanSemonov , if the fix works, could you please submit a PR? I think this issue is quite critical, and I also observe such behavior from CronTriggerTimetable

@snippins
Copy link

The situation you describe is the Airflow feature when catchup=True.

I have catchup=False already.

@BohdanSemonov
Copy link

The situation you describe is the Airflow feature when catchup=True.

I have catchup=False already.

Sorry, maybe we didn't understand each other, but I observe such behavior when catchup=True, otherwise I could not get the described DAG run.

@BohdanSemonov
Copy link

Hey @BohdanSemonov , if the fix works, could you please submit a PR? I think this issue is quite critical, and I also observe such behavior from CronTriggerTimetable

Done.

@RNHTTR
Copy link
Contributor

RNHTTR commented Oct 19, 2023

I'm closing this as it appears to have been fixed in 2.7.1 by #33404 but there was a typo in the description that allowed this Issue to stay open.

@RNHTTR RNHTTR closed this as completed Oct 19, 2023
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
AIP-39 Timetables kind:bug This is a clearly a bug
Projects
None yet
Development

Successfully merging a pull request may close this issue.