Skip to content

[core] Miscellaneous cleanup around storing task outputs #54908

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 our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Merged
merged 11 commits into from
Jul 29, 2025
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
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
27 changes: 16 additions & 11 deletions python/ray/_raylet.pyx
Original file line number Diff line number Diff line change
Expand Up @@ -2380,11 +2380,7 @@ cdef CRayStatus task_execution_handler(
traceback_str = str(e)
logger.error("Exception raised "
f"in creation task: {traceback_str}")
# Cython's bug that doesn't allow reference assignment,
# this is a workaroud.
# See https://github.com/cython/cython/issues/1863
(&creation_task_exception_pb_bytes)[0] = (
ray_error_to_memory_buf(e))
creation_task_exception_pb_bytes = ray_error_to_memory_buf(e)
sys_exit.is_creation_task_error = True
sys_exit.init_error_message = (
"Exception raised from an actor init method. "
Expand Down Expand Up @@ -4411,18 +4407,27 @@ cdef class CoreWorker:
serialized_object.contained_object_refs)

if not self.store_task_output(
serialized_object, return_id,
serialized_object,
return_id,
c_ref_generator_id,
data_size, metadata, contained_id, caller_address,
&task_output_inlined_bytes, return_ptr):
data_size,
metadata,
contained_id,
caller_address,
&task_output_inlined_bytes,
return_ptr):
# If the object already exists, but we fail to pin the copy, it
# means the existing copy might've gotten evicted. Try to
# create another copy.
self.store_task_output(
serialized_object, return_id,
serialized_object,
return_id,
c_ref_generator_id,
data_size, metadata,
contained_id, caller_address, &task_output_inlined_bytes,
data_size,
metadata,
contained_id,
caller_address,
&task_output_inlined_bytes,
return_ptr)
num_outputs_stored += 1

Expand Down
1 change: 0 additions & 1 deletion python/ray/includes/common.pxd
Original file line number Diff line number Diff line change
Expand Up @@ -125,7 +125,6 @@ cdef extern from "ray/common/status.h" namespace "ray" nogil:
c_bool IsTimedOut()
c_bool IsInvalidArgument()
c_bool IsInterrupted()
c_bool ShouldExitWorker()
c_bool IsObjectNotFound()
c_bool IsNotFound()
c_bool IsObjectUnknownOwner()
Expand Down
5 changes: 0 additions & 5 deletions src/mock/ray/object_manager/plasma/client.h
Original file line number Diff line number Diff line change
Expand Up @@ -45,11 +45,6 @@ class MockPlasmaClient : public PlasmaClientInterface {
bool is_from_worker),
(override));

MOCK_METHOD(Status,
ExperimentalMutableObjectRegisterWriter,
(const ObjectID &object_id),
(override));

MOCK_METHOD(Status,
GetExperimentalMutableObject,
(const ObjectID &object_id, std::unique_ptr<MutableObject> *mutable_object),
Expand Down
5 changes: 0 additions & 5 deletions src/ray/common/status.h
Original file line number Diff line number Diff line change
Expand Up @@ -274,11 +274,6 @@ class RAY_EXPORT Status {
bool IsRedisError() const { return code() == StatusCode::RedisError; }
bool IsTimedOut() const { return code() == StatusCode::TimedOut; }
bool IsInterrupted() const { return code() == StatusCode::Interrupted; }
bool ShouldExitWorker() const {
return code() == StatusCode::IntentionalSystemExit ||
code() == StatusCode::UnexpectedSystemExit ||
code() == StatusCode::CreationTaskError;
}
bool IsIntentionalSystemExit() const {
return code() == StatusCode::IntentionalSystemExit;
}
Expand Down
14 changes: 6 additions & 8 deletions src/ray/core_worker/core_worker.cc
Original file line number Diff line number Diff line change
Expand Up @@ -2200,7 +2200,7 @@ void CoreWorker::TriggerGlobalGC() {
Status CoreWorker::GetPlasmaUsage(std::string &output) {
StatusOr<std::string> response = plasma_store_provider_->GetMemoryUsage();
if (response.ok()) {
output = response.value();
output = std::move(response.value());
}
return response.status();
}
Expand Down Expand Up @@ -3192,7 +3192,7 @@ Status CoreWorker::AllocateReturnObject(const ObjectID &object_id,
owner_address,
&data_buffer,
/*created_by_worker=*/true));
object_already_exists = !data_buffer;
object_already_exists = data_buffer == nullptr;
}
}
// Leave the return object as a nullptr if the object already exists.
Expand All @@ -3216,8 +3216,7 @@ Status CoreWorker::ExecuteTask(
std::string *application_error) {
RAY_LOG(DEBUG) << "Executing task, task info = " << task_spec.DebugString();

// If the worker is exited via Exit API, we shouldn't execute
// tasks anymore.
// If the worker is exited via Exit API, we shouldn't execute tasks anymore.
if (IsExiting()) {
absl::MutexLock lock(&mutex_);
return Status::IntentionalSystemExit(
Expand Down Expand Up @@ -3295,7 +3294,6 @@ Status CoreWorker::ExecuteTask(
}
}

Status status;
TaskType task_type = TaskType::NORMAL_TASK;
if (task_spec.IsActorCreationTask()) {
task_type = TaskType::ACTOR_CREATION_TASK;
Expand Down Expand Up @@ -3327,7 +3325,7 @@ Status CoreWorker::ExecuteTask(
name_of_concurrency_group_to_execute = task_spec.ConcurrencyGroupName();
}

status = options_.task_execution_callback(
Status status = options_.task_execution_callback(
task_spec.CallerAddress(),
task_type,
task_spec.GetName(),
Expand Down Expand Up @@ -3416,8 +3414,8 @@ Status CoreWorker::ExecuteTask(
Exit(rpc::WorkerExitType::SYSTEM_ERROR,
absl::StrCat("Worker exits unexpectedly. ", status.message()),
creation_task_exception_pb_bytes);
} else if (!status.ok()) {
RAY_LOG(FATAL) << "Unexpected task status type : " << status;
} else {
RAY_CHECK_OK(status) << "Unexpected task status type : " << status;
}
return status;
}
Expand Down
21 changes: 9 additions & 12 deletions src/ray/core_worker/store_provider/plasma_store_provider.cc
Original file line number Diff line number Diff line change
Expand Up @@ -128,10 +128,9 @@ Status CoreWorkerPlasmaStoreProvider::Create(const std::shared_ptr<Buffer> &meta
std::shared_ptr<Buffer> *data,
bool created_by_worker,
bool is_mutable) {
auto source = plasma::flatbuf::ObjectSource::CreatedByWorker;
if (!created_by_worker) {
source = plasma::flatbuf::ObjectSource::RestoredFromStorage;
}
const auto source = created_by_worker
? plasma::flatbuf::ObjectSource::CreatedByWorker
: plasma::flatbuf::ObjectSource::RestoredFromStorage;
Status status =
store_client_->CreateAndSpillIfNeeded(object_id,
owner_address,
Expand Down Expand Up @@ -164,8 +163,6 @@ Status CoreWorkerPlasmaStoreProvider::Create(const std::shared_ptr<Buffer> &meta
RAY_LOG_EVERY_MS(WARNING, 5000)
<< "Trying to put an object that already existed in plasma: " << object_id << ".";
status = Status::OK();
} else {
RAY_RETURN_NOT_OK(status);
}
return status;
}
Expand Down Expand Up @@ -203,24 +200,24 @@ Status CoreWorkerPlasmaStoreProvider::FetchAndGetFromPlasmaStore(
const auto &object_id = batch_ids[i];
std::shared_ptr<TrackedBuffer> data = nullptr;
std::shared_ptr<Buffer> metadata = nullptr;
if (plasma_results[i].data && plasma_results[i].data->Size()) {
if (plasma_results[i].data && plasma_results[i].data->Size() > 0) {
// We track the set of active data buffers in active_buffers_. On destruction,
// the buffer entry will be removed from the set via callback.
data = std::make_shared<TrackedBuffer>(
plasma_results[i].data, buffer_tracker_, object_id);
std::move(plasma_results[i].data), buffer_tracker_, object_id);
buffer_tracker_->Record(object_id, data.get(), get_current_call_site_());
}
if (plasma_results[i].metadata && plasma_results[i].metadata->Size()) {
metadata = plasma_results[i].metadata;
if (plasma_results[i].metadata && plasma_results[i].metadata->Size() > 0) {
metadata = std::move(plasma_results[i].metadata);
}
const auto result_object = std::make_shared<RayObject>(
auto result_object = std::make_shared<RayObject>(
data, metadata, std::vector<rpc::ObjectReference>());
(*results)[object_id] = result_object;
remaining.erase(object_id);
if (result_object->IsException()) {
RAY_CHECK(!result_object->IsInPlasmaError());
*got_exception = true;
}
(*results)[object_id] = std::move(result_object);
}
}

Expand Down
6 changes: 4 additions & 2 deletions src/ray/core_worker/transport/task_receiver.cc
Original file line number Diff line number Diff line change
Expand Up @@ -163,7 +163,8 @@ void TaskReceiver::HandleTask(rpc::PushTaskRequest request,
}
}
}
if (status.ShouldExitWorker()) {
if (status.IsIntentionalSystemExit() || status.IsUnexpectedSystemExit() ||
status.IsCreationTaskError()) {
// Don't allow the worker to be reused, even though the reply status is OK.
// The worker will be shutting down shortly.
reply->set_worker_exiting(true);
Expand All @@ -174,8 +175,9 @@ void TaskReceiver::HandleTask(rpc::PushTaskRequest request,
send_reply_callback(status, nullptr, nullptr);
}
} else {
RAY_CHECK_OK(status);
RAY_CHECK(objects_valid);
send_reply_callback(status, nullptr, nullptr);
send_reply_callback(Status::OK(), nullptr, nullptr);
}
};

Expand Down
Loading