[PlacementGroup]Fix bug that kill workers mistakenly when gcs restarts (#12568)

This commit is contained in:
fangfengbin 2020-12-03 17:50:48 +08:00 committed by GitHub
parent 7c58a85fed
commit ff34563539
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
3 changed files with 10 additions and 5 deletions

View file

@ -36,9 +36,6 @@ def test_gcs_server_restart(ray_start_regular):
ray.worker._global_node.kill_gcs_server()
ray.worker._global_node.start_gcs_server()
result = ray.get(actor1.method.remote(7))
assert result == 9
actor2 = Increase.remote()
result = ray.get(actor2.method.remote(2))
assert result == 4
@ -46,6 +43,12 @@ def test_gcs_server_restart(ray_start_regular):
result = ray.get(increase.remote(1))
assert result == 2
# Check whether actor1 is alive or not.
# NOTE: We can't execute it immediately after gcs restarts
# because it takes time for the worker to exit.
result = ray.get(actor1.method.remote(7))
assert result == 9
@pytest.mark.parametrize(
"ray_start_regular", [

View file

@ -1249,7 +1249,7 @@ def test_create_placement_group_during_gcs_server_restart(
cluster.head_node.kill_gcs_server()
cluster.head_node.start_gcs_server()
for i in range(0, 10):
for i in range(0, 100):
ray.get(placement_groups[i].ready())

View file

@ -574,7 +574,9 @@ void NodeManager::HandleReleaseUnusedBundles(
std::vector<std::shared_ptr<WorkerInterface>> workers_associated_with_unused_bundles;
for (const auto &worker_it : leased_workers_) {
auto &worker = worker_it.second;
if (0 == in_use_bundles.count(worker->GetBundleId())) {
const auto &bundle_id = worker->GetBundleId();
// We need to filter out the workers used by placement group.
if (!bundle_id.first.IsNil() && 0 == in_use_bundles.count(bundle_id)) {
workers_associated_with_unused_bundles.emplace_back(worker);
}
}