mirror of
https://github.com/vale981/ray
synced 2025-03-06 10:31:39 -05:00
[placement groups] fix gpu ids for bundles (#14574)
This commit is contained in:
parent
2505bc8aa9
commit
ea7d4c6607
2 changed files with 40 additions and 2 deletions
|
@ -1546,5 +1546,41 @@ def test_placement_group_synchronous_registration(ray_start_cluster):
|
|||
wait_for_condition(is_placement_group_removed)
|
||||
|
||||
|
||||
def test_placement_group_gpu_set(ray_start_cluster):
|
||||
cluster = ray_start_cluster
|
||||
# One node which only has one CPU.
|
||||
cluster.add_node(num_cpus=1, num_gpus=1)
|
||||
cluster.add_node(num_cpus=1, num_gpus=1)
|
||||
cluster.wait_for_nodes()
|
||||
ray.init(address=cluster.address)
|
||||
|
||||
placement_group = ray.util.placement_group(
|
||||
name="name",
|
||||
strategy="PACK",
|
||||
bundles=[{
|
||||
"CPU": 1,
|
||||
"GPU": 1
|
||||
}, {
|
||||
"CPU": 1,
|
||||
"GPU": 1
|
||||
}])
|
||||
|
||||
@ray.remote(num_gpus=1)
|
||||
def get_gpus():
|
||||
return ray.get_gpu_ids()
|
||||
|
||||
result = get_gpus.options(
|
||||
placement_group=placement_group,
|
||||
placement_group_bundle_index=0).remote()
|
||||
result = ray.get(result)
|
||||
assert result == [0]
|
||||
|
||||
result = get_gpus.options(
|
||||
placement_group=placement_group,
|
||||
placement_group_bundle_index=1).remote()
|
||||
result = ray.get(result)
|
||||
assert result == [0]
|
||||
|
||||
|
||||
if __name__ == "__main__":
|
||||
sys.exit(pytest.main(["-v", __file__]))
|
||||
|
|
|
@ -424,12 +424,14 @@ def get_gpu_ids():
|
|||
|
||||
# TODO(ilr) Handle inserting resources in local mode
|
||||
all_resource_ids = global_worker.core_worker.resource_ids()
|
||||
assigned_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_"):
|
||||
for resource_id, _ in assignment:
|
||||
assigned_ids.append(resource_id)
|
||||
assigned_ids.add(resource_id)
|
||||
|
||||
assigned_ids = list(assigned_ids)
|
||||
# If the user had already set CUDA_VISIBLE_DEVICES, then respect that (in
|
||||
# the sense that only GPU IDs that appear in CUDA_VISIBLE_DEVICES should be
|
||||
# returned).
|
||||
|
|
Loading…
Add table
Reference in a new issue