Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
54 commits
Select commit Hold shift + click to select a range
752697e
[serve][llm] Fix LLM target_ongoing_requests default (#59323)
eicherseiji Dec 13, 2025
45f8bbd
[core] Fix crash when killing actor handle from previous session
kriyanshii Dec 14, 2025
65ba9ec
Update python/ray/_private/worker.py
kriyanshii Dec 14, 2025
954810b
[core] Move CreateCall to polling thread and add documentation on grp…
dayshah Dec 14, 2025
6956c3e
[rllib] Fix `as-release-test` silently failing results upload (#59386)
pseudo-rnd-thoughts Dec 15, 2025
d0851de
[RLlib] Fix recursive imports in old test utils location (#59435)
ArturNiederfahrenhorst Dec 15, 2025
655a5fc
[Serve][1/n] Improve type annotations for DeploymentHandle, Deploymen…
abrarsheikh Dec 15, 2025
86ac775
[Serve] Fix `make_fastapi_class_based_view` for inherited methods (#5…
kouroshHakha Dec 15, 2025
1cbb7ae
[Serve][2/n] add batching metrics (#59232)
abrarsheikh Dec 15, 2025
df332d4
[release] Change release test default python version 310 (#59396)
elliot-barn Dec 15, 2025
eb1bc60
[Data] Add configurable batching for resolve_block_refs to speed up i…
YoussefEssDS Dec 15, 2025
aeae004
[Data] Streamlining `Unique` aggregation and Preprocessors (#59402)
alexeykudinkin Dec 15, 2025
4e76e84
[Data] Abstract autoscaler scaling delta calculation (#59192)
owenowenisme Dec 15, 2025
076108f
[Serve][LLM] Add dispatch API for control plane operations across all…
kouroshHakha Dec 15, 2025
9fade94
[scripts] remove install-nightly (#59441)
aslonnie Dec 15, 2025
b6266d0
[Core] Add authentication token to c++ opentelemetry_metrics_recorder…
sampan-s-nayak Dec 15, 2025
1757370
[ci] use merge-base for conditional testing (#59452)
aslonnie Dec 15, 2025
3230309
[release test] add helper that lists tests (#59253)
aslonnie Dec 15, 2025
aeeeff6
[ci] change base defaults to python 3.10 (#59442)
aslonnie Dec 15, 2025
69893e9
[ci] remove residue of python 3.9 ci bases (#59443)
aslonnie Dec 15, 2025
69c5402
[doc] removes docs depsets except python 3.10 (#59450)
aslonnie Dec 15, 2025
97834b8
[ci][release] removing 3.9 builds (#59447)
elliot-barn Dec 15, 2025
0574100
[Core] Fix flaky test_autoscaler_init: poll cluster status instead of…
Yicheng-Lu-llll Dec 15, 2025
3449de2
[docs/data] Add `download` to key user journeys in documentation (#59…
richardliaw Dec 16, 2025
dd50c82
Revert "[Data] Fix DataContext deserialization issue with StatsActor"…
raulchen Dec 16, 2025
60dec74
[core] Revert server CreateCall creation change #55904 (#59448)
dayshah Dec 16, 2025
a6cb667
[train] removes single python ml build (#59465)
aslonnie Dec 16, 2025
cbec7ad
[Serve][3/n] Add router queue latency (#59233)
abrarsheikh Dec 16, 2025
1985eb7
[core][rdt] Enable nixl for RDT Microbenchmarks (#59291)
dayshah Dec 16, 2025
122a592
Signed-off-by: kriyanshii <[email protected]>
kriyanshii Dec 16, 2025
5dedf15
Merge branch 'master' into master
kriyanshii Dec 16, 2025
a871656
Merge branch 'ray-project:master' into master
kriyanshii Dec 17, 2025
5bacc93
Signed-off-by: kriyanshii <[email protected]>
kriyanshii Dec 17, 2025
2906b32
Merge branch 'master' into master
kriyanshii Dec 17, 2025
fec93d0
Merge branch 'ray-project:master' into master
kriyanshii Dec 17, 2025
579971e
Merge branch 'master' into master
kriyanshii Dec 17, 2025
c257c20
Signed-off-by: kriyanshii <[email protected]>
kriyanshii Dec 18, 2025
1472a8c
Signed-off-by: kriyanshii <[email protected]>
kriyanshii Dec 18, 2025
31ae72c
Merge branch 'master' into master
kriyanshii Dec 18, 2025
b32ce33
Signed-off-by: kriyanshii <[email protected]>
kriyanshii Dec 19, 2025
64d4c23
Merge branch 'master' into master
kriyanshii Dec 19, 2025
d2e03b4
Merge branch 'ray-project:master' into master
kriyanshii Dec 21, 2025
66e2932
Merge branch 'master' into master
kriyanshii Dec 23, 2025
b8b9a65
Merge branch 'ray-project:master' into master
kriyanshii Dec 23, 2025
a40d599
Merge branch 'ray-project:master' into master
kriyanshii Dec 25, 2025
a584b0f
Merge branch 'ray-project:master' into master
kriyanshii Jan 3, 2026
c6702bb
Merge branch 'ray-project:master' into master
kriyanshii Jan 4, 2026
d9a658c
Merge branch 'ray-project:master' into master
kriyanshii Jan 5, 2026
464b550
Merge branch 'ray-project:master' into master
kriyanshii Jan 6, 2026
a982498
Merge branch 'master' into master
kriyanshii Jan 6, 2026
aad0a4f
Merge branch 'master' into master
kriyanshii Jan 7, 2026
607985a
Merge branch 'master' into master
kriyanshii Jan 7, 2026
7872637
Merge branch 'master' into master
edoakes Jan 7, 2026
4f3c083
Merge branch 'master' into master
kriyanshii Jan 8, 2026
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
23 changes: 21 additions & 2 deletions python/ray/_private/worker.py
Original file line number Diff line number Diff line change
Expand Up @@ -90,7 +90,13 @@
raise_sys_exit_with_custom_error_message,
)
from ray.actor import ActorClass
from ray.exceptions import ObjectStoreFullError, RayError, RaySystemError, RayTaskError
from ray.exceptions import (
ActorHandleNotFoundError,
ObjectStoreFullError,
RayError,
RaySystemError,
RayTaskError,
)
from ray.experimental import tqdm_ray
from ray.experimental.compiled_dag_ref import CompiledDAGRef
from ray.experimental.internal_kv import (
Expand Down Expand Up @@ -3289,7 +3295,20 @@ def kill(actor: "ray.actor.ActorHandle", *, no_restart: bool = True):
"ray.kill() only supported for actors. For tasks, try ray.cancel(). "
"Got: {}.".format(type(actor))
)
worker.core_worker.kill_actor(actor._ray_actor_id, no_restart)

try:
worker.core_worker.kill_actor(actor._ray_actor_id, no_restart)
except ActorHandleNotFoundError as e:
actor_job_id = actor._ray_actor_id.job_id
current_job_id = worker.current_job_id
raise ActorHandleNotFoundError(
f"ActorHandle objects are not valid across Ray sessions. "
f"The actor handle was created in job {actor_job_id.hex()}, "
f"but the current job is {current_job_id.hex()}. "
f"This typically happens when you try to use an actor handle "
f"from a previous session after calling ray.shutdown() and ray.init(). "
f"Please create a new actor handle in the current session."
) from e


@PublicAPI
Expand Down
13 changes: 10 additions & 3 deletions python/ray/_raylet.pyx
Original file line number Diff line number Diff line change
Expand Up @@ -196,8 +196,9 @@ include "includes/rpc_token_authentication.pxi"

import ray
from ray.exceptions import (
RayActorError,
ActorHandleNotFoundError,
ActorDiedError,
RayActorError,
RayError,
RaySystemError,
RayTaskError,
Expand Down Expand Up @@ -3796,10 +3797,16 @@ cdef class CoreWorker:
def kill_actor(self, ActorID actor_id, c_bool no_restart):
cdef:
CActorID c_actor_id = actor_id.native()
CRayStatus status = CRayStatus.OK()

with nogil:
check_status(CCoreWorkerProcess.GetCoreWorker().KillActor(
c_actor_id, True, no_restart))
status = CCoreWorkerProcess.GetCoreWorker().KillActor(
c_actor_id, True, no_restart)

if status.IsNotFound():
raise ActorHandleNotFoundError(status.message().decode())

check_status(status)

def cancel_task(self, ObjectRef object_ref, c_bool force_kill,
c_bool recursive):
Expand Down
23 changes: 23 additions & 0 deletions python/ray/exceptions.py
Original file line number Diff line number Diff line change
Expand Up @@ -1032,6 +1032,28 @@ def __str__(self):
return self.error_message


@DeveloperAPI
class ActorHandleNotFoundError(ValueError, RayError):
"""Raised when trying to kill an actor handle that doesn't exist.

This typically happens when using an actor handle from a previous Ray session
after calling ray.shutdown() and ray.init().

Note that this error is not only a subclass of RayError, but also a subclass of ValueError,
to maintain backward compatibility.

Args:
error_message: The error message that contains information about the actor handle.
"""

def __init__(self, error_message: str):
super().__init__(error_message)
self.error_message = error_message

def __str__(self):
return self.error_message


RAY_EXCEPTION_TYPES = [
PlasmaObjectNotAvailable,
RayError,
Expand Down Expand Up @@ -1063,5 +1085,6 @@ def __str__(self):
RayCgraphCapacityExceeded,
UnserializableException,
ActorAlreadyExistsError,
ActorHandleNotFoundError,
AuthenticationError,
]
28 changes: 28 additions & 0 deletions python/ray/tests/test_ray_shutdown.py
Original file line number Diff line number Diff line change
Expand Up @@ -508,5 +508,33 @@ def verify():
wait_for_condition(verify)


@pytest.mark.skipif(platform.system() == "Windows", reason="Hang on Windows.")
def test_kill_actor_after_restart(shutdown_only):
"""Test that killing an actor from a previous session raises a helpful error."""
# Set include_dashboard=False to have faster startup.
ray.init(num_cpus=1, include_dashboard=False)

@ray.remote
class A:
pass

a = A.remote()

# Restart ray
ray.shutdown()
ray.init(num_cpus=1, include_dashboard=False)

# Attempting to kill an actor from the previous session should raise
# a helpful error message instead of crashing the interpreter.
with pytest.raises(
ray.exceptions.ActorHandleNotFoundError,
match="ActorHandle objects are not valid across Ray sessions",
):
ray.kill(a)

ray.shutdown()
wait_for_condition(lambda: len(get_all_ray_worker_processes()) == 0)


if __name__ == "__main__":
sys.exit(pytest.main(["-sv", __file__]))
8 changes: 6 additions & 2 deletions src/ray/core_worker/core_worker.cc
Original file line number Diff line number Diff line change
Expand Up @@ -2574,12 +2574,16 @@ Status CoreWorker::KillActor(const ActorID &actor_id, bool force_kill, bool no_r
} else {
std::stringstream stream;
stream << "Failed to find a corresponding actor handle for " << actor_id;
cb(Status::Invalid(stream.str()));
cb(Status::NotFound(stream.str()));
}
},
"CoreWorker.KillActor");
const auto &status = f.get();
actor_manager_->OnActorKilled(actor_id);
// Only call OnActorKilled if the kill was successful (status is OK).
// If the actor handle doesn't exist, OnActorKilled would crash.
if (status.ok()) {
actor_manager_->OnActorKilled(actor_id);
}
return status;
}

Expand Down