Skip to content

Commit e793631

Browse files
authored
[core][autoscaler] Fix RAY_NODE_TYPE_NAME handling when autoscaler is in read-only mode (#58460)
This ensures node type names are correctly reported even when the autoscaler is disabled (read-only mode). ## Description Autoscaler v2 fails to report prometheus metrics when operating in read-only mode on KubeRay with the following KeyError error: ``` 2025-11-08 12:06:57,402 ERROR autoscaler.py:215 -- 'small-group' Traceback (most recent call last): File "/home/ray/anaconda3/lib/python3.9/site-packages/ray/autoscaler/v2/autoscaler.py", line 200, in update_autoscaling_state return Reconciler.reconcile( File "/home/ray/anaconda3/lib/python3.9/site-packages/ray/autoscaler/v2/instance_manager/reconciler.py", line 120, in reconcile Reconciler._step_next( File "/home/ray/anaconda3/lib/python3.9/site-packages/ray/autoscaler/v2/instance_manager/reconciler.py", line 275, in _step_next Reconciler._scale_cluster( File "/home/ray/anaconda3/lib/python3.9/site-packages/ray/autoscaler/v2/instance_manager/reconciler.py", line 1125, in _scale_cluster reply = scheduler.schedule(sched_request) File "/home/ray/anaconda3/lib/python3.9/site-packages/ray/autoscaler/v2/scheduler.py", line 933, in schedule ResourceDemandScheduler._enforce_max_workers_per_type(ctx) File "/home/ray/anaconda3/lib/python3.9/site-packages/ray/autoscaler/v2/scheduler.py", line 1006, in _enforce_max_workers_per_type node_config = ctx.get_node_type_configs()[node_type] KeyError: 'small-group' ``` This happens because the `ReadOnlyProviderConfigReader` populates `ctx.get_node_type_configs()` using node IDs as node types, which is correct for local Ray (where local ray does not have `RAY_NODE_TYPE_NAME` set), but incorrect for KubeRay where `ray_node_type_name` is present and expected with `RAY_NODE_TYPE_NAME` set. As a result, in read-only mode the scheduler sees a node type name (ex. small-group) that never exists in the populated configs. This PR fixes the issue by using `ray_node_type_name` when it exists, and only falling back to node ID when it does not. ## Related issues Fixes #58227 Signed-off-by: Rueian <[email protected]>
1 parent 654feda commit e793631

File tree

3 files changed

+72
-9
lines changed

3 files changed

+72
-9
lines changed

python/ray/autoscaler/_private/readonly/defaults.yaml

Lines changed: 0 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -13,7 +13,6 @@ available_node_types:
1313
max_workers: 0
1414
head_node_type: ray.head.default
1515
upscaling_speed: 1.0
16-
idle_timeout_minutes: 0
1716
#
1817
# !!! Configurations below are not supported in fake cluster mode !!!
1918
#

python/ray/autoscaler/v2/instance_manager/config.py

Lines changed: 14 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -522,16 +522,23 @@ def refresh_cached_autoscaling_config(self) -> AutoscalingConfig:
522522

523523
head_node_type = None
524524
for node_state in ray_cluster_resource_state.node_states:
525-
node_type = format_readonly_node_type(binary_to_hex(node_state.node_id))
525+
node_type = node_state.ray_node_type_name
526+
if not node_type:
527+
node_type = format_readonly_node_type(binary_to_hex(node_state.node_id))
528+
526529
if is_head_node(node_state):
527530
head_node_type = node_type
528531

529-
available_node_types[node_type] = {
530-
"resources": dict(node_state.total_resources),
531-
"min_workers": 0,
532-
"max_workers": 0 if is_head_node(node_state) else 1,
533-
"node_config": {},
534-
}
532+
if node_type not in available_node_types:
533+
available_node_types[node_type] = {
534+
"resources": dict(node_state.total_resources),
535+
"min_workers": 0,
536+
"max_workers": 0 if is_head_node(node_state) else 1,
537+
"node_config": {},
538+
}
539+
elif not is_head_node(node_state):
540+
available_node_types[node_type]["max_workers"] += 1
541+
535542
if available_node_types:
536543
self._configs["available_node_types"].update(available_node_types)
537544
self._configs["max_workers"] = len(available_node_types)

python/ray/autoscaler/v2/tests/test_config.py

Lines changed: 58 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -5,9 +5,16 @@
55

66
import pytest # noqa
77

8+
from ray._common.utils import binary_to_hex
89
from ray._private.test_utils import get_test_config_path
910
from ray.autoscaler import AUTOSCALER_DIR_PATH
10-
from ray.autoscaler.v2.instance_manager.config import FileConfigReader, Provider
11+
from ray.autoscaler._private.util import format_readonly_node_type
12+
from ray.autoscaler.v2.instance_manager import config as config_mod
13+
from ray.autoscaler.v2.instance_manager.config import (
14+
FileConfigReader,
15+
Provider,
16+
ReadOnlyProviderConfigReader,
17+
)
1118

1219

1320
@pytest.mark.parametrize(
@@ -179,6 +186,56 @@ def test_read_config():
179186
assert config_reader.get_cached_autoscaling_config().provider == Provider.GCP
180187

181188

189+
def test_readonly_node_type_name_and_fallback(monkeypatch):
190+
class _DummyNodeState:
191+
def __init__(self, ray_node_type_name, node_id, total_resources):
192+
self.ray_node_type_name = ray_node_type_name
193+
self.node_id = node_id
194+
self.total_resources = total_resources
195+
196+
class _DummyClusterState:
197+
def __init__(self, node_states):
198+
self.node_states = node_states
199+
200+
# Avoid real GCS usage.
201+
monkeypatch.setattr(config_mod, "GcsClient", lambda address: object())
202+
# Build a cluster with:
203+
# - 1 named head type
204+
# - 2 named worker types of the same type (aggregation check)
205+
# - 1 worker type without name (fallback to node_id-based type)
206+
unnamed_worker_id = b"\xab"
207+
fallback_name = format_readonly_node_type(binary_to_hex(unnamed_worker_id))
208+
nodes = [
209+
_DummyNodeState(
210+
"ray.head.default", b"\x01", {"CPU": 1, "node:__internal_head__": 1}
211+
),
212+
_DummyNodeState("worker.custom", b"\x02", {"CPU": 2}),
213+
_DummyNodeState("worker.custom", b"\x03", {"CPU": 2}),
214+
_DummyNodeState("", unnamed_worker_id, {"CPU": 3}),
215+
]
216+
monkeypatch.setattr(
217+
config_mod,
218+
"get_cluster_resource_state",
219+
lambda _gc: _DummyClusterState(nodes),
220+
)
221+
222+
reader = ReadOnlyProviderConfigReader("dummy:0")
223+
reader.refresh_cached_autoscaling_config()
224+
cfg = reader.get_cached_autoscaling_config()
225+
226+
node_types = cfg.get_config("available_node_types")
227+
# Head assertions
228+
assert "ray.head.default" in node_types
229+
assert node_types["ray.head.default"]["max_workers"] == 0
230+
assert cfg.get_head_node_type() == "ray.head.default"
231+
# Preferred name aggregation
232+
assert "worker.custom" in node_types
233+
assert node_types["worker.custom"]["max_workers"] == 2
234+
# Fallback for unnamed worker
235+
assert fallback_name in node_types
236+
assert node_types[fallback_name]["max_workers"] == 1
237+
238+
182239
if __name__ == "__main__":
183240
if os.environ.get("PARALLEL_CI"):
184241
sys.exit(pytest.main(["-n", "auto", "--boxed", "-vs", __file__]))

0 commit comments

Comments
 (0)