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
ray.cancel may lead to deadlocks, primarily because the implementation of NormalTaskSubmitter::CancelTask is too risky. It calls functions related to TaskManager (holding TaskManager::mu_) while holding NormalTaskSubmitter::mu_. In doing so, it might also hold TaskManager::object_ref_stream_ops_mu_. Meanwhile, we insert callbacks into MemoryStore through ResolveDependencies, which attempt to hold NormalTaskSubmitter::mu_. At the same time, some functions within TaskManager might call MemoryStore::Put with TaskManager::object_ref_stream_ops_mu_, ultimately leading to a deadlock.
The text was updated successfully, but these errors were encountered:
Catch-Bull
added
bug
Something that is supposed to be working; but isn't
triage
Needs triage (eg: priority, bug/not-bug, and owning component)
labels
Sep 13, 2024
jjyao
added
P0
Issues that should be fixed in short order
and removed
triage
Needs triage (eg: priority, bug/not-bug, and owning component)
labels
Sep 16, 2024
lambda1 at NormalTaskSubmitter::SubmitTask (on_dependencies_resolved)
NormalTaskSubmitter::mu_
lambda1 at LocalDependencyResolver::ResolveDependencies (src/ray/core_worker/transport/dependency_resolver.cc:100)
LocalDependencyResolver::mu_
CoreWorkerMemoryStore::Put
TaskManager::HandleTaskReturn
Excludes TaskManager::mu_
TaskManager::HandleReportGeneratorItemReturns
TaskManager::object_ref_stream_ops_mu_
So in thread 2, CoreWorkerMemoryStore::Put invokes get callback from GetAsync call registered by LocalDependencyResolver::ResolveDependencies. I think this GetAsync should be really async - that the callback is posted to another asio task and should not hold any locks.
Regarding #47650 , I also agree we can reduce scope of mu_ in CancelTask before calling MarkTaskCanceled. However I am a bit concerned on the Releasable mutex, and especially we acquire it once and release it in a for loop.
I made a PR #47833 to put CoreWorkerMemoryStore callbacks really to async. @Catch-Bull can you talk a look and try to see if it solves this deadlock? Thanks!
What happened + What you expected to happen
ray.cancel may lead to deadlocks, primarily because the implementation of
NormalTaskSubmitter::CancelTask
is too risky. It calls functions related toTaskManager
(holdingTaskManager::mu_
) while holdingNormalTaskSubmitter::mu_
. In doing so, it might also holdTaskManager::object_ref_stream_ops_mu_
. Meanwhile, we insert callbacks intoMemoryStore
throughResolveDependencies
, which attempt to holdNormalTaskSubmitter::mu_
. At the same time, some functions withinTaskManager
might callMemoryStore::Put
withTaskManager::object_ref_stream_ops_mu_
, ultimately leading to a deadlock.Versions / Dependencies
ray: https://github.com/Catch-Bull/ray/tree/ray_cancel_deadlock
To facilitate the reproduction of specific timing, a commit was added on ray master 1dd8d60.
Reproduction script
it will hang, stack:
NormalTaskSubmitter::mu_
will deadlockIssue Severity
None
The text was updated successfully, but these errors were encountered: