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

Tasks in DAGs with depends_on_past or task_concurrency are not being scheduled #12659

Closed
nathadfield opened this issue Nov 27, 2020 · 10 comments · Fixed by #12663
Closed

Tasks in DAGs with depends_on_past or task_concurrency are not being scheduled #12659

nathadfield opened this issue Nov 27, 2020 · 10 comments · Fixed by #12663
Assignees
Labels
kind:bug This is a clearly a bug

Comments

@nathadfield
Copy link
Collaborator

Apache Airflow version: 1.10.13

What happened:

After performing an upgrade to v1.10.13 we noticed that tasks in some of our DAGs were not be scheduled. After a bit of investigation we discovered that by commenting out 'depends_on_past': True the issue went away.

What you expected to happen:

We think the issue might have something to do with this which was introduced to 1.10.13

[AIRFLOW-3607] Only query DB once per DAG run for TriggerRuleDep (#4751)

How to reproduce it:

  1. Install Airflow v1.10.13 from pip
  2. Start webserver and scheduler
  3. Add the following code as a DAG
  4. Switch the DAG on in the UI.
from airflow import models
from airflow.operators.dummy_operator import DummyOperator
from datetime import datetime, timedelta

default_args = {
    'owner': 'airflow',
    'start_date': datetime(2018, 10, 31),
    'depends_on_past': True,
    'retries': 3,
    'retry_delay': timedelta(minutes=5)
}

dag_name = 'my-test-dag'

with models.DAG(dag_name,
                default_args=default_args,
                schedule_interval='0 0 * * *',
                catchup=False,
                max_active_runs=5,
                ) as dag:

    test = DummyOperator(
        task_id='test'
    ) 
@nathadfield nathadfield added the kind:bug This is a clearly a bug label Nov 27, 2020
@kaxil kaxil added this to the Airflow 1.10.14 milestone Nov 27, 2020
@kaxil
Copy link
Member

kaxil commented Nov 27, 2020

On Master, this was fixed by #7402 & further optimised by #7503

in 1.10.13 -- this was clubbed by the following 2 commits:

I will investigate this further

@kaxil
Copy link
Member

kaxil commented Nov 27, 2020

@nathadfield has confirmed the issue does not exist on 2.0.0b3

@mthoretton
Copy link

Since v1.10.13 we also noticed, for some dags, the tasks are not being scheduled. They stay forever with a None state. Nothing in the scheduler logs (DEBUG level). Running the tasks manually work fine though. In our case, some/most of the dags have indeed depends_on_past set to true but not all of them it seems. So maybe this is something different. I will try to investigate deeper and share any relevant info.

@kaxil
Copy link
Member

kaxil commented Nov 27, 2020

Since v1.10.13 we also noticed, for some dags, the tasks are not being scheduled. They stay forever with a None state. Nothing in the scheduler logs (DEBUG level). Running the tasks manually work fine though. In our case, some/most of the dags have indeed depends_on_past set to true but not all of them it seems. So maybe this is something different. I will try to investigate deeper and share any relevant info.

Can you check if the other DAGs (not using depends_on_past but are still stuck) have task_concurrency set? @mthoretton

@mthoretton
Copy link

I did not check absolutely all dags we have but yes, the "broken" dags either have depends_on_past or task_concurrency set. I was not aware of the 2 issues you mentionened above, I will have a look but it definitely looks related.

@kaxil kaxil self-assigned this Nov 27, 2020
@kaxil kaxil changed the title Tasks in DAGs with depends_on_past are not being scheduled Tasks in DAGs with depends_on_past or task_concurrency are not being scheduled Nov 27, 2020
@kaxil
Copy link
Member

kaxil commented Nov 27, 2020

I can confirm the bug. I was able to reproduce it with task with task_concurrency or depends_on_past with LocalExecutor and the following DAG:

from airflow import models
from airflow.operators.dummy_operator import DummyOperator
from airflow.operators.bash_operator import BashOperator
from datetime import datetime, timedelta

default_args = {
    'owner': 'airflow',
    'start_date': datetime(2018, 10, 31),
    'retries': 3,
    'retry_delay': timedelta(minutes=5)
}

dag_name = 'dag-bugcheck'

with models.DAG(dag_name,
                default_args=default_args,
                schedule_interval='0 0 * * *',
                catchup=False,
                max_active_runs=5,
                ) as dag:

    test1 = DummyOperator(
        task_id='test1',
        task_concurrency=10,
    )

    test2 = BashOperator(
        task_id='test2',
        bash_command='echo hi',
        depends_on_past=True,
    )

    test3 = BashOperator(
        task_id='test3',
        bash_command='echo hi',
    )

@kaxil
Copy link
Member

kaxil commented Nov 27, 2020

#12663 should fix it @nathadfield @mthoretton

@nathadfield
Copy link
Collaborator Author

Nice one @kaxil! Will this force the need for a 1.10.14 then?

@kaxil
Copy link
Member

kaxil commented Nov 27, 2020

Nice one @kaxil! Will this force the need for a 1.10.14 then?

Yup, indeed. I hope to get it out by early next week

kaxil added a commit to astronomer/airflow that referenced this issue Nov 27, 2020
@kaxil kaxil closed this as completed Nov 27, 2020
@kaxil
Copy link
Member

kaxil commented Nov 27, 2020

Closed by #12663

kaxil added a commit to astronomer/airflow that referenced this issue Dec 4, 2020
closes apache#12659

(cherry picked from commit 67807ee)
(cherry picked from commit 56e4468)
cfei18 pushed a commit to cfei18/incubator-airflow that referenced this issue Mar 5, 2021
leahecole pushed a commit to GoogleCloudPlatform/composer-airflow that referenced this issue Jun 9, 2021
…663)

closes apache/airflow#12659

GitOrigin-RevId: 2a7944da85c3ec7cc913640ff89a66e06d486480
leahecole pushed a commit to GoogleCloudPlatform/composer-airflow that referenced this issue Jun 9, 2021
…663)

closes apache/airflow#12659

GitOrigin-RevId: 2a7944da85c3ec7cc913640ff89a66e06d486480
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
kind:bug This is a clearly a bug
Projects
None yet
Development

Successfully merging a pull request may close this issue.

3 participants