Skip to content

Commit e5d651d

Browse files
authored
[Core] Remove Status::GrpcUnavailable and Status::GrpcUnknown since we have Status::RpcError (ray-project#46167)
Signed-off-by: Jiajun Yao <jeromeyjj@gmail.com>
1 parent 50cbe6a commit e5d651d

File tree

12 files changed

+20
-49
lines changed

12 files changed

+20
-49
lines changed

python/ray/tests/test_unavailable_actors.py

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -80,7 +80,7 @@ def body():
8080
# Break the grpc connection from this process to the actor process. The
8181
# next `ray.get` call should fail with ActorUnavailableError.
8282
close_common_connections(pid)
83-
with pytest.raises(ActorUnavailableError, match="Grpc"):
83+
with pytest.raises(ActorUnavailableError, match="RpcError"):
8484
ray.get(task)
8585
# Since the remote() call happens *before* the break, the actor did receive the
8686
# request, so the side effects are observable, and the actor recovered.
@@ -93,7 +93,7 @@ def body():
9393
# itself won't raise an error.
9494
close_common_connections(pid)
9595
task2 = a.slow_increment.remote(5, 0.1)
96-
with pytest.raises(ActorUnavailableError, match="Grpc"):
96+
with pytest.raises(ActorUnavailableError, match="RpcError"):
9797
ray.get(task2)
9898
assert ray.get(a.read.remote()) == 9
9999

@@ -206,7 +206,7 @@ def test_unavailable_then_actor_error(ray_start_regular):
206206
# calls get ActorUnavailableError.
207207
sigkill_actor(a)
208208

209-
with pytest.raises(ActorUnavailableError, match="Grpc"):
209+
with pytest.raises(ActorUnavailableError, match="RpcError"):
210210
print(ray.get(a.ping.remote("unavailable")))
211211
# When the actor is restarting, any method call raises ActorUnavailableError.
212212
with pytest.raises(ActorUnavailableError, match="The actor is restarting"):

src/ray/common/grpc_util.h

Lines changed: 2 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -108,13 +108,9 @@ inline Status GrpcStatusToRayStatus(const grpc::Status &grpc_status) {
108108
// See RayStatusToGrpcStatus for details.
109109
return Status(Status::StringToCode(grpc_status.error_message()),
110110
grpc_status.error_details());
111-
} else if (grpc_status.error_code() == grpc::StatusCode::UNAVAILABLE) {
112-
return Status::GrpcUnavailable(GrpcStatusToRayStatusMessage(grpc_status));
113111
} else {
114-
// TODO(jjyao) Use GrpcUnknown as the catch-all status for all
115-
// the unhandled grpc status.
116-
// If needed, we can define a ray status for each grpc status in the future.
117-
return Status::GrpcUnknown(GrpcStatusToRayStatusMessage(grpc_status));
112+
return Status::RpcError(GrpcStatusToRayStatusMessage(grpc_status),
113+
grpc_status.error_code());
118114
}
119115
}
120116

src/ray/common/status.cc

Lines changed: 0 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -58,8 +58,6 @@ namespace ray {
5858
#define STATUS_CODE_OBJECT_ALREADY_SEALED "ObjectAlreadySealed"
5959
#define STATUS_CODE_OBJECT_STORE_FULL "ObjectStoreFull"
6060
#define STATUS_CODE_TRANSIENT_OBJECT_STORE_FULL "TransientObjectStoreFull"
61-
#define STATUS_CODE_GRPC_UNAVAILABLE "GrpcUnavailable"
62-
#define STATUS_CODE_GRPC_UNKNOWN "GrpcUnknown"
6361
#define STATUS_CODE_OUT_OF_DISK "OutOfDisk"
6462
#define STATUS_CODE_OBJECT_UNKNOWN_OWNER "ObjectUnknownOwner"
6563
#define STATUS_CODE_RPC_ERROR "RpcError"
@@ -98,8 +96,6 @@ const absl::flat_hash_map<StatusCode, std::string> kCodeToStr = {
9896
{StatusCode::ObjectAlreadySealed, STATUS_CODE_OBJECT_ALREADY_SEALED},
9997
{StatusCode::ObjectStoreFull, STATUS_CODE_OBJECT_STORE_FULL},
10098
{StatusCode::TransientObjectStoreFull, STATUS_CODE_TRANSIENT_OBJECT_STORE_FULL},
101-
{StatusCode::GrpcUnavailable, STATUS_CODE_GRPC_UNAVAILABLE},
102-
{StatusCode::GrpcUnknown, STATUS_CODE_GRPC_UNKNOWN},
10399
{StatusCode::OutOfDisk, STATUS_CODE_OUT_OF_DISK},
104100
{StatusCode::ObjectUnknownOwner, STATUS_CODE_OBJECT_UNKNOWN_OWNER},
105101
{StatusCode::RpcError, STATUS_CODE_RPC_ERROR},

src/ray/common/status.h

Lines changed: 0 additions & 19 deletions
Original file line numberDiff line numberDiff line change
@@ -103,13 +103,6 @@ enum class StatusCode : char {
103103
ObjectAlreadySealed = 23,
104104
ObjectStoreFull = 24,
105105
TransientObjectStoreFull = 25,
106-
// grpc status
107-
// This represents UNAVAILABLE status code
108-
// returned by grpc.
109-
GrpcUnavailable = 26,
110-
// This represents all other status codes
111-
// returned by grpc that are not defined above.
112-
GrpcUnknown = 27,
113106
// Object store is both out of memory and
114107
// out of disk.
115108
OutOfDisk = 28,
@@ -243,14 +236,6 @@ class RAY_EXPORT Status {
243236
return Status(StatusCode::OutOfDisk, msg);
244237
}
245238

246-
static Status GrpcUnavailable(const std::string &msg) {
247-
return Status(StatusCode::GrpcUnavailable, msg);
248-
}
249-
250-
static Status GrpcUnknown(const std::string &msg) {
251-
return Status(StatusCode::GrpcUnknown, msg);
252-
}
253-
254239
static Status RpcError(const std::string &msg, int rpc_code) {
255240
return Status(StatusCode::RpcError, msg, rpc_code);
256241
}
@@ -306,10 +291,6 @@ class RAY_EXPORT Status {
306291
bool IsTransientObjectStoreFull() const {
307292
return code() == StatusCode::TransientObjectStoreFull;
308293
}
309-
bool IsGrpcUnavailable() const { return code() == StatusCode::GrpcUnavailable; }
310-
bool IsGrpcUnknown() const { return code() == StatusCode::GrpcUnknown; }
311-
312-
bool IsGrpcError() const { return IsGrpcUnknown() || IsGrpcUnavailable(); }
313294

314295
bool IsRpcError() const { return code() == StatusCode::RpcError; }
315296

src/ray/common/test/status_test.cc

Lines changed: 4 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -49,11 +49,13 @@ TEST_F(StatusTest, GrpcStatusToRayStatus) {
4949

5050
grpc_status = grpc::Status(grpc::StatusCode::UNAVAILABLE, "foo", "bar");
5151
ray_status = GrpcStatusToRayStatus(grpc_status);
52-
ASSERT_TRUE(ray_status.IsGrpcUnavailable());
52+
ASSERT_TRUE(ray_status.IsRpcError());
53+
ASSERT_EQ(ray_status.rpc_code(), grpc::StatusCode::UNAVAILABLE);
5354

5455
grpc_status = grpc::Status(grpc::StatusCode::UNKNOWN, "foo", "bar");
5556
ray_status = GrpcStatusToRayStatus(grpc_status);
56-
ASSERT_TRUE(ray_status.IsGrpcUnknown());
57+
ASSERT_TRUE(ray_status.IsRpcError());
58+
ASSERT_EQ(ray_status.rpc_code(), grpc::StatusCode::UNKNOWN);
5759

5860
grpc_status = grpc::Status(grpc::StatusCode::ABORTED, "foo", "bar");
5961
ray_status = GrpcStatusToRayStatus(grpc_status);

src/ray/core_worker/core_worker_process.cc

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -206,7 +206,8 @@ void CoreWorkerProcessImpl::InitializeSystemConfig() {
206206
}
207207

208208
// If there's no more attempt to try.
209-
if (status.IsGrpcUnavailable()) {
209+
if (status.IsRpcError() &&
210+
status.rpc_code() == grpc::StatusCode::UNAVAILABLE) {
210211
std::ostringstream ss;
211212
ss << "Failed to get the system config from raylet because "
212213
<< "it is dead. Worker will terminate. Status: " << status

src/ray/core_worker/test/direct_task_transport_test.cc

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -287,7 +287,7 @@ class MockRayletClient : public WorkerLeaseInterface {
287287
return false;
288288
} else {
289289
auto callback = callbacks.front();
290-
callback(Status::GrpcUnavailable("unavailable"), reply);
290+
callback(Status::RpcError("unavailable", grpc::StatusCode::UNAVAILABLE), reply);
291291
callbacks.pop_front();
292292
return true;
293293
}

src/ray/core_worker/test/task_event_buffer_test.cc

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -271,7 +271,7 @@ TEST_F(TaskEventBufferTest, TestFailedFlush) {
271271
.Times(2)
272272
.WillOnce([&](std::unique_ptr<rpc::TaskEventData> actual_data,
273273
ray::gcs::StatusCallback callback) {
274-
callback(Status::GrpcUnknown("grpc error"));
274+
callback(Status::RpcError("grpc error", grpc::StatusCode::UNKNOWN));
275275
return Status::OK();
276276
})
277277
.WillOnce([&](std::unique_ptr<rpc::TaskEventData> actual_data,

src/ray/core_worker/transport/direct_task_transport.cc

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -528,7 +528,8 @@ void CoreWorkerDirectTaskSubmitter::RequestNewWorkerIfNeeded(
528528
RequestNewWorkerIfNeeded(scheduling_key);
529529

530530
} else {
531-
if (status.IsGrpcUnavailable()) {
531+
if (status.IsRpcError() &&
532+
status.rpc_code() == grpc::StatusCode::UNAVAILABLE) {
532533
RAY_LOG(WARNING)
533534
<< "The worker failed to receive a response from the local "
534535
<< "raylet because the raylet is unavailable (crashed). "

src/ray/gcs/gcs_client/gcs_client.cc

Lines changed: 3 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -184,7 +184,7 @@ Status PythonGcsClient::Connect(const ClusterID &cluster_id,
184184
RAY_LOG(DEBUG) << "Received cluster ID from GCS server: " << cluster_id_;
185185
RAY_CHECK(!cluster_id_.IsNil());
186186
break;
187-
} else if (!connect_status.IsGrpcError()) {
187+
} else if (!connect_status.IsRpcError()) {
188188
return HandleGcsError(reply.status());
189189
}
190190
std::this_thread::sleep_for(std::chrono::milliseconds(1000));
@@ -415,15 +415,9 @@ Status PythonGcsClient::PinRuntimeEnvUri(const std::string &uri,
415415
if (status.ok()) {
416416
if (reply.status().code() == static_cast<int>(StatusCode::OK)) {
417417
return Status::OK();
418-
} else if (reply.status().code() == static_cast<int>(StatusCode::GrpcUnavailable)) {
419-
std::string msg =
420-
"Failed to pin URI reference for " + uri + " due to the GCS being " +
421-
"unavailable, most likely it has crashed: " + reply.status().message() + ".";
422-
return Status::GrpcUnavailable(msg);
418+
} else {
419+
return Status(StatusCode(reply.status().code()), reply.status().message());
423420
}
424-
std::string msg = "Failed to pin URI reference for " + uri +
425-
" due to unexpected error " + reply.status().message() + ".";
426-
return Status::GrpcUnknown(msg);
427421
}
428422
return Status::RpcError(status.error_message(), status.error_code());
429423
}

src/ray/gcs/gcs_server/test/gcs_placement_group_scheduler_test.cc

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -994,7 +994,7 @@ TEST_F(GcsPlacementGroupSchedulerTest, TestNodeErrorDuringCommittingResources) {
994994
ASSERT_TRUE(raylet_clients_[0]->GrantCommitBundleResources());
995995
// node1 is experiencing transient connection failure.
996996
ASSERT_TRUE(raylet_clients_[1]->GrantCommitBundleResources(
997-
ray::Status::GrpcUnavailable("unavailable")));
997+
ray::Status::RpcError("unavailable", grpc::StatusCode::UNAVAILABLE)));
998998
WaitPlacementGroupPendingDone(1, GcsPlacementGroupStatus::FAILURE);
999999
}
10001000

src/ray/rpc/gcs_server/gcs_rpc_client.h

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -238,7 +238,7 @@ class GcsRpcClient {
238238
callback(status, reply);
239239
}
240240
delete executor;
241-
} else if (!status.IsGrpcError()) {
241+
} else if (!status.IsRpcError()) {
242242
callback(status, reply);
243243
delete executor;
244244
} else {

0 commit comments

Comments
 (0)