Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
35 commits
Select commit Hold shift + click to select a range
db60fca
Refactor tests to use real reference counter
Oct 10, 2025
df1e34d
Merge branch 'master' into actor_manager_rc
Kunchd Oct 10, 2025
5b746e4
Enforce same function call ordering for EmplaceNewActorHandle
Oct 10, 2025
6d29949
Merge branch 'actor_manager_rc' of github.com:Kunchd/ray into actor_m…
Oct 10, 2025
7820ec6
Address comments for style and removal of mocks
Oct 10, 2025
8527f32
Address comments for EmplaceNewActorHandle to throw ray check failure
Oct 14, 2025
2123374
Merge branch 'master' into actor_manager_rc
Kunchd Oct 14, 2025
31e1fdd
Merge branch 'master' into actor_manager_rc
Kunchd Oct 14, 2025
f345454
Bring back sentinel to make EmplaceNewActorHandle atomic
Oct 14, 2025
feb8818
Merge branch 'master' into actor_manager_rc
Kunchd Oct 15, 2025
6328581
Merge branch 'master' into actor_manager_rc
israbbani Oct 16, 2025
992460e
Refactor tests to use real reference counter
Oct 10, 2025
fafc519
Enforce same function call ordering for EmplaceNewActorHandle
Oct 10, 2025
c92c29e
Address comments for style and removal of mocks
Oct 10, 2025
db8f166
Address comments for EmplaceNewActorHandle to throw ray check failure
Oct 14, 2025
773a117
Bring back sentinel to make EmplaceNewActorHandle atomic
Oct 14, 2025
3ceb574
Merge branch 'actor_manager_rc' of github.com:Kunchd/ray into actor_m…
Oct 22, 2025
1bfe1b2
Merge branch 'master' into actor_manager_rc
Oct 22, 2025
ab7841d
Merge branch 'master' into actor_manager_rc
Kunchd Oct 22, 2025
3f9bb16
Add check for sentinels when accessing actor handles
Oct 22, 2025
b1bf3a6
Merge branch 'actor_manager_rc' of github.com:Kunchd/ray into actor_m…
Oct 22, 2025
38b565a
Merge branch 'master' into actor_manager_rc
israbbani Oct 23, 2025
8daa31e
Revert EmplaceNewActorHandle to return bool instead of ray check
Oct 25, 2025
52a24f2
Merge branch 'actor_manager_rc' of github.com:Kunchd/ray into actor_m…
Oct 25, 2025
c64bb6b
Address actor_manager_test compiler error
Oct 25, 2025
0e8350d
Remove actor manager sentinel
Oct 28, 2025
00b06fa
Merge branch 'master' into actor_manager_rc
Kunchd Oct 28, 2025
616c110
Merge branch 'master' into actor_manager_rc
Nov 13, 2025
57fbf0f
Update tests using reference counting to use fake gauge
Nov 13, 2025
400ba1b
Merge branch 'master' into actor_manager_rc
Kunchd Nov 13, 2025
667a31d
Ensure object is properly stored in object store in tests with real r…
Nov 13, 2025
e553489
Address comments to for logging and ptr style
Nov 14, 2025
c7cde27
Merge branch 'master' into actor_manager_rc
Nov 15, 2025
bc9457c
Update src/ray/core_worker/core_worker.h
Kunchd Nov 17, 2025
8e3a2a6
Merge branch 'master' into actor_manager_rc
Kunchd Nov 17, 2025
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
218 changes: 0 additions & 218 deletions src/mock/ray/core_worker/reference_counter.h

This file was deleted.

26 changes: 17 additions & 9 deletions src/ray/core_worker/actor_manager.cc
Original file line number Diff line number Diff line change
Expand Up @@ -80,10 +80,10 @@ std::pair<std::shared_ptr<const ActorHandle>, Status> ActorManager::GetNamedActo
if (status.ok()) {
auto actor_handle = std::make_unique<ActorHandle>(actor_table_data, task_spec);
actor_id = actor_handle->GetActorID();
AddNewActorHandle(std::move(actor_handle),
call_site,
caller_address,
/*owned*/ false);
EmplaceNewActorHandle(std::move(actor_handle),
call_site,
caller_address,
/*owned*/ false);
} else {
// Use a NIL actor ID to signal that the actor wasn't found.
RAY_LOG(DEBUG) << "Failed to look up actor with name: " << name;
Expand Down Expand Up @@ -129,16 +129,24 @@ std::shared_ptr<ActorHandle> ActorManager::GetActorHandleIfExists(
return nullptr;
}

bool ActorManager::AddNewActorHandle(std::unique_ptr<ActorHandle> actor_handle,
const std::string &call_site,
const rpc::Address &caller_address,
bool owned) {
bool ActorManager::EmplaceNewActorHandle(std::unique_ptr<ActorHandle> actor_handle,
const std::string &call_site,
const rpc::Address &caller_address,
bool owned) {
const auto &actor_id = actor_handle->GetActorID();
const auto actor_creation_return_id = ObjectID::ForActorHandle(actor_id);

{
absl::MutexLock lock(&mutex_);
if (actor_handles_.contains(actor_id)) {
return false;
}
}

// Detached actor doesn't need ref counting.
if (owned) {
reference_counter_.AddOwnedObject(actor_creation_return_id,
/*inner_ids=*/{},
/*contained_ids=*/{},
caller_address,
call_site,
/*object_size*/ -1,
Expand Down
9 changes: 5 additions & 4 deletions src/ray/core_worker/actor_manager.h
Original file line number Diff line number Diff line change
Expand Up @@ -106,6 +106,7 @@ class ActorManager {
///
/// NOTE: Getting an actor handle from GCS (named actor) is considered as adding a new
/// actor handle.
/// NOTE: Attempting to add the same actor in parallel can cause RAY CHECK failure.
///
/// \param actor_handle The handle to the actor.
/// \param[in] call_site The caller's site.
Expand All @@ -114,10 +115,10 @@ class ActorManager {
/// task.
/// \return True if the handle was added and False if we already had a handle to
/// the same actor.
bool AddNewActorHandle(std::unique_ptr<ActorHandle> actor_handle,
const std::string &call_site,
const rpc::Address &caller_address,
bool owned);
bool EmplaceNewActorHandle(std::unique_ptr<ActorHandle> actor_handle,
const std::string &call_site,
const rpc::Address &caller_address,
bool owned);

/// Wait for actor reference deletion.
///
Expand Down
6 changes: 4 additions & 2 deletions src/ray/core_worker/core_worker.cc
Original file line number Diff line number Diff line change
Expand Up @@ -2164,9 +2164,11 @@ Status CoreWorker::CreateActor(const RayFunction &function,
// Add the actor handle before we submit the actor creation task, since the
// actor handle must be in scope by the time the GCS sends the
// WaitForActorRefDeletedRequest.
RAY_CHECK(actor_manager_->AddNewActorHandle(
RAY_CHECK(actor_manager_->EmplaceNewActorHandle(
std::move(actor_handle), CurrentCallSite(), rpc_address_, /*owned=*/!is_detached))
<< "Actor " << actor_id << " already exists";
<< "Attempt to emplace new actor handle for the actor being created with actor id: "
<< actor_id
<< " failed because an actor handle with the same actor id has already been added";
*return_actor_id = actor_id;
TaskSpecification task_spec = std::move(builder).ConsumeAndBuild();
RAY_LOG(DEBUG) << "Submitting actor creation task " << task_spec.DebugString();
Expand Down
3 changes: 3 additions & 0 deletions src/ray/core_worker/core_worker.h
Original file line number Diff line number Diff line change
Expand Up @@ -865,6 +865,9 @@ class CoreWorker {

/// Create an actor.
///
/// NOTE: RAY CHECK fails if an actor handle with the same actor id has already been
/// added, or if the scheduling strategy for actor creation is not set.
///
/// \param[in] caller_id ID of the task submitter.
/// \param[in] function The remote function that generates the actor object.
/// \param[in] args Arguments of this task.
Expand Down
2 changes: 2 additions & 0 deletions src/ray/core_worker/reference_counter_interface.h
Original file line number Diff line number Diff line change
Expand Up @@ -125,6 +125,8 @@ class ReferenceCounterInterface {
/// owner ID will change for workers executing normal tasks and it is
/// possible to have leftover references after a task has finished.
///
/// NOTE: RAY CHECK fails if the object was already added.
///
/// \param[in] object_id The ID of the object that we own.
/// \param[in] contained_ids ObjectIDs that are contained in the object's value.
/// As long as the object_id is in scope, the inner objects should not be GC'ed.
Expand Down
5 changes: 5 additions & 0 deletions src/ray/core_worker/task_submission/tests/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,10 @@ ray_cc_test(
tags = ["team:core"],
deps = [
"//:ray_mock",
"//src/ray/core_worker:reference_counter",
"//src/ray/core_worker/task_submission:actor_task_submitter",
"//src/ray/pubsub:fake_publisher",
"//src/ray/pubsub:fake_subscriber",
"@com_google_googletest//:gtest",
"@com_google_googletest//:gtest_main",
],
Expand All @@ -54,6 +57,8 @@ ray_cc_test(
"//src/ray/core_worker:reference_counter",
"//src/ray/core_worker:task_manager",
"//src/ray/core_worker_rpc_client:fake_core_worker_client",
"//src/ray/pubsub:fake_publisher",
"//src/ray/pubsub:fake_subscriber",
"@com_google_googletest//:gtest",
"@com_google_googletest//:gtest_main",
],
Expand Down
Loading