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

successful DAG run fails to be scheduled after being manually cleared if Dag.dagrun_timeout is set #14265

Closed
YangMuye opened this issue Feb 17, 2021 · 8 comments
Labels
affected_version:2.0 Issues Reported for 2.0 area:Scheduler including HA (high availability) scheduler kind:bug This is a clearly a bug
Milestone

Comments

@YangMuye
Copy link

YangMuye commented Feb 17, 2021

Apache Airflow version: 2.0.0

Kubernetes version (if you are using kubernetes) (use kubectl version):

Environment:

  • Cloud provider or hardware configuration:
  • OS (e.g. from /etc/os-release): redhat 7.9
  • Kernel (e.g. uname -a): 3.10.0-1160.11.1.el7.x86_64
  • Install tools: pip
  • Others:

What happened:

I cleared a success DAG run and it failed to be scheduled again with the following error message:

{scheduler_job.py:1639} INFO - Run scheduled__2021-02-02T16:00:00+00:00 of some_job has timed-out

After removing dagrun_timeout, the same dag run can be rescheduled.

What you expected to happen:

I expect "timeout" is counted from the moment the DAG is reset.

Anything else we need to know:

I can reproduce it with the following code. Not sure if it is the intended behavior.

from airflow import DAG
from airflow.operators.dummy import DummyOperator
import pendulum
with DAG("test_timeout",
    default_args={'owner': 'airflow'},
    start_date= pendulum.yesterday(),
    schedule_interval='@daily',
    # add the following parameter after the first run is complete and then clear the success run
    # dagrun_timeout=pendulum.duration(minutes=1),
) as dag:
    DummyOperator(task_id='dummy')
@YangMuye YangMuye added the kind:bug This is a clearly a bug label Feb 17, 2021
@YangMuye
Copy link
Author

maybe a duplicate of #13853.

@vikramkoka vikramkoka added the affected_version:2.0 Issues Reported for 2.0 label Feb 17, 2021
@kaxil
Copy link
Member

kaxil commented Feb 22, 2021

Can you test it with 2.0.1 too please @YangMuye ?

@YangMuye
Copy link
Author

@kaxil tested with 2.0.1. It has the same issue.

[airflow@TEST01 test]$ airflow version
2.0.1
[airflow@TEST01 test]$ systemctl status airflow-scheduler | tail -2
Feb 25 21:36:26 TEST01 airflow[198583]: [2021-02-25 21:36:26,692] {dagrun.py:445} INFO - Marking run <DagRun test_timeout @ 2021-02-23 16:00:00+00:00: scheduled__2021-02-23T16:00:00+00:00, externally triggered: False> successful
Feb 25 21:37:27 TEST01 airflow[198583]: [2021-02-25 21:37:27,766] {scheduler_job.py:1722} INFO - Run scheduled__2021-02-23T16:00:00+00:00 of test_timeout has timed-out

@kaxil kaxil added this to the Airflow 2.0.2 milestone Feb 25, 2021
@YangMuye
Copy link
Author

YangMuye commented Mar 1, 2021

It seems that the dag run does not need to be successful first to hit this issue.
A dag run that fails due to timeout can never be rerun.

@mattellis
Copy link

mattellis commented Mar 5, 2021

@YangMuye @kaxil
Our users have the same issue. In case it's of use, I figured out that the models.dag.clear method being invoked when a dag run is cleared from the graph or tree view, does not set the activate_dag_runs flag to True in the models.taskinstance.clear_task_instances method, which is where the start_date value for the run is reset to now():

activate_dag_runs=False, # We will set DagRun state later.

if activate_dag_runs and tis:
...
    dr.start_date = timezone.utcnow()

dr.start_date = timezone.utcnow()

Alternatively, the browse DAG runs view uses a different path to clear dag runs, and is still effective in resetting the dagrun start date:

models.clear_task_instances(tis, session, dag=dag)

We've advised our users to use this method as a workaround for now, and set our normal 24 hour dag_run timeout to 7 days to cover most common use cases of clearing recently failed runs.

@ephraimbuddy
Copy link
Contributor

I suspect this will be resolved by #16401 when merged. Would wait for it to be merged and reproduce this because it touched on dr.start_date

@kaxil kaxil modified the milestones: Airflow 2.1.1, Airflow 2.1.2 Jun 22, 2021
@uranusjr
Copy link
Member

uranusjr commented Jul 5, 2021

Actually this was already fixed in main with #15382 (not sure if it’s intended though).

As mentioned above, the issue is caused by DAG.clear(), when calling clear_task_instances, does not reset a dagrun’s start_date. But after #15382, DagRun.start_date is unconditionally set, so the issue went away.

However, that patch is currently only in main, not the 2.1 branch, so we’ll need to figure out what to do for 2.1.2. Either backport a part of #15382, or implement a more band-aid fix for 2.1.x specifically, I think.

Wait, I was wrong, #15382 was backported to 2.1.1! Could someone verify whether the issue is still present in 2.1.1?

@ashb ashb modified the milestones: Airflow 2.1.2, Airflow 2.1.3 Jul 7, 2021
@kaxil
Copy link
Member

kaxil commented Jul 27, 2021

Closing -- fixed by #16401 -- which will be part of Airflow 2.1.3

@kaxil kaxil closed this as completed Jul 27, 2021
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
affected_version:2.0 Issues Reported for 2.0 area:Scheduler including HA (high availability) scheduler kind:bug This is a clearly a bug
Projects
None yet
Development

No branches or pull requests

7 participants