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 race condition between triggerer and scheduler #21316

Merged

Conversation

malthe
Copy link
Contributor

@malthe malthe commented Feb 4, 2022

This fixes an issue that can occur when a task is deferred and the resulting trigger completes before the executor is notified that the task was completed, whereby the rescheduling of the task fails since the executor refuses to enqueue a task which is already running.

The executor's tracking of running tasks is always lagging behind the task instance state but this is typically not a problem because tasks are not rescheduled fast enough for this to be a problem. But with deferred tasks and triggering logic, this picture changes since a trigger condition can in some cases be met right away – for example, checking whether a certain external system has a given state (i.e., "sensoring").

An alternative solution is to for example sleep for a small amount of time in the triggerer before changing the task state back to SCHEDULED – perhaps in the form of a minimum triggering duration to allow the executor to register that the initial task has completed.

@boring-cyborg boring-cyborg bot added the area:Scheduler Scheduler or dag parsing Issues label Feb 4, 2022
@malthe malthe force-pushed the retry-queuing-when-task-is-in-running-state branch from 22a12cc to 95538a1 Compare February 9, 2022 06:29
@potiuk
Copy link
Member

potiuk commented Feb 14, 2022

I do not know as much about the triggerer, but maybe @andrewgodwin might chime-in?

@kaxil kaxil requested a review from dstandish February 14, 2022 16:12
@kaxil kaxil added this to the Airflow 2.3.0 milestone Feb 14, 2022
@dstandish
Copy link
Contributor

@malthe do you think you could write a test for this?

@andrewgodwin
Copy link
Contributor

This seems to make sense to me - deferring definitely exposed some edge cases in the scheduler where things transitioned state "too fast", and since the "true fix" would be an entire rejiggling of the database schema and state machine, this looks like a sensible fix that won't take ages!

@potiuk
Copy link
Member

potiuk commented Feb 14, 2022

Rebasing should fix the docker failure.

@potiuk
Copy link
Member

potiuk commented Feb 14, 2022

cc: @malthe

@malthe malthe force-pushed the retry-queuing-when-task-is-in-running-state branch from 367011e to 4b7d512 Compare February 14, 2022 21:30
@malthe
Copy link
Contributor Author

malthe commented Feb 14, 2022

@dstandish tests added in 367011e339c5d8afeaeea98690e1bc29c19641ba and branch rebased.

@github-actions github-actions bot added the full tests needed We need to run full set of tests for this PR to merge label Feb 14, 2022
@github-actions
Copy link

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.

Copy link
Contributor

@dstandish dstandish left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

suggested a test modification but looks good to me

tests/executors/test_base_executor.py Outdated Show resolved Hide resolved
Co-authored-by: Daniel Standish <15932138+dstandish@users.noreply.github.com>
@potiuk potiuk merged commit 2a6792d into apache:main Feb 15, 2022
@jedcunningham jedcunningham added the type:bug-fix Changelog: Bug Fixes label Feb 28, 2022
ephraimbuddy pushed a commit that referenced this pull request Mar 16, 2022
ephraimbuddy pushed a commit that referenced this pull request Mar 20, 2022
ephraimbuddy pushed a commit that referenced this pull request Mar 22, 2022
ephraimbuddy pushed a commit that referenced this pull request Mar 22, 2022
ephraimbuddy pushed a commit that referenced this pull request Mar 22, 2022
ephraimbuddy pushed a commit that referenced this pull request Mar 22, 2022
ephraimbuddy pushed a commit that referenced this pull request Mar 24, 2022
ephraimbuddy pushed a commit that referenced this pull request Mar 26, 2022
@tanelk
Copy link
Contributor

tanelk commented Apr 11, 2022

A very late "review".

CeleryExecutor overwrites the trigger_tasks and this fix does not help when using celery.

@malthe
Copy link
Contributor Author

malthe commented Apr 11, 2022

@tanelk nice find – it seems like the Celery executor has its own retrying logic in task_publish_retries which combined with this pull request ends up still solving the problem.

But it seems like we should be able to rework this to share the same logic.

ashb added a commit that referenced this pull request Apr 26, 2022
This is a follow-up to #21316 which did not take into account that CeleryExecutor
overrides trigger_tasks and thus would ignore if a task was already running.

Co-authored-by: Ash Berlin-Taylor <ash_github@firemirror.com>
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 type:bug-fix Changelog: Bug Fixes
Projects
None yet
Development

Successfully merging this pull request may close these issues.

None yet

7 participants