Skip to content

Commit

Permalink
Fix leased worker leak bug if lease worker requests that are still wa…
Browse files Browse the repository at this point in the history
…iting to be scheduled when GCS restarts (ray-project#9719)
  • Loading branch information
ffbin committed Jul 29, 2020
1 parent 2cafc7c commit a484947
Show file tree
Hide file tree
Showing 4 changed files with 49 additions and 10 deletions.
14 changes: 11 additions & 3 deletions python/ray/tests/test_gcs_fault_tolerance.py
Original file line number Diff line number Diff line change
Expand Up @@ -22,8 +22,12 @@ def increase(x):


@pytest.mark.skipif(
os.environ.get("RAY_GCS_ACTOR_SERVICE_ENABLED") != "true",
os.environ.get("RAY_GCS_ACTOR_SERVICE_ENABLED", "true") != "true",
reason=("This testcase can only be run when GCS actor management is on."))
@pytest.mark.parametrize(
"ray_start_regular",
[generate_internal_config_map(num_heartbeats_timeout=20)],
indirect=True)
def test_gcs_server_restart(ray_start_regular):
actor1 = Increase.remote()
result = ray.get(actor1.method.remote(1))
Expand All @@ -44,8 +48,12 @@ def test_gcs_server_restart(ray_start_regular):


@pytest.mark.skipif(
os.environ.get("RAY_GCS_ACTOR_SERVICE_ENABLED") != "true",
os.environ.get("RAY_GCS_ACTOR_SERVICE_ENABLED", "true") != "true",
reason=("This testcase can only be run when GCS actor management is on."))
@pytest.mark.parametrize(
"ray_start_regular",
[generate_internal_config_map(num_heartbeats_timeout=20)],
indirect=True)
def test_gcs_server_restart_during_actor_creation(ray_start_regular):
ids = []
for i in range(0, 100):
Expand All @@ -62,7 +70,7 @@ def test_gcs_server_restart_during_actor_creation(ray_start_regular):


@pytest.mark.skipif(
os.environ.get("RAY_GCS_ACTOR_SERVICE_ENABLED") != "true",
os.environ.get("RAY_GCS_ACTOR_SERVICE_ENABLED", "true") != "true",
reason=("This testcase can only be run when GCS actor management is on."))
@pytest.mark.parametrize(
"ray_start_cluster_head",
Expand Down
7 changes: 6 additions & 1 deletion src/ray/core_worker/reference_count.cc
Original file line number Diff line number Diff line change
Expand Up @@ -493,7 +493,12 @@ bool ReferenceCounter::SetDeleteCallback(
return false;
}

RAY_CHECK(!it->second.on_delete) << object_id;
// NOTE: In two cases, `GcsActorManager` will send `WaitForActorOutOfScope` request more
// than once, causing the delete callback to be set repeatedly.
// 1.If actors have not been registered successfully before GCS restarts, gcs client
// will resend the registration request after GCS restarts.
// 2.After GCS restarts, GCS will send `WaitForActorOutOfScope` request to owned actors
// again.
it->second.on_delete = callback;
return true;
}
Expand Down
24 changes: 21 additions & 3 deletions src/ray/gcs/gcs_server/gcs_actor_manager.cc
Original file line number Diff line number Diff line change
Expand Up @@ -931,8 +931,22 @@ void GcsActorManager::LoadInitialData(const EmptyCallback &done) {
named_actors_.emplace(actor->GetName(), actor->GetActorID());
}

created_actors_[actor->GetNodeID()].emplace(actor->GetWorkerID(),
actor->GetActorID());
if (item.second.state() == ray::rpc::ActorTableData::DEPENDENCIES_UNREADY) {
const auto &owner = actor->GetOwnerAddress();
const auto &owner_node = ClientID::FromBinary(owner.raylet_id());
const auto &owner_worker = WorkerID::FromBinary(owner.worker_id());
RAY_CHECK(unresolved_actors_[owner_node][owner_worker]
.emplace(actor->GetActorID())
.second);
if (!actor->IsDetached() && worker_client_factory_) {
// This actor is owned. Send a long polling request to the actor's
// owner to determine when the actor should be removed.
PollOwnerForActorOutOfScope(actor);
}
} else if (item.second.state() == ray::rpc::ActorTableData::ALIVE) {
created_actors_[actor->GetNodeID()].emplace(actor->GetWorkerID(),
actor->GetActorID());
}

auto &workers = owners_[actor->GetNodeID()];
auto it = workers.find(actor->GetWorkerID());
Expand All @@ -958,7 +972,11 @@ void GcsActorManager::LoadInitialData(const EmptyCallback &done) {
<< ", and the number of created actors is " << created_actors_.size();
for (auto &item : registered_actors_) {
auto &actor = item.second;
if (actor->GetState() != ray::rpc::ActorTableData::ALIVE) {
if (actor->GetState() == ray::rpc::ActorTableData::PENDING_CREATION ||
actor->GetState() == ray::rpc::ActorTableData::RESTARTING) {
// We should not reschedule actors in state of `ALIVE`.
// We could not reschedule actors in state of `DEPENDENCIES_UNREADY` because the
// dependencies of them may not have been resolved yet.
RAY_LOG(DEBUG) << "Rescheduling a non-alive actor, actor id = "
<< actor->GetActorID() << ", state = " << actor->GetState();
gcs_actor_scheduler_->Reschedule(actor);
Expand Down
14 changes: 11 additions & 3 deletions src/ray/raylet/node_manager.cc
Original file line number Diff line number Diff line change
Expand Up @@ -1763,7 +1763,17 @@ void NodeManager::HandleRequestWorkerLease(const rpc::RequestWorkerLeaseRequest
rid->set_quantity(id.second.ToDouble());
}
}
send_reply_callback(Status::OK(), nullptr, nullptr);

auto reply_failure_handler = [this, worker_id]() {
if (RayConfig::instance().gcs_actor_service_enabled()) {
RAY_LOG(WARNING)
<< "Failed to reply to GCS server, because it might have restarted. GCS "
"cannot obtain the information of the leased worker, so we need to "
"release the leased worker to avoid leakage.";
leased_workers_.erase(worker_id);
}
};
send_reply_callback(Status::OK(), nullptr, reply_failure_handler);
RAY_CHECK(leased_workers_.find(worker_id) == leased_workers_.end())
<< "Worker is already leased out " << worker_id;

Expand Down Expand Up @@ -1855,8 +1865,6 @@ void NodeManager::HandleReturnWorker(const rpc::ReturnWorkerRequest &request,
void NodeManager::HandleReleaseUnusedWorkers(
const rpc::ReleaseUnusedWorkersRequest &request,
rpc::ReleaseUnusedWorkersReply *reply, rpc::SendReplyCallback send_reply_callback) {
// TODO(ffbin): At present, we have not cleaned up the lease worker requests that are
// still waiting to be scheduled, which will be implemented in the next pr.
std::unordered_set<WorkerID> in_use_worker_ids;
for (int index = 0; index < request.worker_ids_in_use_size(); ++index) {
auto worker_id = WorkerID::FromBinary(request.worker_ids_in_use(index));
Expand Down

0 comments on commit a484947

Please sign in to comment.