Movatterモバイル変換


[0]ホーム

URL:


Skip to content

Navigation Menu

Search code, repositories, users, issues, pull requests...

Provide feedback

We read every piece of feedback, and take your input very seriously.

Saved searches

Use saved searches to filter your results more quickly

Sign up
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 ourterms of service andprivacy statement. We’ll occasionally send you account related emails.

Already on GitHub?Sign in to your account

[core] Threaded actors get stuck forever if they receive two exit signals#51582

Merged
edoakes merged 12 commits intomasterfrom20250320-devbox1-tmux3-ray7
Mar 25, 2025

Conversation

kevin85421
Copy link
Member

@kevin85421kevin85421 commentedMar 21, 2025
edited
Loading

Why are these changes needed?

image

If a threaded actor receives exit signals twice as shown in the above screenshot, it will executetask_receiver_->Stop() twice. However, the second call totask_receiver_->Stop() will get stuck forever when executing thereleaser.

Reproduction

ray start --head --include-dashboard=True --num-cpus=1# https://gist.github.com/kevin85421/7a42ac3693537c2148fa554065bb5223python3 test.py# Some actors are still ALIVE. If all actors are DEAD, increase the number of actors.ray list actors

Related issue number

Checks

  • I've signed off every commit(by using the -s flag, i.e.,git commit -s) in this PR.
  • I've runscripts/format.sh to lint the changes in this PR.
  • I've included any doc changes needed forhttps://docs.ray.io/en/master/.
    • I've added any new APIs to the API Reference. For example, if I added a
      method in Tune, I've added it indoc/source/tune/api/ under the
      corresponding.rst file.
  • I've made sure the tests are passing. Note that there might be a few flaky tests, see the recent failures athttps://flakey-tests.ray.io/
  • Testing Strategy
    • Unit tests
    • Release tests
    • This PR is not tested :(

Signed-off-by: kaihsun <kaihsun@anyscale.com>
@kevin85421
Copy link
MemberAuthor

This is a blocker of#51058.

@@ -1167,6 +1160,13 @@ void CoreWorker::Exit(
const rpc::WorkerExitType exit_type,
const std::string &detail,
const std::shared_ptr<LocalMemoryBuffer> &creation_task_exception_pb_bytes) {
if (is_shutdown_) {
Copy link
MemberAuthor

Choose a reason for hiding this comment

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

SinceShutdown is only called in theshutdown callback withinExit, promising to executeExit at most once also ensures thatShutdown is executed at most once.

Copy link
Member

Choose a reason for hiding this comment

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

I don't think this is correct.CoreWorker::Shutdown is also called byCoreWorkerProcessImpl::ShutdownDriver, which is called inCoreWorker.shutdown_driver in_raylet.pyx.

voidCoreWorkerProcessImpl::ShutdownDriver() {
RAY_CHECK(options_.worker_type == WorkerType::DRIVER)
<<"The `Shutdown` interface is for driver only.";
auto global_worker =GetCoreWorker();
RAY_CHECK(global_worker);
global_worker->Disconnect(/*exit_type*/ rpc::WorkerExitType::INTENDED_USER_EXIT,
/*exit_detail*/"Shutdown by ray.shutdown().");
global_worker->Shutdown();
{
auto write_locked = core_worker_.LockForWrite();
write_locked.Get().reset();
}
}

ray/python/ray/_raylet.pyx

Lines 3047 to 3055 inc6639d2

defshutdown_driver(self):
# If it's a worker, the core worker process should have been
# shutdown. So we can't call
# `CCoreWorkerProcess.GetCoreWorker().GetWorkerType()` here.
# Instead, we use the cached `is_driver` flag to test if it's a
# driver.
assertself.is_driver
with nogil:
CCoreWorkerProcess.Shutdown()

Copy link
Member

Choose a reason for hiding this comment

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

Should also add a test to check the case thatray.shutdown is called.

Copy link
MemberAuthor

@kevin85421kevin85421Mar 22, 2025
edited
Loading

Choose a reason for hiding this comment

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

Good catch! I only checked the call incore_worker.cc. I'm surprised there's another call toShutdown outside ofcore_worker.cc. Maybe we should consider unifying the code paths for both the core worker driver and the worker shutdown process.

Copy link
MemberAuthor

Choose a reason for hiding this comment

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

I will use another flag instead of reusingis_shutdown_.

Copy link
Collaborator

Choose a reason for hiding this comment

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

In the future, is there a way that we can better unify this shutdown sequence or is there a reason why we need separate public interfaces?

Copy link
MemberAuthor

Choose a reason for hiding this comment

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

I don't actually delve deeply into it, but I fully agree that we should have a single public function to terminate the core worker, whether it is a driver or a worker as I said#51582 (comment).

Copy link
MemberAuthor

Choose a reason for hiding this comment

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

We can leaveExit as the only one public interface to gracefully terminate a core worker process, and onlyExit can callShutdown.

Copy link
Collaborator

Choose a reason for hiding this comment

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

Can you take it as a follow-up to investigate and fix it if it isn't too hard?

Copy link
MemberAuthor

Choose a reason for hiding this comment

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

Can you take it as a follow-up to investigate and fix it if it isn't too hard?

Sure! Create an issue to track the progress:#51642.

@kevin85421kevin85421 added the goadd ONLY when ready to merge, run all tests labelMar 21, 2025
@edoakes
Copy link
Collaborator

ping when ready for review

@kevin85421kevin85421 marked this pull request as ready for reviewMarch 21, 2025 16:09
@edoakes
Copy link
Collaborator

what's the testing plan for this change?

@kevin85421
Copy link
MemberAuthor

Sorry, I initially thought it would be tested in#51058. Let me add a test in this PR.

Signed-off-by: kaihsun <kaihsun@anyscale.com>
@kevin85421
Copy link
MemberAuthor

Added a test.

@edoakes
Copy link
Collaborator

please help review@MortalHappiness //@dayshah

MortalHappiness reacted with thumbs up emoji

Signed-off-by: kaihsun <kaihsun@anyscale.com>
@MortalHappiness
Copy link
Member

MortalHappiness commentedMar 22, 2025
edited
Loading

Not sure whether the following reproduciton script addresses the same issue as this one.

importtimeimportrayray.init(address="auto")@ray.remote(concurrency_groups={"io":1})classTestActor:defexit(self):ray.actor.exit_actor()actors= [TestActor.remote()for_inrange(50)]ray.get([a.__ray_ready__.remote()forainactors])ray.wait([a.exit.remote()forainactors],timeout=10.0)print("Sleeping")time.sleep(3600)

Run this script until theSleeping message is printed, and then runray list actors in another terminal. Some actors are still alive.

I'm also wondering where the actor received the exit signal twice?

Signed-off-by: kaihsun <kaihsun@anyscale.com>
Signed-off-by: kaihsun <kaihsun@anyscale.com>
Signed-off-by: kaihsun <kaihsun@anyscale.com>
Signed-off-by: kaihsun <kaihsun@anyscale.com>
Signed-off-by: kaihsun <kaihsun@anyscale.com>
@kevin85421kevin85421 marked this pull request as draftMarch 23, 2025 23:29
Signed-off-by: kaihsun <kaihsun@anyscale.com>
Signed-off-by: kaihsun <kaihsun@anyscale.com>
@kevin85421kevin85421 marked this pull request as ready for reviewMarch 24, 2025 15:49
Copy link
Collaborator

@edoakesedoakes left a comment

Choose a reason for hiding this comment

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

LGTM pending comment

Comment on lines 1889 to 1891
/// Whether the `Exit` function has been called, to avoid executing the exit
/// process multiple times.
std::atomic<bool> is_exit_ = false;
Copy link
Collaborator

Choose a reason for hiding this comment

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

drop a comment about the relationship between this andis_shutdown_

in general, in cases where we have confusing duplicate behaviors we should document it as clearly as possible to help future readers

Copy link
MemberAuthor

Choose a reason for hiding this comment

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

Added

Signed-off-by: kaihsun <kaihsun@anyscale.com>
@edoakesedoakesenabled auto-merge (squash)March 24, 2025 17:17
@kevin85421
Copy link
MemberAuthor

The CI failures seem to be unrelated. Let me figure out what happened.

image

@kevin85421
Copy link
MemberAuthor

CI passes cc@edoakes would you mind merging this PR? The auto-merge was canceled when I sync with the master branch.

@edoakesedoakes merged commit6b805b5 intomasterMar 25, 2025
5 checks passed
@edoakesedoakes deleted the 20250320-devbox1-tmux3-ray7 branchMarch 25, 2025 18:24
"signal and is shutting down.";
return;
}
is_exit_ = true;
Copy link
Contributor

Choose a reason for hiding this comment

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

(not familiar with the code, just question for my curiosity)
is it possible multiple threads checkingis_exit_ and all of them pass through?
If it's not supposed to be called in multi-threaded env, we don't need atomic variable here.

In a word, reading through the code, I think you should use CAS
Ref:https://en.cppreference.com/w/cpp/atomic/atomic/compare_exchange

angelinalg pushed a commit to angelinalg/ray that referenced this pull requestMar 25, 2025
…nals (ray-project#51582)If a threaded actor receives exit signals twice as shown in the abovescreenshot, it will execute[task_receiver_->Stop()](https://github.com/ray-project/ray/blob/6bb9cef9257046ae31f78f6c52015a8ebf009f81/src/ray/core_worker/core_worker.cc#L1224)twice. However, the second call to `task_receiver_->Stop()` will getstuck forever when executing the[releaser](https://github.com/ray-project/ray/blob/6bb9cef9257046ae31f78f6c52015a8ebf009f81/src/ray/core_worker/transport/concurrency_group_manager.cc#L135).### Reproduction```shray start --head --include-dashboard=True --num-cpus=1#https://gist.github.com/kevin85421/7a42ac3693537c2148fa554065bb5223python3 test.py# Some actors are still ALIVE. If all actors are DEAD, increase the number of actors.ray list actors```---------Signed-off-by: kaihsun <kaihsun@anyscale.com>
dentiny pushed a commit to dentiny/ray that referenced this pull requestMar 25, 2025
…nals (ray-project#51582)If a threaded actor receives exit signals twice as shown in the abovescreenshot, it will execute[task_receiver_->Stop()](https://github.com/ray-project/ray/blob/6bb9cef9257046ae31f78f6c52015a8ebf009f81/src/ray/core_worker/core_worker.cc#L1224)twice. However, the second call to `task_receiver_->Stop()` will getstuck forever when executing the[releaser](https://github.com/ray-project/ray/blob/6bb9cef9257046ae31f78f6c52015a8ebf009f81/src/ray/core_worker/transport/concurrency_group_manager.cc#L135).### Reproduction```shray start --head --include-dashboard=True --num-cpus=1#https://gist.github.com/kevin85421/7a42ac3693537c2148fa554065bb5223python3 test.py# Some actors are still ALIVE. If all actors are DEAD, increase the number of actors.ray list actors```---------Signed-off-by: kaihsun <kaihsun@anyscale.com>
dhakshin32 pushed a commit to dhakshin32/ray that referenced this pull requestMar 27, 2025
…nals (ray-project#51582)If a threaded actor receives exit signals twice as shown in the abovescreenshot, it will execute[task_receiver_->Stop()](https://github.com/ray-project/ray/blob/6bb9cef9257046ae31f78f6c52015a8ebf009f81/src/ray/core_worker/core_worker.cc#L1224)twice. However, the second call to `task_receiver_->Stop()` will getstuck forever when executing the[releaser](https://github.com/ray-project/ray/blob/6bb9cef9257046ae31f78f6c52015a8ebf009f81/src/ray/core_worker/transport/concurrency_group_manager.cc#L135).### Reproduction```shray start --head --include-dashboard=True --num-cpus=1#https://gist.github.com/kevin85421/7a42ac3693537c2148fa554065bb5223python3 test.py# Some actors are still ALIVE. If all actors are DEAD, increase the number of actors.ray list actors```---------Signed-off-by: kaihsun <kaihsun@anyscale.com>Signed-off-by: Dhakshin Suriakannu <d_suriakannu@apple.com>
d-miketa pushed a commit to d-miketa/ray that referenced this pull requestMar 28, 2025
…nals (ray-project#51582)If a threaded actor receives exit signals twice as shown in the abovescreenshot, it will execute[task_receiver_->Stop()](https://github.com/ray-project/ray/blob/6bb9cef9257046ae31f78f6c52015a8ebf009f81/src/ray/core_worker/core_worker.cc#L1224)twice. However, the second call to `task_receiver_->Stop()` will getstuck forever when executing the[releaser](https://github.com/ray-project/ray/blob/6bb9cef9257046ae31f78f6c52015a8ebf009f81/src/ray/core_worker/transport/concurrency_group_manager.cc#L135).### Reproduction```shray start --head --include-dashboard=True --num-cpus=1#https://gist.github.com/kevin85421/7a42ac3693537c2148fa554065bb5223python3 test.py# Some actors are still ALIVE. If all actors are DEAD, increase the number of actors.ray list actors```---------Signed-off-by: kaihsun <kaihsun@anyscale.com>
srinathk10 pushed a commit that referenced this pull requestMar 28, 2025
…nals (#51582)If a threaded actor receives exit signals twice as shown in the abovescreenshot, it will execute[task_receiver_->Stop()](https://github.com/ray-project/ray/blob/6bb9cef9257046ae31f78f6c52015a8ebf009f81/src/ray/core_worker/core_worker.cc#L1224)twice. However, the second call to `task_receiver_->Stop()` will getstuck forever when executing the[releaser](https://github.com/ray-project/ray/blob/6bb9cef9257046ae31f78f6c52015a8ebf009f81/src/ray/core_worker/transport/concurrency_group_manager.cc#L135).### Reproduction```shray start --head --include-dashboard=True --num-cpus=1#https://gist.github.com/kevin85421/7a42ac3693537c2148fa554065bb5223python3 test.py# Some actors are still ALIVE. If all actors are DEAD, increase the number of actors.ray list actors```---------Signed-off-by: kaihsun <kaihsun@anyscale.com>Signed-off-by: Srinath Krishnamachari <srinath.krishnamachari@anyscale.com>
Sign up for freeto join this conversation on GitHub. Already have an account?Sign in to comment
Reviewers

@dentinydentinydentiny left review comments

@edoakesedoakesedoakes approved these changes

@MortalHappinessMortalHappinessMortalHappiness approved these changes

@dayshahdayshahAwaiting requested review from dayshah

Assignees

@edoakesedoakes

Labels
goadd ONLY when ready to merge, run all tests
Projects
None yet
Milestone
No milestone
Development

Successfully merging this pull request may close these issues.

4 participants
@kevin85421@edoakes@MortalHappiness@dentiny

[8]ページ先頭

©2009-2025 Movatter.jp