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

Fix max_active_runs not allowing moving of queued dagruns to running #17945

Merged
merged 3 commits into from Sep 2, 2021

Conversation

ephraimbuddy
Copy link
Contributor

@ephraimbuddy ephraimbuddy commented Aug 31, 2021

Currently, if you set max_active_runs for a dag and that dag has many queued dagruns
with execution dates older than another dag's queued dagruns, airflow will not move
the newer queued dagruns to running with the effect that only one dagruns would be in running at any time

This PR fixes this by updating the DagRun.last_scheduling_decision whenever a decision of scheduling
was made

Related: #14205


^ Add meaningful description above

Read the Pull Request Guidelines for more information.
In case of fundamental code change, Airflow Improvement Proposal (AIP) is needed.
In case of a new dependency, check compliance with the ASF 3rd Party License Policy.
In case of backwards incompatible changes please leave a note in UPDATING.md.

@boring-cyborg boring-cyborg bot added the area:Scheduler Scheduler or dag parsing Issues label Aug 31, 2021
@ephraimbuddy ephraimbuddy changed the title Fix max_active_runs properly Fix max_active_runs not allowing scheduling of other dagruns Aug 31, 2021
@ephraimbuddy
Copy link
Contributor Author

ephraimbuddy commented Sep 2, 2021

What happens is that the method DagRun.next_dagruns_to_examine gets the earliest dagruns without considering the dag that has the dagrun.
For example:
If you have a dag with execution_date 2020,1,1 and set catchup=True, max_active_runs=1, schedule_interval='@daily' and another dag with execution_date 2021,1,1 and also set catchup=True, schedule_interval='@daily'.
When you unpause the two dags(the one with max_active_runs first), the dagruns would be created but only one dagrun would be active because of how DagRun.next_dagruns_to_examine works.
(worried about performance on this PR. Please advice)

airflow/models/dagrun.py Outdated Show resolved Hide resolved
airflow/jobs/scheduler_job.py Outdated Show resolved Hide resolved
airflow/jobs/scheduler_job.py Outdated Show resolved Hide resolved
@ashb
Copy link
Member

ashb commented Sep 2, 2021

will not schedule the newer dagruns with effect that only one dagruns would be in running at any time

When you say wont schedule -- do you mean that the dagruns will take a long time to get out of queued state, but once a dag run is in running state tasks should be scheduled fine -- cos the code you have changed only affects DagRuns going from queued -> running state -- nothing there will affect task scheduling.

@ephraimbuddy
Copy link
Contributor Author

will not schedule the newer dagruns with effect that only one dagruns would be in running at any time

When you say wont schedule -- do you mean that the dagruns will take a long time to get out of queued state, but once a dag run is in running state tasks should be scheduled fine -- cos the code you have changed only affects DagRuns going from queued -> running state -- nothing there will affect task scheduling.

You are right. Not scheduling but moving from queued to running, I will update it rightly now

Currently, if you set max_active_runs for a dag and that dag has many queued dagruns
with execution dates older than another dag's queued dagruns, airflow will not move
the newer queued dagruns to running with effect that only one dagruns would be in running at any time

This PR fixes this by updating the DagRun.last_scheduling_decision whenever a decision of scheduling
was made

Update airflow/jobs/scheduler_job.py

Co-authored-by: Tzu-ping Chung <uranusjr@gmail.com>

Correctly set dagrun state to running
@ephraimbuddy ephraimbuddy changed the title Fix max_active_runs not allowing scheduling of other dagruns Fix max_active_runs not allowing moving of queued dagruns to running Sep 2, 2021
@ashb ashb added this to the Airflow 2.1.4 milestone Sep 2, 2021
@github-actions github-actions bot added the full tests needed We need to run full set of tests for this PR to merge label Sep 2, 2021
@github-actions
Copy link

github-actions bot commented Sep 2, 2021

The PR most likely needs to run full matrix of tests because it modifies parts of the core of Airflow. However, committers might decide to merge it quickly and take the risk. If they don't merge it quickly - please rebase it to the latest main at your convenience, or amend the last commit of the PR, and push it with --force-with-lease.

@kaxil kaxil merged commit 430976c into apache:main Sep 2, 2021
@kaxil kaxil deleted the fix-max-active-runs branch September 2, 2021 12:14
ephraimbuddy added a commit to astronomer/airflow that referenced this pull request Sep 8, 2021
We made a fix that resolved max_active_runs not allowing other dagruns to move to
running state, see apache#17945 and introduced a bug that dagruns were not following the
execution_date order when moving to running state.

This PR fixes it by adding a 'max_active_runs` column in dagmodel. Also an extra test
not connected with this change was added because I was able to trigger the bug while
working on this

fixup! Fix DagRun execution order not being properly followed

fixup! fixup! Fix DagRun execution order not being properly followed

fixup! Fix DagRun execution order not being properly followed

fixup! fixup! Fix DagRun execution order not being properly followed

fixup! Fix DagRun execution order not being properly followed

Use subquery as mysql 5.7 doesn't support cte

fix doc error

Apply suggestions from code review
ephraimbuddy added a commit that referenced this pull request Sep 9, 2021
…followed (#18061)

We made a fix that resolved max_active_runs not allowing other dagruns to move to
running state, see #17945 and introduced a bug that dagruns were not following the
execution_date order when moving to running state.

This PR fixes it by adding a 'max_active_runs` column in dagmodel. Also an extra test
not connected with this change was added because I was able to trigger the bug while
working on this
kaxil pushed a commit that referenced this pull request Sep 10, 2021
…17945)

Currently, if you set max_active_runs for a dag and that dag has many queued dagruns
with execution dates older than another dag's queued dagruns, airflow will not move
the newer queued dagruns to running with the effect that only one dagruns would be in running at any time

This PR fixes this by updating the DagRun.last_scheduling_decision whenever a decision of scheduling
was made

(cherry picked from commit 430976c)
kaxil pushed a commit that referenced this pull request Sep 10, 2021
…followed (#18061)

We made a fix that resolved max_active_runs not allowing other dagruns to move to
running state, see #17945 and introduced a bug that dagruns were not following the
execution_date order when moving to running state.

This PR fixes it by adding a 'max_active_runs` column in dagmodel. Also an extra test
not connected with this change was added because I was able to trigger the bug while
working on this

(cherry picked from commit ebbe2b4)
@dhuang
Copy link
Contributor

dhuang commented Sep 22, 2021

FWIW can confirm this fixed my issue in #17638, so big thanks! Note I did have an issue after upgrading where the max_active_runs columns for some DAGs remained NULL even after scheduler parsed them. This seemed to prevent their tasks from getting scheduled. I eventually set it manually in the database and that fixed everything, but not sure if I did something wrong to get in that state in the first place 🤷

@ephraimbuddy
Copy link
Contributor Author

ephraimbuddy commented Sep 22, 2021

FWIW can confirm this fixed my issue in #17638, so big thanks! Note I did have an issue after upgrading where the max_active_runs columns for some DAGs remained NULL even after scheduler parsed them. This seemed to prevent their tasks from getting scheduled. I eventually set it manually in the database and that fixed everything, but not sure if I did something wrong to get in that state in the first place 🤷

Interesting.
Did you run airflow db upgrade after upgrade?
cc: @kaxil

robinedwards added a commit to robinedwards/airflow that referenced this pull request Oct 11, 2021
https://github.com/apache/airflow/pull/17945/files# which is reverted as
schedules tasks out of order but does work correctly when depends on
past is true.
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
area:Scheduler Scheduler or dag parsing Issues full tests needed We need to run full set of tests for this PR to merge
Projects
None yet
Development

Successfully merging this pull request may close these issues.

None yet

5 participants