Skip to content

Airflow scheduler stopped working #35272

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

Closed
2 tasks done
stijndehaes opened this issue Oct 30, 2023 · 14 comments
Closed
2 tasks done

Airflow scheduler stopped working #35272

stijndehaes opened this issue Oct 30, 2023 · 14 comments
Assignees
Labels
area:core area:Scheduler including HA (high availability) scheduler kind:bug This is a clearly a bug Stale Bug Report

Comments

@stijndehaes
Copy link
Contributor

Apache Airflow version

2.7.2

What happened

When running a DAG with a timezone set to Europe/Brussels we noticed it stopped being scheduled around the 29th, this is right around summer/winter time change. The scheduled logs show:

test-airflow-scheduler-1  | [2023-10-30T12:49:52.330+0000] {dag.py:3722} INFO - Setting next_dagrun for time-zone-issue to 2023-10-29T04:00:00+00:00, run_after=2023-10-29T05:00:00+00:00
test-airflow-scheduler-1  | [2023-10-30T12:49:53.474+0000] {dag.py:3722} INFO - Setting next_dagrun for time-zone-issue to 2023-10-28T21:45:00+00:00, run_after=2023-10-29T04:00:00+00:00
test-airflow-scheduler-1  | [2023-10-30T12:49:54.569+0000] {dag.py:3722} INFO - Setting next_dagrun for time-zone-issue to 2023-10-29T04:00:00+00:00, run_after=2023-10-29T05:00:00+00:00
test-airflow-scheduler-1  | [2023-10-30T12:49:55.665+0000] {dag.py:3722} INFO - Setting next_dagrun for time-zone-issue to 2023-10-28T21:45:00+00:00, run_after=2023-10-29T04:00:00+00:00
test-airflow-scheduler-1  | [2023-10-30T12:49:56.031+0000] {dag.py:3722} INFO - Setting next_dagrun for time-zone-issue to 2023-10-29T04:00:00+00:00, run_after=2023-10-29T05:00:00+00:00
test-airflow-scheduler-1  | [2023-10-30T12:49:57.133+0000] {dag.py:3722} INFO - Setting next_dagrun for time-zone-issue to 2023-10-28T21:45:00+00:00, run_after=2023-10-29T04:00:00+00:00
test-airflow-scheduler-1  | [2023-10-30T12:49:57.454+0000] {dag.py:3722} INFO - Setting next_dagrun for time-zone-issue to 2023-10-29T04:00:00+00:00, run_after=2023-10-29T05:00:00+00:00
test-airflow-scheduler-1  | [2023-10-30T12:49:58.584+0000] {dag.py:3722} INFO - Setting next_dagrun for time-zone-issue to 2023-10-28T21:45:00+00:00, run_after=2023-10-29T04:00:00+00:00
test-airflow-scheduler-1  | <jemalloc>: MADV_DONTNEED does not work (memset will be used instead)
test-airflow-scheduler-1  | <jemalloc>: (This is the expected behaviour if you are running under QEMU)
test-airflow-scheduler-1  | [2023-10-30T12:49:59.553+0000] {dag.py:3722} INFO - Setting next_dagrun for time-zone-issue to 2023-10-29T04:00:00+00:00, run_after=2023-10-29T05:00:00+00:00
test-airflow-scheduler-1  | [2023-10-30T12:50:00.667+0000] {dag.py:3722} INFO - Setting next_dagrun for time-zone-issue to 2023-10-28T21:45:00+00:00, run_after=2023-10-29T04:00:00+00:00
test-airflow-scheduler-1  | [2023-10-30T12:50:01.774+0000] {dag.py:3722} INFO - Setting next_dagrun for time-zone-issue to 2023-10-29T04:00:00+00:00, run_after=2023-10-29T05:00:00+00:00
test-airflow-scheduler-1  | [2023-10-30T12:50:02.580+0000] {dag.py:3722} INFO - Setting next_dagrun for time-zone-issue to 2023-10-28T21:45:00+00:00, run_after=2023-10-29T04:00:00+00:00
test-airflow-scheduler-1  | [2023-10-30T12:50:02.956+0000] {dag.py:3722} INFO - Setting next_dagrun for time-zone-issue to 2023-10-29T04:00:00+00:00, run_after=2023-10-29T05:00:00+00:00
test-airflow-webserver-1  | 127.0.0.1 - - [30/Oct/2023:12:50:03 +0000] "GET /health HTTP/1.1" 200 318 "-" "curl/7.74.0"
test-airflow-scheduler-1  | [2023-10-30T12:50:04.063+0000] {dag.py:3722} INFO - Setting next_dagrun for time-zone-issue to 2023-10-28T21:45:00+00:00, run_after=2023-10-29T04:00:00+00:00
test-airflow-scheduler-1  | [2023-10-30T12:50:05.167+0000] {dag.py:3722} INFO - Setting next_dagrun for time-zone-issue to 2023-10-29T04:00:00+00:00, run_after=2023-10-29T05:00:00+00:00
test-airflow-scheduler-1  | [2023-10-30T12:50:06.271+0000] {dag.py:3722} INFO - Setting next_dagrun for time-zone-issue to 2023-10-28T21:45:00+00:00, run_after=2023-10-29T04:00:00+00:00
test-airflow-scheduler-1  | [2023-10-30T12:50:07.382+0000] {dag.py:3722} INFO - Setting next_dagrun for time-zone-issue to 2023-10-29T04:00:00+00:00, run_after=2023-10-29T05:00:00+00:00
test-airflow-scheduler-1  | [2023-10-30T12:50:07.702+0000] {dag.py:3722} INFO - Setting next_dagrun for time-zone-issue to 2023-10-28T21:45:00+00:00, run_after=2023-10-29T04:00:00+00:00
test-airflow-scheduler-1  | [2023-10-30T12:50:08.811+0000] {dag.py:3722} INFO - Setting next_dagrun for time-zone-issue to 2023-10-29T04:00:00+00:00, run_after=2023-10-29T05:00:00+00:00
test-airflow-scheduler-1  | [2023-10-30T12:50:09.671+0000] {dag.py:3722} INFO - Setting next_dagrun for time-zone-issue to 2023-10-28T21:45:00+00:00, run_after=2023-10-29T04:00:00+00:00
test-airflow-scheduler-1  | [2023-10-30T12:50:10.793+0000] {dag.py:3722} INFO - Setting next_dagrun for time-zone-issue to 2023-10-29T04:00:00+00:00, run_after=2023-10-29T05:00:00+00:00
test-airflow-scheduler-1  | [2023-10-30T12:50:11.894+0000] {dag.py:3722} INFO - Setting next_dagrun for time-zone-issue to 2023-10-28T21:45:00+00:00, run_after=2023-10-29T04:00:00+00:00
test-airflow-scheduler-1  | [2023-10-30T12:50:12.813+0000] {dag.py:3722} INFO - Setting next_dagrun for time-zone-issue to 2023-10-29T04:00:00+00:00, run_after=2023-10-29T05:00:00+00:00

What you think should happen instead

The dag should continue to be scheduled

How to reproduce

Run the DAG attached below, I reproduced this case using the docker-compose setup from the docs.

from airflow.decorators import task

import pendulum
from airflow.models import DAG
from datetime import timedelta

default_args = {
    "depends_on_past": False,
    "start_date": pendulum.datetime(year=2023, month=10, day=28, tz="Europe/Brussels"),
    "email": [],
    "email_on_failure": False,
    "email_on_retry": False,
    "retries": 3,
    "retry_delay": timedelta(minutes=5),
}

with DAG(
        "time-zone-issue",
        default_args=default_args,
        max_active_runs=1,
        schedule_interval="0,15,30,45 6-22 * * *",
) as dag:
    @task(task_id="print_the_context")
    def print_context(ds=None, **kwargs):
        """Print the Airflow context and ds variable from the context."""
        print(ds)
        return "finished!"

    run_this = print_context()

Operating System

MAC OS, also linux

Versions of Apache Airflow Providers

Default ones included in docker-compose setup

Deployment

Docker-Compose

Deployment details

It also happens on a kubernetes cluster, the installation does not matter

Anything else

At one point the DAG started scheduling again, but I can still reproduce the case.

I am willing to create a PR but don't know where to start looking for this

Are you willing to submit PR?

  • Yes I am willing to submit a PR!

Code of Conduct

@stijndehaes stijndehaes added area:core kind:bug This is a clearly a bug needs-triage label for new issues that we didn't triage yet labels Oct 30, 2023
@erplus
Copy link

erplus commented Oct 30, 2023

I met this problem as well. Airflow version 2.5.1.
I have a second cluster with airflow 2.3.4. Everything seems are working properly though there are other dags and slightly different versions of python packages than in the first airflow cluster.

@pspeter
Copy link

pspeter commented Nov 2, 2023

We've also run into this problem. It only affects DAGs which have a start_time with a timezone, which have not gotten a new dag_run since the weekend. Our scheduler logs also look similar to OP.

The schedule for our affected DAGs is "15 7,8,13,19 * * *". Might be related since OP also uses a complex schedule.

@stijndehaes
Copy link
Contributor Author

stijndehaes commented Nov 2, 2023

I have created a drawing of what the current state is.
The schedule for this DAG is a bit complex: 50 5-11,13-23 * * *

New Drawing 7

So basically there is one dag run created (the middle one), that is questionable if it should ever be created.
The left and right one have either a questionable start of the interval or questionable end.

Also after creating these the scheduler stops, in the logs I can see the next dag run is calculated as 2023-10-28T22:50:00+00:00 which in CET is equal to 2023-10-28T23:50:00+01:00, this dag run already exists so it then calculated the next dag run from that one to: 2023-10-29T03:50:00+00:00 in CET: 2023-10-29T04:50:00+01:00 which also already exists, and it continues like this.

This test reproduces the calculation:

def test_new_calculate_next_time():
    timezone = "Europe/Brussels"
    tz = pendulum.tz.timezone(timezone)
    start = pendulum.DateTime(2023, 10, 29, hour=4, minute=50, tzinfo=tz)
    end = pendulum.DateTime(2023, 10, 29, hour=5, minute=50, tzinfo=tz)
    interval = DataInterval(start=start, end=end)
    table = CronDataIntervalTimetable("50 5-11,13-23 * * *", timezone)
    next_info = table.next_dagrun_info(last_automated_data_interval=interval, restriction=TimeRestriction(earliest=None, latest=None, catchup=True))
    expected_start = end
    expected_end = pendulum.DateTime(2023, 10, 29, hour=6, minute=50, tzinfo=tz)
    assert next_info == DagRunInfo.interval(start=expected_start, end=expected_end)

And fails to succeed. I would love some advice from someone at Airflow what would be the best behavior. @potiuk can you help me find someone I can discuss solutions with?

I believe it should never have created these 3 dag runs but instead should have created a dagrun with:

  • start: 23:50 CEST
  • end: 5:50 CET

Any thoughts?

As for a workardound I have two for people:

  • Set the start_date to after the issue
  • Update the next_dagrun, next_dagrun_data_interval_start, next_dagrun_data_interval_end manually to the correct next dag run

@potiuk
Copy link
Member

potiuk commented Nov 2, 2023

You've done all you could hopefully someone will take a look and help you. I am individual contributor and I have no power over telling other volunteers to do with their free time. But you described it well enough and I hope someone here will be able to help you. I keep fingers crossed.

@nathadfield nathadfield added area:Scheduler including HA (high availability) scheduler and removed needs-triage label for new issues that we didn't triage yet labels Nov 3, 2023
@ivan-afonichkin
Copy link
Contributor

@potiuk I'm happy to take a look :)

@ivan-afonichkin
Copy link
Contributor

@potiuk I've created a PR #35445, would be great if you can take a look and give some feedback as it's my first PR in the scheduler and I might be missing something. Thanks!

@timkpaine
Copy link
Contributor

this issue is a duplicate of #7999 and can be closed

@pspeter
Copy link

pspeter commented Nov 4, 2023

I am not sure if this is really a duplicate of #7999. It‘s related, but at least in our case the scheduler completely stopped executing that DAG until we removed the timezone from the start_date.

@nclaeys
Copy link
Contributor

nclaeys commented Nov 22, 2023

I just wanted to pitch in and make sure that a fix is provided for this issue. I see a long discussion on PR: #30083 on whether it is breaking existing functionality. On this I have 2 thoughts:

  • If I use a cron expression "0 9 * * *", I expect it always to run at 9 if I specify my local timezone irregardless of whether we switch from DST. Otherwise the dag code does not describe what the behavior is.
  • The more pressing is not that the schedule is 1 hour off but that it completely breaks the scheduling in some scenarios (as it pings between 1 hour when transitioning DST). This is really problematic and I just encountered this issue just again for one of our customers.

Please make sure that whatever solution you choose, calculating the next dagrun works across DST transitions when using a non UTC timezone.

@timkpaine
Copy link
Contributor

timkpaine commented Nov 24, 2023

@nclaeys ”0 9 * * *” should already work at 9 in the timezone of your DAG as it is a fixed schedule. The problem is for schedules like “0 9,10 * * *” which involve intervals, they will run at the same time relative to UTC (which means they shift with time changes, so e.g. not always at 9am local time). This is what the linked PR seeks to adjust.

It is possible that one of the unintended side effects of the current interval timetable behavior is that certain runs get swallowed up, as the schedule tries to calculate the “next” run without adjusting for DST, this run gets skipped and so subsequent runs dont get calculated or run until you jiggle the scheduler’s calculations by tweaking the inputs to the timetable (e.g. start date). This is just a theory (which I have also seen, so its not just the OP @stijndehaes / @pspeter ), but its also possible that this is completely unrelated to timezone stuff.

@nclaeys
Copy link
Contributor

nclaeys commented Nov 24, 2023

Yes you are right, but the crashing pipelines use a cron expression like "0 9,10 * * *", my first comment was more like what I expect when using timezones and cron expression as there is typically a reason why I use local timezone and business expects the pipeline to run at a given time regardless of whether it is DST or not.

The crashing issue is caused by how you handle timezones, which you can check by running the following test:

    dag_bag = DagBag(resources_path())
    dag = dag_bag.get_dag("demo")
    now = pendulum.now()
    before_dst = now.subtract(months=2)
    result = dag.iter_dagrun_infos_between(
        earliest=before_dst, latest=now
    )
    assert result is not None

If the dag I use has a timezone ("Europe/Brussels") and a cron expression as specified this test never finishes. If I run it using UTC it does work correctly.

@timkpaine
Copy link
Contributor

Should hopefully be resolved in #35887

Copy link

github-actions bot commented Dec 6, 2024

This issue has been automatically marked as stale because it has been open for 365 days without any activity. There has been several Airflow releases since last activity on this issue. Kindly asking to recheck the report against latest Airflow version and let us know if the issue is reproducible. The issue will be closed in next 30 days if no further activity occurs from the issue author.

Copy link

github-actions bot commented Jan 7, 2025

This issue has been closed because it has not received response from the issue author.

@github-actions github-actions bot closed this as not planned Won't fix, can't repro, duplicate, stale Jan 7, 2025
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
area:core area:Scheduler including HA (high availability) scheduler kind:bug This is a clearly a bug Stale Bug Report
Projects
None yet
Development

Successfully merging a pull request may close this issue.

8 participants
pFad - Phonifier reborn

Pfad - The Proxy pFad of © 2024 Garber Painting. All rights reserved.

Note: This service is not intended for secure transactions such as banking, social media, email, or purchasing. Use at your own risk. We assume no liability whatsoever for broken pages.


Alternative Proxies:

Alternative Proxy

pFad Proxy

pFad v3 Proxy

pFad v4 Proxy