Skip to content

[core] Detached actor being killed when its parent actor crashes #40864

@edoakes

Description

@edoakes
Collaborator

While debugging a release test failure, we discovered that in some cases Serve replica actors are being killed due to fate sharing with the controller.

This should never happen because all actors started by Serve (the controller, replicas, proxies) are detached, so they should not fate share with the controller (relevant code in the raylet).

We see a number of log lines like the following in the Raylet logs in multiple runs of the Serve long-running failure test case:

[2023-11-01 07:10:17,273 I 825 825] (raylet) node_manager.cc:1104: The leased worker dd7d4d82da8fef21e59667dba16f2bce15203c8832039284cbb26461 is killed because the owner process 2b60b506544d378c192b7e1cbf989be4058f41015c00fa7f30e50f91 died.
[2023-11-01 07:10:17,273 I 825 825] (raylet) node_manager.cc:1104: The leased worker 3844620037d1ea4a19c830bb548edd9726cd4521cc78f2c7871367d6 is killed because the owner process 2b60b506544d378c192b7e1cbf989be4058f41015c00fa7f30e50f91 died.

All of the referenced actors are detached actors.

Activity

added
bugSomething that is supposed to be working; but isn't
triageNeeds triage (eg: priority, bug/not-bug, and owning component)
coreIssues that should be addressed in Ray Core
on Nov 1, 2023
edoakes

edoakes commented on Nov 1, 2023

@edoakes
CollaboratorAuthor

Here are the full cluster logs for the release test failure run: logs.zip

edoakes

edoakes commented on Nov 1, 2023

@edoakes
CollaboratorAuthor

One possible (unsubstantiated) theory: in this test, the controller and the actors it creates are killed repeatedly. It could be that the controller is killed immediately after creating a replica, in which case the raylet may not yet have marked the worker running the replica as being a detached actor.

edoakes

edoakes commented on Nov 1, 2023

@edoakes
CollaboratorAuthor

Looks like we only mark the actor as detached after the creation task finishes:

worker.MarkDetachedActor();

This means if the creation task is sent to the actor, then the owner dies before it finishes, the actor may be killed by HandleUnexpectedWorkerFailure.

added
P1Issue that should be fixed within a few weeks
and removed
triageNeeds triage (eg: priority, bug/not-bug, and owning component)
on Nov 1, 2023
added and removed
P1Issue that should be fixed within a few weeks
on May 10, 2024
added
P1Issue that should be fixed within a few weeks
and removed on Oct 30, 2024
rueian

rueian commented on Jun 4, 2025

@rueian
Contributor

This issue can be reproduced on Ray 2.46.0 by the following 3 scripts:

# setup_cluster.py
from ray.cluster_utils import AutoscalingCluster
cluster = AutoscalingCluster(
    head_resources={"CPU": 0},
    worker_node_types={
        "type-1": {
            "resources": {"CPU": 10},
            "node_config": {},
            "min_workers": 1,
            "max_workers": 1,
        },
    },
    max_workers=2,
)
cluster.start()
# create_actor.py
import ray
import random
import string

@ray.remote(num_cpus=1)
class Actor:
    pass

if __name__ == '__main__':
    name = ''.join(random.choices(string.ascii_letters + string.digits, k=8))
    ray.init("auto", namespace="namespace")
    Actor.options(name=name, lifetime="detached").remote()
    print(name)
# delete_actor.py
import ray
import sys
import time
from ray.util.state import list_actors

def wait_for_actor_creation(name):
    while True:
        time.sleep(1)
        actors = list_actors(filters=[("name", "=", name)])
        if actors and actors[0]['state'] == 'ALIVE':
            return True
        if actors and actors[0]['state'] == 'DEAD':
            return False

if __name__ == '__main__':
    name = next(sys.stdin).rstrip()
    ray.init("auto", namespace="namespace")
    if wait_for_actor_creation(name):
        ray.kill(ray.get_actor(name)) # clean the actor for the next iteration.
    else:
        raise Exception(f"Actor {name} was dead.")

And run the above scripts like this:

▶ python setup_cluster.py
...
▶ while python create_actor.py > name.out && python delete_actor.py < name.out; do :; done
2025-06-03 22:23:50,035	INFO worker.py:1694 -- Connecting to existing Ray cluster at address: 127.0.0.1:6379...
2025-06-03 22:23:50,046	INFO worker.py:1879 -- Connected to Ray cluster. View the dashboard at http://127.0.0.1:8265
2025-06-03 22:23:51,692	INFO worker.py:1694 -- Connecting to existing Ray cluster at address: 127.0.0.1:6379...
2025-06-03 22:23:51,705	INFO worker.py:1879 -- Connected to Ray cluster. View the dashboard at http://127.0.0.1:8265
2025-06-03 22:23:53,951	INFO worker.py:1694 -- Connecting to existing Ray cluster at address: 127.0.0.1:6379...
2025-06-03 22:23:53,962	INFO worker.py:1879 -- Connected to Ray cluster. View the dashboard at http://127.0.0.1:8265
2025-06-03 22:23:55,720	INFO worker.py:1694 -- Connecting to existing Ray cluster at address: 127.0.0.1:6379...
2025-06-03 22:23:55,735	INFO worker.py:1879 -- Connected to Ray cluster. View the dashboard at http://127.0.0.1:8265
2025-06-03 22:23:58,050	INFO worker.py:1694 -- Connecting to existing Ray cluster at address: 127.0.0.1:6379...
2025-06-03 22:23:58,062	INFO worker.py:1879 -- Connected to Ray cluster. View the dashboard at http://127.0.0.1:8265
2025-06-03 22:23:59,444	INFO worker.py:1694 -- Connecting to existing Ray cluster at address: 127.0.0.1:6379...
2025-06-03 22:23:59,455	INFO worker.py:1879 -- Connected to Ray cluster. View the dashboard at http://127.0.0.1:8265
2025-06-03 22:24:01,772	INFO worker.py:1694 -- Connecting to existing Ray cluster at address: 127.0.0.1:6379...
2025-06-03 22:24:01,782	INFO worker.py:1879 -- Connected to Ray cluster. View the dashboard at http://127.0.0.1:8265
2025-06-03 22:24:03,188	INFO worker.py:1694 -- Connecting to existing Ray cluster at address: 127.0.0.1:6379...
2025-06-03 22:24:03,201	INFO worker.py:1879 -- Connected to Ray cluster. View the dashboard at http://127.0.0.1:8265
2025-06-03 22:24:05,503	INFO worker.py:1694 -- Connecting to existing Ray cluster at address: 127.0.0.1:6379...
2025-06-03 22:24:05,516	INFO worker.py:1879 -- Connected to Ray cluster. View the dashboard at http://127.0.0.1:8265
2025-06-03 22:24:07,269	INFO worker.py:1694 -- Connecting to existing Ray cluster at address: 127.0.0.1:6379...
2025-06-03 22:24:07,291	INFO worker.py:1879 -- Connected to Ray cluster. View the dashboard at http://127.0.0.1:8265
2025-06-03 22:24:09,710	INFO worker.py:1694 -- Connecting to existing Ray cluster at address: 127.0.0.1:6379...
2025-06-03 22:24:09,743	INFO worker.py:1879 -- Connected to Ray cluster. View the dashboard at http://127.0.0.1:8265
2025-06-03 22:24:11,509	INFO worker.py:1694 -- Connecting to existing Ray cluster at address: 127.0.0.1:6379...
2025-06-03 22:24:11,522	INFO worker.py:1879 -- Connected to Ray cluster. View the dashboard at http://127.0.0.1:8265
2025-06-03 22:24:13,841	INFO worker.py:1694 -- Connecting to existing Ray cluster at address: 127.0.0.1:6379...
2025-06-03 22:24:13,862	INFO worker.py:1879 -- Connected to Ray cluster. View the dashboard at http://127.0.0.1:8265
2025-06-03 22:24:15,264	INFO worker.py:1694 -- Connecting to existing Ray cluster at address: 127.0.0.1:6379...
2025-06-03 22:24:15,282	INFO worker.py:1879 -- Connected to Ray cluster. View the dashboard at http://127.0.0.1:8265
2025-06-03 22:24:17,591	INFO worker.py:1694 -- Connecting to existing Ray cluster at address: 127.0.0.1:6379...
2025-06-03 22:24:17,602	INFO worker.py:1879 -- Connected to Ray cluster. View the dashboard at http://127.0.0.1:8265
2025-06-03 22:24:19,173	INFO worker.py:1694 -- Connecting to existing Ray cluster at address: 127.0.0.1:6379...
2025-06-03 22:24:19,187	INFO worker.py:1879 -- Connected to Ray cluster. View the dashboard at http://127.0.0.1:8265
2025-06-03 22:24:21,502	INFO worker.py:1694 -- Connecting to existing Ray cluster at address: 127.0.0.1:6379...
2025-06-03 22:24:21,513	INFO worker.py:1879 -- Connected to Ray cluster. View the dashboard at http://127.0.0.1:8265
2025-06-03 22:24:23,029	INFO worker.py:1694 -- Connecting to existing Ray cluster at address: 127.0.0.1:6379...
2025-06-03 22:24:23,042	INFO worker.py:1879 -- Connected to Ray cluster. View the dashboard at http://127.0.0.1:8265
2025-06-03 22:24:25,290	INFO worker.py:1694 -- Connecting to existing Ray cluster at address: 127.0.0.1:6379...
2025-06-03 22:24:25,301	INFO worker.py:1879 -- Connected to Ray cluster. View the dashboard at http://127.0.0.1:8265
2025-06-03 22:24:26,716	INFO worker.py:1694 -- Connecting to existing Ray cluster at address: 127.0.0.1:6379...
2025-06-03 22:24:26,730	INFO worker.py:1879 -- Connected to Ray cluster. View the dashboard at http://127.0.0.1:8265
2025-06-03 22:24:29,077	INFO worker.py:1694 -- Connecting to existing Ray cluster at address: 127.0.0.1:6379...
2025-06-03 22:24:29,088	INFO worker.py:1879 -- Connected to Ray cluster. View the dashboard at http://127.0.0.1:8265
2025-06-03 22:24:30,686	INFO worker.py:1694 -- Connecting to existing Ray cluster at address: 127.0.0.1:6379...
2025-06-03 22:24:30,699	INFO worker.py:1879 -- Connected to Ray cluster. View the dashboard at http://127.0.0.1:8265
2025-06-03 22:24:33,054	INFO worker.py:1694 -- Connecting to existing Ray cluster at address: 127.0.0.1:6379...
2025-06-03 22:24:33,066	INFO worker.py:1879 -- Connected to Ray cluster. View the dashboard at http://127.0.0.1:8265
2025-06-03 22:24:34,556	INFO worker.py:1694 -- Connecting to existing Ray cluster at address: 127.0.0.1:6379...
2025-06-03 22:24:34,569	INFO worker.py:1879 -- Connected to Ray cluster. View the dashboard at http://127.0.0.1:8265
2025-06-03 22:24:36,824	INFO worker.py:1694 -- Connecting to existing Ray cluster at address: 127.0.0.1:6379...
2025-06-03 22:24:36,835	INFO worker.py:1879 -- Connected to Ray cluster. View the dashboard at http://127.0.0.1:8265
2025-06-03 22:24:38,453	INFO worker.py:1694 -- Connecting to existing Ray cluster at address: 127.0.0.1:6379...
2025-06-03 22:24:38,467	INFO worker.py:1879 -- Connected to Ray cluster. View the dashboard at http://127.0.0.1:8265
2025-06-03 22:24:40,823	INFO worker.py:1694 -- Connecting to existing Ray cluster at address: 127.0.0.1:6379...
2025-06-03 22:24:40,840	INFO worker.py:1879 -- Connected to Ray cluster. View the dashboard at http://127.0.0.1:8265
2025-06-03 22:24:42,470	INFO worker.py:1694 -- Connecting to existing Ray cluster at address: 127.0.0.1:6379...
2025-06-03 22:24:42,482	INFO worker.py:1879 -- Connected to Ray cluster. View the dashboard at http://127.0.0.1:8265
2025-06-03 22:24:44,856	INFO worker.py:1694 -- Connecting to existing Ray cluster at address: 127.0.0.1:6379...
2025-06-03 22:24:44,868	INFO worker.py:1879 -- Connected to Ray cluster. View the dashboard at http://127.0.0.1:8265
2025-06-03 22:24:46,473	INFO worker.py:1694 -- Connecting to existing Ray cluster at address: 127.0.0.1:6379...
2025-06-03 22:24:46,484	INFO worker.py:1879 -- Connected to Ray cluster. View the dashboard at http://127.0.0.1:8265
2025-06-03 22:24:49,060	INFO worker.py:1694 -- Connecting to existing Ray cluster at address: 127.0.0.1:6379...
2025-06-03 22:24:49,071	INFO worker.py:1879 -- Connected to Ray cluster. View the dashboard at http://127.0.0.1:8265
2025-06-03 22:24:50,716	INFO worker.py:1694 -- Connecting to existing Ray cluster at address: 127.0.0.1:6379...
2025-06-03 22:24:50,728	INFO worker.py:1879 -- Connected to Ray cluster. View the dashboard at http://127.0.0.1:8265
2025-06-03 22:24:53,221	INFO worker.py:1694 -- Connecting to existing Ray cluster at address: 127.0.0.1:6379...
2025-06-03 22:24:53,234	INFO worker.py:1879 -- Connected to Ray cluster. View the dashboard at http://127.0.0.1:8265
2025-06-03 22:24:54,677	INFO worker.py:1694 -- Connecting to existing Ray cluster at address: 127.0.0.1:6379...
2025-06-03 22:24:54,691	INFO worker.py:1879 -- Connected to Ray cluster. View the dashboard at http://127.0.0.1:8265
2025-06-03 22:24:57,009	INFO worker.py:1694 -- Connecting to existing Ray cluster at address: 127.0.0.1:6379...
2025-06-03 22:24:57,021	INFO worker.py:1879 -- Connected to Ray cluster. View the dashboard at http://127.0.0.1:8265
2025-06-03 22:24:58,548	INFO worker.py:1694 -- Connecting to existing Ray cluster at address: 127.0.0.1:6379...
2025-06-03 22:24:58,595	INFO worker.py:1879 -- Connected to Ray cluster. View the dashboard at http://127.0.0.1:8265
2025-06-03 22:25:00,916	INFO worker.py:1694 -- Connecting to existing Ray cluster at address: 127.0.0.1:6379...
2025-06-03 22:25:00,930	INFO worker.py:1879 -- Connected to Ray cluster. View the dashboard at http://127.0.0.1:8265
2025-06-03 22:25:02,367	INFO worker.py:1694 -- Connecting to existing Ray cluster at address: 127.0.0.1:6379...
2025-06-03 22:25:02,379	INFO worker.py:1879 -- Connected to Ray cluster. View the dashboard at http://127.0.0.1:8265
2025-06-03 22:25:04,588	INFO worker.py:1694 -- Connecting to existing Ray cluster at address: 127.0.0.1:6379...
2025-06-03 22:25:04,600	INFO worker.py:1879 -- Connected to Ray cluster. View the dashboard at http://127.0.0.1:8265
2025-06-03 22:25:06,314	INFO worker.py:1694 -- Connecting to existing Ray cluster at address: 127.0.0.1:6379...
2025-06-03 22:25:06,328	INFO worker.py:1879 -- Connected to Ray cluster. View the dashboard at http://127.0.0.1:8265
2025-06-03 22:25:08,564	INFO worker.py:1694 -- Connecting to existing Ray cluster at address: 127.0.0.1:6379...
2025-06-03 22:25:08,576	INFO worker.py:1879 -- Connected to Ray cluster. View the dashboard at http://127.0.0.1:8265
2025-06-03 22:25:10,200	INFO worker.py:1694 -- Connecting to existing Ray cluster at address: 127.0.0.1:6379...
2025-06-03 22:25:10,212	INFO worker.py:1879 -- Connected to Ray cluster. View the dashboard at http://127.0.0.1:8265
2025-06-03 22:25:12,556	INFO worker.py:1694 -- Connecting to existing Ray cluster at address: 127.0.0.1:6379...
2025-06-03 22:25:12,568	INFO worker.py:1879 -- Connected to Ray cluster. View the dashboard at http://127.0.0.1:8265
2025-06-03 22:25:14,109	INFO worker.py:1694 -- Connecting to existing Ray cluster at address: 127.0.0.1:6379...
2025-06-03 22:25:14,123	INFO worker.py:1879 -- Connected to Ray cluster. View the dashboard at http://127.0.0.1:8265
2025-06-03 22:25:16,452	INFO worker.py:1694 -- Connecting to existing Ray cluster at address: 127.0.0.1:6379...
2025-06-03 22:25:16,465	INFO worker.py:1879 -- Connected to Ray cluster. View the dashboard at http://127.0.0.1:8265
2025-06-03 22:25:18,063	INFO worker.py:1694 -- Connecting to existing Ray cluster at address: 127.0.0.1:6379...
2025-06-03 22:25:18,074	INFO worker.py:1879 -- Connected to Ray cluster. View the dashboard at http://127.0.0.1:8265
2025-06-03 22:25:20,331	INFO worker.py:1694 -- Connecting to existing Ray cluster at address: 127.0.0.1:6379...
2025-06-03 22:25:20,344	INFO worker.py:1879 -- Connected to Ray cluster. View the dashboard at http://127.0.0.1:8265
2025-06-03 22:25:21,752	INFO worker.py:1694 -- Connecting to existing Ray cluster at address: 127.0.0.1:6379...
2025-06-03 22:25:21,765	INFO worker.py:1879 -- Connected to Ray cluster. View the dashboard at http://127.0.0.1:8265
Traceback (most recent call last):
  File "/Users/ruian/Code/python/ray/testdelete.py", line 21, in <module>
    raise Exception(f"Actor {name} was dead.")
    ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
Exception: Actor KifMRWkH was dead.

I think the issue was caused by #14184, where the first worker.MarkDetachedActor() was deleted here https://github.com/ray-project/ray/pull/14184/files#diff-d2f22b8f1bf5f9be47dacae8b467a72ee94629df12ffcc18b13447192ff3dbcfL1982, which made a leased worker possible to be killed by HandleUnexpectedWorkerFailure.

I think a proper fix now would be to

  1. replace all worker->IsDetachedActor() with worker->GetAssignedTask().GetTaskSpecification().IsDetachedActor()
  2. remove Worker::IsDetachedActor and Worker::MarkDetachedActor to avoid further confusion.

I will open a PR soon.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Metadata

Metadata

Assignees

No one assigned

    Labels

    P1Issue that should be fixed within a few weeksbugSomething that is supposed to be working; but isn'tcoreIssues that should be addressed in Ray Corecore-workerstability

    Type

    No type

    Projects

    No projects

    Milestone

    No milestone

    Relationships

    None yet

      Development

      Participants

      @jjyao@rueian@cszhu@edoakes@anyscalesam

      Issue actions

        [core] Detached actor being killed when its parent actor crashes · Issue #40864 · ray-project/ray