Skip to content
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

[core] Always make request on fetch_local in wait even if num_objects in memory #50121

Open
wants to merge 4 commits into
base: master
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from 2 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
5 changes: 3 additions & 2 deletions bazel/ray.bzl
Original file line number Diff line number Diff line change
@@ -1,6 +1,6 @@
load("@com_github_google_flatbuffers//:build_defs.bzl", "flatbuffer_library_public")
load("@bazel_skylib//rules:copy_file.bzl", "copy_file")
load("@bazel_common//tools/maven:pom_file.bzl", "pom_file")
load("@bazel_skylib//rules:copy_file.bzl", "copy_file")
load("@com_github_google_flatbuffers//:build_defs.bzl", "flatbuffer_library_public")
load("@rules_cc//cc:defs.bzl", "cc_binary", "cc_library", "cc_test")

COPTS_WITHOUT_LOG = select({
Expand All @@ -14,6 +14,7 @@ COPTS_WITHOUT_LOG = select({
"//conditions:default": [
"-Wunused-result",
"-Wconversion-null",
"-Wmisleading-indentation",
Copy link
Contributor Author

Choose a reason for hiding this comment

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

],
}) + select({
"//:clang-cl": [
Expand Down
5 changes: 4 additions & 1 deletion src/ray/core_worker/core_worker.cc
Original file line number Diff line number Diff line change
Expand Up @@ -2095,7 +2095,10 @@ Status CoreWorker::Wait(const std::vector<ObjectID> &ids,
if (fetch_local) {
RetryObjectInPlasmaErrors(
memory_store_, worker_context_, memory_object_ids, plasma_object_ids, ready);
if (static_cast<int>(ready.size()) < num_objects && !plasma_object_ids.empty()) {
// We make the request to the plasma store even if we have num_objects ready since we
// want to at least make the request to pull these objects if the user specified
// fetch_local so the pulling can start.
if (!plasma_object_ids.empty()) {
RAY_RETURN_NOT_OK(plasma_store_provider_->Wait(
plasma_object_ids,
std::min(static_cast<int>(plasma_object_ids.size()),
Expand Down
25 changes: 25 additions & 0 deletions src/ray/raylet/node_manager.cc
Original file line number Diff line number Diff line change
Expand Up @@ -1759,6 +1759,31 @@ void NodeManager::ProcessWaitRequestMessage(
current_task_id,
/*ray_get=*/false);
}
if (message->num_ready_objects() == 0) {
// If we don't need to wait for any, return immediately after making the pull
// requests through AsyncResolveObjects above.
flatbuffers::FlatBufferBuilder fbb;
auto wait_reply = protocol::CreateWaitReply(fbb,
to_flatbuf(fbb, std::vector<ObjectID>{}),
to_flatbuf(fbb, std::vector<ObjectID>{}));
fbb.Finish(wait_reply);
const auto status =
client->WriteMessage(static_cast<int64_t>(protocol::MessageType::WaitReply),
fbb.GetSize(),
fbb.GetBufferPointer());
if (status.ok()) {
if (resolve_objects) {
AsyncResolveObjectsFinish(client, current_task_id);
}
} else {
// We failed to write to the client, so disconnect the client.
std::ostringstream stream;
stream << "Failed to write WaitReply to the client. Status " << status
<< ", message: " << status.message();
DisconnectClient(client, rpc::WorkerExitType::SYSTEM_ERROR, stream.str());
}
return;
}
uint64_t num_required_objects = static_cast<uint64_t>(message->num_ready_objects());
wait_manager_.Wait(
object_ids,
Expand Down
1 change: 0 additions & 1 deletion src/ray/raylet/wait_manager.cc
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,6 @@ void WaitManager::Wait(const std::vector<ObjectID> &object_ids,
<< "Waiting duplicate objects is not allowed. Please make sure all object IDs are "
"unique before calling `WaitManager::Wait`.";
RAY_CHECK(timeout_ms >= 0 || timeout_ms == -1);
RAY_CHECK_NE(num_required_objects, 0u);
RAY_CHECK_LE(num_required_objects, object_ids.size());

const uint64_t wait_id = next_wait_id_++;
Expand Down