mirror of
https://github.com/vale981/ray
synced 2025-03-06 10:31:39 -05:00
[Placement Group]Fix actor scheduling with Placement Group bug. (#16006)
This commit is contained in:
parent
69d0e8e4fe
commit
ea0ee86063
2 changed files with 52 additions and 1 deletions
|
@ -1724,5 +1724,56 @@ def test_placement_group_client_option_serialization():
|
|||
dict_to_pg_to_dict(pg_dict_no_bundles)
|
||||
|
||||
|
||||
def test_actor_scheduling_not_block_with_placement_group(ray_start_cluster):
|
||||
"""Tests the scheduling of lots of actors will not be blocked
|
||||
when using placement groups.
|
||||
|
||||
For more detailed information please refer to:
|
||||
https://github.com/ray-project/ray/issues/15801.
|
||||
"""
|
||||
|
||||
cluster = ray_start_cluster
|
||||
cluster.add_node(num_cpus=1)
|
||||
ray.init(address=cluster.address)
|
||||
|
||||
@ray.remote
|
||||
class A:
|
||||
def ready(self):
|
||||
pass
|
||||
|
||||
actor_num = 1000
|
||||
pgs = [ray.util.placement_group([{"CPU": 1}]) for _ in range(actor_num)]
|
||||
actors = [A.options(placement_group=pg).remote() for pg in pgs]
|
||||
refs = [actor.ready.remote() for actor in actors]
|
||||
|
||||
expected_created_num = 1
|
||||
|
||||
def is_actor_created_number_correct():
|
||||
ready, not_ready = ray.wait(refs, num_returns=len(refs), timeout=1)
|
||||
return len(ready) == expected_created_num
|
||||
|
||||
def is_pg_created_number_correct():
|
||||
created_pgs = [
|
||||
pg for _, pg in ray.util.placement_group_table().items()
|
||||
if pg["state"] == "CREATED"
|
||||
]
|
||||
return len(created_pgs) == expected_created_num
|
||||
|
||||
wait_for_condition(is_pg_created_number_correct, timeout=3)
|
||||
wait_for_condition(
|
||||
is_actor_created_number_correct, timeout=30, retry_interval_ms=0)
|
||||
|
||||
# NOTE: we don't need to test all the actors create successfully.
|
||||
for _ in range(20):
|
||||
expected_created_num += 1
|
||||
cluster.add_node(num_cpus=1)
|
||||
|
||||
wait_for_condition(is_pg_created_number_correct, timeout=10)
|
||||
# Make sure the node add event will cause a waiting actor
|
||||
# to create successfully in time.
|
||||
wait_for_condition(
|
||||
is_actor_created_number_correct, timeout=30, retry_interval_ms=0)
|
||||
|
||||
|
||||
if __name__ == "__main__":
|
||||
sys.exit(pytest.main(["-sv", __file__]))
|
||||
|
|
|
@ -78,7 +78,7 @@ void TaskSpecification::ComputeResources() {
|
|||
// the actor tasks need not be scheduled.
|
||||
|
||||
// Map the scheduling class descriptor to an integer for performance.
|
||||
auto sched_cls = GetRequiredResources();
|
||||
auto sched_cls = GetRequiredPlacementResources();
|
||||
sched_cls_id_ = GetSchedulingClass(sched_cls);
|
||||
}
|
||||
}
|
||||
|
|
Loading…
Add table
Reference in a new issue