Skip to content

Commit

Permalink
[Autoscaler] Allow GPU upscaling for non-GPU workloads, with lowest p…
Browse files Browse the repository at this point in the history
…riority (ray-project#31202)

Closes ray-project#20476:
Instead of preventing GPU upscaling due to non-GPU tasks, prefer non-GPU nodes by assigning low utilization score to the GPU nodes.

Signed-off-by: Dmitri Gekhtman <[email protected]>
Signed-off-by: tmynn <[email protected]>
  • Loading branch information
DmitriGekhtman authored and tamohannes committed Jan 25, 2023
1 parent 9120b06 commit b036a02
Show file tree
Hide file tree
Showing 3 changed files with 80 additions and 41 deletions.
28 changes: 15 additions & 13 deletions python/ray/autoscaler/_private/resource_demand_scheduler.py
Original file line number Diff line number Diff line change
Expand Up @@ -806,17 +806,8 @@ def _resource_based_utilization_scorer(
resources: List[ResourceDict],
*,
node_availability_summary: NodeAvailabilitySummary,
) -> Optional[Tuple[float, float]]:
) -> Optional[Tuple[bool, int, float, float]]:
remaining = copy.deepcopy(node_resources)
is_gpu_node = "GPU" in node_resources and node_resources["GPU"] > 0
any_gpu_task = any("GPU" in r for r in resources)

# Avoid launching GPU nodes if there aren't any GPU tasks at all. Note that
# if there *is* a GPU task, then CPU tasks can be scheduled as well.
if AUTOSCALER_CONSERVE_GPU_NODES:
if is_gpu_node and not any_gpu_task:
return None

fittable = []
resource_types = set()
for r in resources:
Expand Down Expand Up @@ -846,10 +837,21 @@ def _resource_based_utilization_scorer(
if not util_by_resources:
return None

# Prioritize matching multiple resource types first, then prioritize
# using all resources, then prioritize overall balance
# of multiple resources.
# Prefer not to launch a GPU node if there aren't any GPU requirements in the
# resource bundle.
gpu_ok = True
if AUTOSCALER_CONSERVE_GPU_NODES:
is_gpu_node = "GPU" in node_resources and node_resources["GPU"] > 0
any_gpu_task = any("GPU" in r for r in resources)
if is_gpu_node and not any_gpu_task:
gpu_ok = False

# Prioritize avoiding gpu nodes for non-gpu workloads first,
# then prioritize matching multiple resource types,
# then prioritize using all resources,
# then prioritize overall balance of multiple resources.
return (
gpu_ok,
num_matching_resource_types,
min(util_by_resources),
np.mean(util_by_resources),
Expand Down
36 changes: 36 additions & 0 deletions python/ray/tests/test_autoscaler_fake_multinode.py
Original file line number Diff line number Diff line change
Expand Up @@ -86,6 +86,42 @@ def ping(self):
cluster.shutdown()


def test_autoscaler_cpu_task_gpu_node_up():
"""Validates that CPU tasks can trigger GPU upscaling.
See https://github.com/ray-project/ray/pull/31202.
"""
cluster = AutoscalingCluster(
head_resources={"CPU": 0},
worker_node_types={
"gpu_node_type": {
"resources": {
"CPU": 1,
"GPU": 1,
},
"node_config": {},
"min_workers": 0,
"max_workers": 1,
},
},
)

try:
cluster.start()
ray.init("auto")

@ray.remote(num_cpus=1)
def task():
return True

# Make sure the task can be scheduled.
# Since the head has 0 CPUs, this requires upscaling a GPU worker.
ray.get(task.remote(), timeout=30)
ray.shutdown()

finally:
cluster.shutdown()


if __name__ == "__main__":
import os
import sys
Expand Down
57 changes: 29 additions & 28 deletions python/ray/tests/test_resource_demand_scheduler.py
Original file line number Diff line number Diff line change
Expand Up @@ -93,76 +93,67 @@ def test_util_score():
)
assert _resource_based_utilization_scorer(
{"GPU": 4}, [{"GPU": 2}], node_availability_summary=EMPTY_AVAILABILITY_SUMMARY
) == (1, 0.5, 0.5)
) == (True, 1, 0.5, 0.5)
assert _resource_based_utilization_scorer(
{"GPU": 4},
[{"GPU": 1}, {"GPU": 1}],
node_availability_summary=EMPTY_AVAILABILITY_SUMMARY,
) == (1, 0.5, 0.5)
) == (True, 1, 0.5, 0.5)
assert _resource_based_utilization_scorer(
{"GPU": 2}, [{"GPU": 2}], node_availability_summary=EMPTY_AVAILABILITY_SUMMARY
) == (1, 2, 2)
) == (True, 1, 2, 2)
assert _resource_based_utilization_scorer(
{"GPU": 2},
[{"GPU": 1}, {"GPU": 1}],
node_availability_summary=EMPTY_AVAILABILITY_SUMMARY,
) == (1, 2, 2)
) == (True, 1, 2, 2)
assert _resource_based_utilization_scorer(
{"GPU": 1},
[{"GPU": 1, "CPU": 1}, {"GPU": 1}],
node_availability_summary=EMPTY_AVAILABILITY_SUMMARY,
) == (
1,
1,
1,
)
) == (True, 1, 1, 1)
assert _resource_based_utilization_scorer(
{"GPU": 1, "CPU": 1},
[{"GPU": 1, "CPU": 1}, {"GPU": 1}],
node_availability_summary=EMPTY_AVAILABILITY_SUMMARY,
) == (2, 1, 1)
) == (True, 2, 1, 1)
assert _resource_based_utilization_scorer(
{"GPU": 2, "TPU": 1},
[{"GPU": 2}],
node_availability_summary=EMPTY_AVAILABILITY_SUMMARY,
) == (1, 0, 1)
) == (True, 1, 0, 1)
assert _resource_based_utilization_scorer(
{"CPU": 64}, [{"CPU": 64}], node_availability_summary=EMPTY_AVAILABILITY_SUMMARY
) == (1, 64, 64)
) == (True, 1, 64, 64)
assert _resource_based_utilization_scorer(
{"CPU": 64}, [{"CPU": 32}], node_availability_summary=EMPTY_AVAILABILITY_SUMMARY
) == (1, 8, 8)
) == (True, 1, 8, 8)
assert _resource_based_utilization_scorer(
{"CPU": 64},
[{"CPU": 16}, {"CPU": 16}],
node_availability_summary=EMPTY_AVAILABILITY_SUMMARY,
) == (1, 8, 8)
) == (True, 1, 8, 8)


def test_gpu_node_util_score():
# Avoid scheduling CPU tasks on GPU node.
assert (
_resource_based_utilization_scorer(
{"GPU": 1, "CPU": 1},
[{"CPU": 1}],
node_availability_summary=EMPTY_AVAILABILITY_SUMMARY,
)
is None
utilization_score = _resource_based_utilization_scorer(
{"GPU": 1, "CPU": 1},
[{"CPU": 1}],
node_availability_summary=EMPTY_AVAILABILITY_SUMMARY,
)
gpu_ok = utilization_score[0]
assert gpu_ok is False
assert _resource_based_utilization_scorer(
{"GPU": 1, "CPU": 1},
[{"CPU": 1, "GPU": 1}],
node_availability_summary=EMPTY_AVAILABILITY_SUMMARY,
) == (
2,
1.0,
1.0,
)
) == (True, 2, 1.0, 1.0)
assert _resource_based_utilization_scorer(
{"GPU": 1, "CPU": 1},
[{"GPU": 1}],
node_availability_summary=EMPTY_AVAILABILITY_SUMMARY,
) == (1, 0.0, 0.5)
) == (True, 1, 0.0, 0.5)


def test_zero_resource():
Expand Down Expand Up @@ -322,13 +313,23 @@ def test_gpu_node_avoid_cpu_task():
},
}
r1 = [{"CPU": 1}] * 100
# max_to_add ten nodes allowed. All chosen to be "cpu".
assert get_nodes_for(
types,
{},
"empty_node",
100,
10,
r1,
) == {"cpu": 10}
# max_to_add eleven nodes allowed. First ten chosen to be "cpu",
# last chosen to be "gpu" due max_workers constraint on "cpu".
assert get_nodes_for(
types,
{},
"empty_node",
11,
r1,
) == {"cpu": 10, "gpu": 1}
r2 = [{"GPU": 1}] + [{"CPU": 1}] * 100
assert get_nodes_for(
types,
Expand Down

0 comments on commit b036a02

Please sign in to comment.