Skip to content
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
65 changes: 62 additions & 3 deletions python/ray/tests/test_raylet_fault_tolerance.py
Original file line number Diff line number Diff line change
Expand Up @@ -92,7 +92,7 @@ def node_is_dead():
# propagated to all the raylets. Since this is inherently racy, we block CancelResourceReserve RPCs
# from ever succeeding to make this test deterministic.
@pytest.fixture
def inject_rpc_failures(monkeypatch, request):
def inject_release_unused_bundles_rpc_failure(monkeypatch, request):
deterministic_failure = request.param
monkeypatch.setenv(
"RAY_testing_rpc_failure",
Expand All @@ -102,14 +102,16 @@ def inject_rpc_failures(monkeypatch, request):
)


@pytest.mark.parametrize("inject_rpc_failures", ["request", "response"], indirect=True)
@pytest.mark.parametrize(
"inject_release_unused_bundles_rpc_failure", ["request", "response"], indirect=True
)
@pytest.mark.parametrize(
"ray_start_cluster_head_with_external_redis",
[{"num_cpus": 1}],
indirect=True,
)
def test_release_unused_bundles_idempotent(
inject_rpc_failures,
inject_release_unused_bundles_rpc_failure,
ray_start_cluster_head_with_external_redis,
):
cluster = ray_start_cluster_head_with_external_redis
Expand Down Expand Up @@ -141,6 +143,63 @@ def task():
assert result == "success"


@pytest.fixture
def inject_notify_gcs_restart_rpc_failure(monkeypatch, request):
deterministic_failure = request.param
monkeypatch.setenv(
"RAY_testing_rpc_failure",
"NodeManagerService.grpc_client.NotifyGCSRestart=1:"
+ ("100:0" if deterministic_failure == "request" else "0:100"),
)


@pytest.mark.parametrize(
"inject_notify_gcs_restart_rpc_failure", ["request", "response"], indirect=True
)
@pytest.mark.parametrize(
"ray_start_cluster_head_with_external_redis",
[
{
"_system_config": {
# Extending the fallback timeout to focus on death
# notification received from GCS_ACTOR_CHANNEL pubsub
"timeout_ms_task_wait_for_death_info": 10000,
}
}
],
indirect=True,
)
def test_notify_gcs_restart_idempotent(
inject_notify_gcs_restart_rpc_failure,
ray_start_cluster_head_with_external_redis,
):
cluster = ray_start_cluster_head_with_external_redis

@ray.remote(num_cpus=1, max_restarts=0)
class DummyActor:
def get_pid(self):
return psutil.Process().pid

def ping(self):
return "pong"

actor = DummyActor.remote()
ray.get(actor.ping.remote())
actor_pid = ray.get(actor.get_pid.remote())

cluster.head_node.kill_gcs_server()
cluster.head_node.start_gcs_server()

p = psutil.Process(actor_pid)
p.kill()

# If the actor death notification is not received from the GCS pubsub, this will timeout since
# the fallback via wait_for_death_info_tasks in the actor task submitter will never trigger
# since it's set to 10 seconds.
with pytest.raises(ray.exceptions.RayActorError):
ray.get(actor.ping.remote(), timeout=5)


def test_kill_local_actor_rpc_retry_and_idempotency(monkeypatch, shutdown_only):
"""Test that KillLocalActor RPC retries work correctly and guarantee actor death.
Not testing response since the actor is killed either way.
Expand Down
9 changes: 4 additions & 5 deletions src/mock/ray/gcs_client/accessor.h
Original file line number Diff line number Diff line change
Expand Up @@ -63,13 +63,13 @@ class MockActorInfoAccessor : public ActorInfoAccessor {
(const TaskSpecification &task_spec,
const rpc::ClientCallback<rpc::CreateActorReply> &callback),
(override));
MOCK_METHOD(Status,
MOCK_METHOD(void,
AsyncSubscribe,
(const ActorID &actor_id,
(const SubscribeCallback<ActorID, rpc::ActorTableData> &subscribe),
const StatusCallback &done),
(override));
MOCK_METHOD(Status, AsyncUnsubscribe, (const ActorID &actor_id), (override));
MOCK_METHOD(void, AsyncUnsubscribe, (const ActorID &actor_id), (override));
MOCK_METHOD(void, AsyncResubscribe, (), (override));
MOCK_METHOD(bool, IsActorUnsubscribed, (const ActorID &actor_id), (override));
};
Expand All @@ -91,7 +91,7 @@ class MockJobInfoAccessor : public JobInfoAccessor {
AsyncMarkFinished,
(const JobID &job_id, const StatusCallback &callback),
(override));
MOCK_METHOD(Status,
MOCK_METHOD(void,
AsyncSubscribeAll,
((const SubscribeCallback<JobID, rpc::JobTableData> &subscribe),
const StatusCallback &done),
Expand Down Expand Up @@ -191,7 +191,6 @@ class MockNodeResourceInfoAccessor : public NodeResourceInfoAccessor {
AsyncGetAllAvailableResources,
(const MultiItemCallback<rpc::AvailableResources> &callback),
(override));
MOCK_METHOD(void, AsyncResubscribe, (), (override));
MOCK_METHOD(void,
AsyncGetAllResourceUsage,
(const ItemCallback<rpc::ResourceUsageBatchData> &callback),
Expand Down Expand Up @@ -231,7 +230,7 @@ namespace gcs {

class MockWorkerInfoAccessor : public WorkerInfoAccessor {
public:
MOCK_METHOD(Status,
MOCK_METHOD(void,
AsyncSubscribeToWorkerFailures,
(const ItemCallback<rpc::WorkerDeltaData> &subscribe,
const StatusCallback &done),
Expand Down
4 changes: 2 additions & 2 deletions src/ray/core_worker/actor_manager.cc
Original file line number Diff line number Diff line change
Expand Up @@ -308,7 +308,7 @@ void ActorManager::SubscribeActorState(const ActorID &actor_id) {
this,
std::placeholders::_1,
std::placeholders::_2);
RAY_CHECK_OK(gcs_client_->Actors().AsyncSubscribe(
gcs_client_->Actors().AsyncSubscribe(
actor_id,
actor_notification_callback,
[this, actor_id, cached_actor_name](Status status) {
Expand All @@ -323,7 +323,7 @@ void ActorManager::SubscribeActorState(const ActorID &actor_id) {
cached_actor_name_to_ids_.emplace(cached_actor_name, actor_id);
}
}
}));
});
}

void ActorManager::MarkActorKilledOrOutOfScope(
Expand Down
3 changes: 1 addition & 2 deletions src/ray/core_worker/tests/actor_manager_test.cc
Original file line number Diff line number Diff line change
Expand Up @@ -37,15 +37,14 @@ class MockActorInfoAccessor : public gcs::ActorInfoAccessor {

~MockActorInfoAccessor() {}

Status AsyncSubscribe(
void AsyncSubscribe(
const ActorID &actor_id,
const gcs::SubscribeCallback<ActorID, rpc::ActorTableData> &subscribe,
const gcs::StatusCallback &done) {
auto callback_entry = std::make_pair(actor_id, subscribe);
callback_map_.emplace(actor_id, subscribe);
subscribe_finished_callback_map_[actor_id] = done;
actor_subscribed_times_[actor_id]++;
return Status::OK();
}

bool ActorStateNotificationPublished(const ActorID &actor_id,
Expand Down
9 changes: 8 additions & 1 deletion src/ray/gcs/gcs_node_manager.cc
Original file line number Diff line number Diff line change
Expand Up @@ -690,7 +690,14 @@ void GcsNodeManager::Initialize(const GcsInitData &gcs_init_data) {
auto remote_address = rpc::RayletClientPool::GenerateRayletAddress(
node_id, node_info.node_manager_address(), node_info.node_manager_port());
auto raylet_client = raylet_client_pool_->GetOrConnectByAddress(remote_address);
raylet_client->NotifyGCSRestart(nullptr);
raylet_client->NotifyGCSRestart(
[](const Status &status, const rpc::NotifyGCSRestartReply &reply) {
if (!status.ok()) {
RAY_LOG(WARNING) << "NotifyGCSRestart failed. This is expected if the "
"target node has died. Status: "
<< status;
}
});
} else if (node_info.state() == rpc::GcsNodeInfo::DEAD) {
dead_nodes_.emplace(node_id, std::make_shared<rpc::GcsNodeInfo>(node_info));
sorted_dead_node_list_.emplace_back(node_id, node_info.end_time_ms());
Expand Down
44 changes: 16 additions & 28 deletions src/ray/gcs_rpc_client/accessor.cc
Original file line number Diff line number Diff line change
Expand Up @@ -69,7 +69,7 @@ void JobInfoAccessor::AsyncMarkFinished(const JobID &job_id,
});
}

Status JobInfoAccessor::AsyncSubscribeAll(
void JobInfoAccessor::AsyncSubscribeAll(
const SubscribeCallback<JobID, rpc::JobTableData> &subscribe,
const StatusCallback &done) {
RAY_CHECK(subscribe != nullptr);
Expand All @@ -91,9 +91,9 @@ Status JobInfoAccessor::AsyncSubscribeAll(
/*timeout_ms=*/-1);
};
subscribe_operation_ = [this, subscribe](const StatusCallback &done_callback) {
return client_impl_->GetGcsSubscriber().SubscribeAllJobs(subscribe, done_callback);
client_impl_->GetGcsSubscriber().SubscribeAllJobs(subscribe, done_callback);
};
return subscribe_operation_(
subscribe_operation_(
[this, done](const Status &status) { fetch_all_data_operation_(done); });
}

Expand All @@ -105,9 +105,9 @@ void JobInfoAccessor::AsyncResubscribe() {
};

if (subscribe_operation_ != nullptr) {
RAY_CHECK_OK(subscribe_operation_([this, fetch_all_done](const Status &) {
subscribe_operation_([this, fetch_all_done](const Status &) {
fetch_all_data_operation_(fetch_all_done);
}));
});
}
}

Expand Down Expand Up @@ -391,7 +391,7 @@ void ActorInfoAccessor::AsyncReportActorOutOfScope(
timeout_ms);
}

Status ActorInfoAccessor::AsyncSubscribe(
void ActorInfoAccessor::AsyncSubscribe(
const ActorID &actor_id,
const SubscribeCallback<ActorID, rpc::ActorTableData> &subscribe,
const StatusCallback &done) {
Expand All @@ -418,28 +418,27 @@ Status ActorInfoAccessor::AsyncSubscribe(
absl::MutexLock lock(&mutex_);
resubscribe_operations_[actor_id] =
[this, actor_id, subscribe](const StatusCallback &subscribe_done) {
return client_impl_->GetGcsSubscriber().SubscribeActor(
client_impl_->GetGcsSubscriber().SubscribeActor(
actor_id, subscribe, subscribe_done);
};
fetch_data_operations_[actor_id] = fetch_data_operation;
}

return client_impl_->GetGcsSubscriber().SubscribeActor(
client_impl_->GetGcsSubscriber().SubscribeActor(
actor_id, subscribe, [fetch_data_operation, done](const Status &) {
fetch_data_operation(done);
});
}

Status ActorInfoAccessor::AsyncUnsubscribe(const ActorID &actor_id) {
void ActorInfoAccessor::AsyncUnsubscribe(const ActorID &actor_id) {
RAY_LOG(DEBUG).WithField(actor_id).WithField(actor_id.JobId())
<< "Cancelling subscription to an actor";
auto status = client_impl_->GetGcsSubscriber().UnsubscribeActor(actor_id);
client_impl_->GetGcsSubscriber().UnsubscribeActor(actor_id);
absl::MutexLock lock(&mutex_);
resubscribe_operations_.erase(actor_id);
fetch_data_operations_.erase(actor_id);
RAY_LOG(DEBUG).WithField(actor_id).WithField(actor_id.JobId())
<< "Finished cancelling subscription to an actor";
return status;
}

void ActorInfoAccessor::AsyncResubscribe() {
Expand All @@ -449,7 +448,7 @@ void ActorInfoAccessor::AsyncResubscribe() {
// server first, then fetch data from the GCS server.
absl::MutexLock lock(&mutex_);
for (auto &[actor_id, resubscribe_op] : resubscribe_operations_) {
RAY_CHECK_OK(resubscribe_op([this, id = actor_id](const Status &status) {
resubscribe_op([this, id = actor_id](const Status &status) {
absl::MutexLock callback_lock(&mutex_);
auto fetch_data_operation = fetch_data_operations_[id];
// `fetch_data_operation` is called in the callback function of subscribe.
Expand All @@ -458,7 +457,7 @@ void ActorInfoAccessor::AsyncResubscribe() {
if (fetch_data_operation != nullptr) {
fetch_data_operation(nullptr);
}
}));
});
}
}

Expand Down Expand Up @@ -935,16 +934,6 @@ void NodeResourceInfoAccessor::AsyncGetDrainingNodes(
});
}

void NodeResourceInfoAccessor::AsyncResubscribe() {
Copy link
Contributor Author

Choose a reason for hiding this comment

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

This was dead code, we never set these callbacks ever since the subscribe was removed a couple years ago:
https://github.com/ray-project/ray/pull/24857/files

RAY_LOG(DEBUG) << "Reestablishing subscription for node resource info.";
if (subscribe_resource_operation_ != nullptr) {
RAY_CHECK_OK(subscribe_resource_operation_(nullptr));
}
if (subscribe_batch_resource_usage_operation_ != nullptr) {
RAY_CHECK_OK(subscribe_batch_resource_usage_operation_(nullptr));
}
}

void NodeResourceInfoAccessor::AsyncGetAllResourceUsage(
const ItemCallback<rpc::ResourceUsageBatchData> &callback) {
rpc::GetAllResourceUsageRequest request;
Expand Down Expand Up @@ -1009,14 +998,13 @@ void ErrorInfoAccessor::AsyncReportJobError(rpc::ErrorTableData data) {
WorkerInfoAccessor::WorkerInfoAccessor(GcsClient *client_impl)
: client_impl_(client_impl) {}

Status WorkerInfoAccessor::AsyncSubscribeToWorkerFailures(
void WorkerInfoAccessor::AsyncSubscribeToWorkerFailures(
const ItemCallback<rpc::WorkerDeltaData> &subscribe, const StatusCallback &done) {
RAY_CHECK(subscribe != nullptr);
subscribe_operation_ = [this, subscribe](const StatusCallback &done_callback) {
return client_impl_->GetGcsSubscriber().SubscribeAllWorkerFailures(subscribe,
done_callback);
client_impl_->GetGcsSubscriber().SubscribeAllWorkerFailures(subscribe, done_callback);
};
return subscribe_operation_(done);
subscribe_operation_(done);
}

void WorkerInfoAccessor::AsyncResubscribe() {
Expand All @@ -1025,7 +1013,7 @@ void WorkerInfoAccessor::AsyncResubscribe() {
RAY_LOG(DEBUG) << "Reestablishing subscription for worker failures.";
// The pub-sub server has restarted, we need to resubscribe to the pub-sub server.
if (subscribe_operation_ != nullptr) {
RAY_CHECK_OK(subscribe_operation_(nullptr));
subscribe_operation_(nullptr);
}
}

Expand Down
26 changes: 5 additions & 21 deletions src/ray/gcs_rpc_client/accessor.h
Original file line number Diff line number Diff line change
Expand Up @@ -38,7 +38,7 @@ namespace gcs {
// RAY_gcs_server_request_timeout_seconds
int64_t GetGcsTimeoutMs();

using SubscribeOperation = std::function<Status(const StatusCallback &done)>;
using SubscribeOperation = std::function<void(const StatusCallback &done)>;
using FetchDataOperation = std::function<void(const StatusCallback &done)>;

class GcsClient;
Expand Down Expand Up @@ -170,17 +170,15 @@ class ActorInfoAccessor {
/// \param actor_id The ID of actor to be subscribed to.
/// \param subscribe Callback that will be called each time when the actor is updated.
/// \param done Callback that will be called when subscription is complete.
/// \return Status
virtual Status AsyncSubscribe(
virtual void AsyncSubscribe(
const ActorID &actor_id,
const SubscribeCallback<ActorID, rpc::ActorTableData> &subscribe,
const StatusCallback &done);

/// Cancel subscription to an actor.
///
/// \param actor_id The ID of the actor to be unsubscribed to.
/// \return Status
virtual Status AsyncUnsubscribe(const ActorID &actor_id);
virtual void AsyncUnsubscribe(const ActorID &actor_id);

/// Reestablish subscription.
/// This should be called when GCS server restarts from a failure.
Expand Down Expand Up @@ -237,8 +235,7 @@ class JobInfoAccessor {
///
/// \param subscribe Callback that will be called each time when a job updates.
/// \param done Callback that will be called when subscription is complete.
/// \return Status
virtual Status AsyncSubscribeAll(
virtual void AsyncSubscribeAll(
const SubscribeCallback<JobID, rpc::JobTableData> &subscribe,
const StatusCallback &done);

Expand Down Expand Up @@ -511,13 +508,6 @@ class NodeResourceInfoAccessor {
virtual void AsyncGetDrainingNodes(
const ItemCallback<std::unordered_map<NodeID, int64_t>> &callback);

/// Reestablish subscription.
/// This should be called when GCS server restarts from a failure.
/// PubSub server restart will cause GCS server restart. In this case, we need to
/// resubscribe from PubSub server, otherwise we only need to fetch data from GCS
/// server.
virtual void AsyncResubscribe();

/// Get newest resource usage of all nodes from GCS asynchronously.
///
/// \param callback Callback that will be called after lookup finishes.
Expand All @@ -533,11 +523,6 @@ class NodeResourceInfoAccessor {
rpc::GetAllResourceUsageReply &reply);

private:
/// Save the subscribe operation in this function, so we can call it again when PubSub
/// server restarts from a failure.
SubscribeOperation subscribe_resource_operation_;
SubscribeOperation subscribe_batch_resource_usage_operation_;

GcsClient *client_impl_;

Sequencer<NodeID> sequencer_;
Expand Down Expand Up @@ -607,8 +592,7 @@ class WorkerInfoAccessor {
///
/// \param subscribe Callback that will be called each time when a worker failed.
/// \param done Callback that will be called when subscription is complete.
/// \return Status
virtual Status AsyncSubscribeToWorkerFailures(
virtual void AsyncSubscribeToWorkerFailures(
const ItemCallback<rpc::WorkerDeltaData> &subscribe, const StatusCallback &done);

/// Report a worker failure to GCS asynchronously.
Expand Down
Loading