mirror of
https://github.com/vale981/ray
synced 2025-03-06 02:21:39 -05:00
[Placement Group] Fix resource index assignment between with bundle index and without bundle index pg (#17318)
This commit is contained in:
parent
e1cd8580a0
commit
2699b0f3ab
2 changed files with 50 additions and 1 deletions
|
@ -1803,5 +1803,50 @@ def test_actor_scheduling_not_block_with_placement_group(ray_start_cluster):
|
|||
is_actor_created_number_correct, timeout=30, retry_interval_ms=0)
|
||||
|
||||
|
||||
@pytest.mark.parametrize("connect_to_client", [False, True])
|
||||
def test_placement_group_gpu_unique_assigned(ray_start_cluster,
|
||||
connect_to_client):
|
||||
cluster = ray_start_cluster
|
||||
cluster.add_node(num_gpus=4, num_cpus=4)
|
||||
ray.init(address=cluster.address)
|
||||
gpu_ids_res = set()
|
||||
|
||||
# Create placement group with 4 bundles using 1 GPU each.
|
||||
num_gpus = 4
|
||||
bundles = [{"GPU": 1, "CPU": 1} for _ in range(num_gpus)]
|
||||
pg = placement_group(bundles)
|
||||
ray.get(pg.ready())
|
||||
|
||||
# Actor using 1 GPU that has a method to get
|
||||
# $CUDA_VISIBLE_DEVICES env variable.
|
||||
@ray.remote(num_gpus=1, num_cpus=1)
|
||||
class Actor:
|
||||
def get_gpu(self):
|
||||
import os
|
||||
return os.environ["CUDA_VISIBLE_DEVICES"]
|
||||
|
||||
# Create actors out of order.
|
||||
actors = []
|
||||
actors.append(
|
||||
Actor.options(placement_group=pg,
|
||||
placement_group_bundle_index=0).remote())
|
||||
actors.append(
|
||||
Actor.options(placement_group=pg,
|
||||
placement_group_bundle_index=3).remote())
|
||||
actors.append(
|
||||
Actor.options(placement_group=pg,
|
||||
placement_group_bundle_index=2).remote())
|
||||
actors.append(
|
||||
Actor.options(placement_group=pg,
|
||||
placement_group_bundle_index=1).remote())
|
||||
|
||||
for actor in actors:
|
||||
gpu_ids = ray.get(actor.get_gpu.remote())
|
||||
assert len(gpu_ids) == 1
|
||||
gpu_ids_res.add(gpu_ids)
|
||||
|
||||
assert len(gpu_ids_res) == 4
|
||||
|
||||
|
||||
if __name__ == "__main__":
|
||||
sys.exit(pytest.main(["-sv", __file__]))
|
||||
|
|
|
@ -499,7 +499,11 @@ def get_gpu_ids():
|
|||
assigned_ids = set()
|
||||
for resource, assignment in all_resource_ids.items():
|
||||
# Handle both normal and placement group GPU resources.
|
||||
if resource == "GPU" or resource.startswith("GPU_group_"):
|
||||
# Note: We should only get the GPU ids from the placement
|
||||
# group resource that does not contain the bundle index!
|
||||
import re
|
||||
if resource == "GPU" or re.match(r"^GPU_group_[0-9A-Za-z]+$",
|
||||
resource):
|
||||
for resource_id, _ in assignment:
|
||||
assigned_ids.add(resource_id)
|
||||
|
||||
|
|
Loading…
Add table
Reference in a new issue