You signed in with another tab or window. Reload to refresh your session.You signed out in another tab or window. Reload to refresh your session.You switched accounts on another tab or window. Reload to refresh your session.Dismiss alert
I was running an AIR data ingest example where the map tasks throw an exception and the script hung in map_batches. ray stack shows this is due to a deadlock in task cancellation:
The text was updated successfully, but these errors were encountered:
stephanie-wang
added
bug
Something that is supposed to be working; but isn't
triage
Needs triage (eg: priority, bug/not-bug, and owning component)
P0
Issues that should be fixed in short order
and removed
triage
Needs triage (eg: priority, bug/not-bug, and owning component)
labels
Oct 27, 2022
ray.cancel is executed by a background worker thread that invokes a Python callback to kill the current task. To avoid killing the wrong task, the worker holds the global CoreWorker lock during this process. However, this can cause deadlock because the worker also needs to acquire the GIL to call the kill callback. The GIL is usually held by the main thread, which is executing the normal task code. If this thread does not release the GIL to call CoreWorker methods, it can cause deadlock.
To avoid the deadlock, we can either always release the GIL before calling any CoreWorker method, or we can change task cancellation to not hold the CoreWorker lock. I chose the latter for the fix; while it is generally good to release the GIL before calling CoreWorker methods, it's hard to guarantee that we are always doing this. Therefore, modifying task cancellation seems safer.
This PR changes task cancellation to guard against the race condition in the Python code instead of relying on the CoreWorker lock. It does this by setting the current task ID during task execution, then holding a lock during task cancellation. This is guaranteed not to deadlock because we always acquire the GIL before the current task ID lock.
Related issue number
Closes#29739.
Signed-off-by: Stephanie Wang [email protected]
ray.cancel is executed by a background worker thread that invokes a Python callback to kill the current task. To avoid killing the wrong task, the worker holds the global CoreWorker lock during this process. However, this can cause deadlock because the worker also needs to acquire the GIL to call the kill callback. The GIL is usually held by the main thread, which is executing the normal task code. If this thread does not release the GIL to call CoreWorker methods, it can cause deadlock.
To avoid the deadlock, we can either always release the GIL before calling any CoreWorker method, or we can change task cancellation to not hold the CoreWorker lock. I chose the latter for the fix; while it is generally good to release the GIL before calling CoreWorker methods, it's hard to guarantee that we are always doing this. Therefore, modifying task cancellation seems safer.
This PR changes task cancellation to guard against the race condition in the Python code instead of relying on the CoreWorker lock. It does this by setting the current task ID during task execution, then holding a lock during task cancellation. This is guaranteed not to deadlock because we always acquire the GIL before the current task ID lock.
Related issue number
Closesray-project#29739.
Signed-off-by: Stephanie Wang [email protected]
Signed-off-by: Weichen Xu <[email protected]>
What happened + What you expected to happen
I was running an AIR data ingest example where the map tasks throw an exception and the script hung in map_batches.
ray stack
shows this is due to a deadlock in task cancellation:ray-stack.txt
Versions / Dependencies
3.0dev
Reproduction script
Issue Severity
No response
The text was updated successfully, but these errors were encountered: