Skip to content

[Autoscaler] Fix v1 autoscaler TypeError when using bundle_label_selectors#59850

Merged
dayshah merged 1 commit intoray-project:masterfrom
dragongu:fix/autoscaler_v1_type_error
Jan 13, 2026
Merged

[Autoscaler] Fix v1 autoscaler TypeError when using bundle_label_selectors#59850
dayshah merged 1 commit intoray-project:masterfrom
dragongu:fix/autoscaler_v1_type_error

Conversation

@dragongu
Copy link
Contributor

@dragongu dragongu commented Jan 5, 2026

Fix autoscaler v1 TypeError when request_resources uses label selectors

Summary

Fix a TypeError in autoscaler v1 when request_resources() is called with bundle_label_selectors.

Problem

PR #54843 introduced bundle_label_selectors parameter to request_resources(). When this parameter is used, the function writes resources in the new format to GCS KV:

{"resources": {"CPU": 1}, "label_selector": {"region": "us-west1"}}

For autoscaler v2, this format is handled correctly via GCS RPC. However, autoscaler v1 reads from KV and extracts only the resources field for backward compatibility (commands.py:237). But when monitor.load_metrics.summary() is called, it invokes freq_of_dicts() which attempts to hash these resource dictionaries.

If the resource dictionary contains nested structures (e.g., when label selectors are present in v2 format), the default serializer fails with:

2026-01-05 09:59:27,832 ERROR monitor.py:595 -- Error in monitor loop
Traceback (most recent call last):
  File "/usr/local/lib/python3.10/dist-packages/ray/autoscaler/_private/monitor.py", line 593, in run
    self._run()
  File "/usr/local/lib/python3.10/dist-packages/ray/autoscaler/_private/monitor.py", line 380, in _run
    load_metrics_summary = self.load_metrics.summary()
  File "/usr/local/lib/python3.10/dist-packages/ray/autoscaler/_private/load_metrics.py", line 271, in summary
    summarized_resource_requests = freq_of_dicts(self.get_resource_requests())
  File "/usr/local/lib/python3.10/dist-packages/ray/autoscaler/_private/load_metrics.py", line 60, in freq_of_dicts
    freqs = Counter(serializer(d) for d in dicts)
  File "/usr/lib/python3.10/collections/__init__.py", line 577, in __init__
    self.update(iterable, **kwds)
  File "/usr/lib/python3.10/collections/__init__.py", line 670, in update
    _count_elements(self, iterable)
  File "/usr/local/lib/python3.10/dist-packages/ray/autoscaler/_private/load_metrics.py", line 60, in <genexpr>
    freqs = Counter(serializer(d) for d in dicts)
  File "/usr/local/lib/python3.10/dist-packages/ray/autoscaler/_private/load_metrics.py", line 55, in <lambda>
    serializer = lambda d: frozenset(d.items())  # noqa: E731
TypeError: unhashable type: 'dict' 

This happens because Python dicts are mutable and cannot be used as dictionary keys or set elements directly.

Reproduction

import os
os.environ["RAY_ENABLE_AUTOSCALER_V2"] = "0"

import ray
from ray.autoscaler._private.monitor import Monitor
from ray.autoscaler.sdk import request_resources

ray.init(num_cpus=1, ignore_reinit_error=True)

gcs_address = ray.get_runtime_context().gcs_address
monitor = Monitor(address=gcs_address, autoscaling_config=None)

# This writes new format to KV
request_resources(
    bundles=[{"CPU": 1}],
    bundle_label_selectors=[{"region": "us-west1"}]
)

monitor.update_resource_requests()
monitor.load_metrics.summary()  # TypeError here

Solution

Update commands.py to ensure autoscaler v1 compatibility by always extracting only the resources field when writing to KV store, regardless of format:

# For v2 autoscaler: use new format with label_selectors via GCS RPC
if is_autoscaler_v2():
    from ray.autoscaler.v2.sdk import request_cluster_resources
    gcs_address = internal_kv_get_gcs_client().address
    request_cluster_resources(gcs_address, to_request)
else:
    # For v1 autoscaler: write old format (ResourceDict) to KV
    # Extract resources field for backward compatibility
    to_request_v1 = [req["resources"] for req in to_request]
    _internal_kv_put(
        AUTOSCALER_RESOURCE_REQUEST_CHANNEL, json.dumps(to_request_v1), overwrite=True
    )

This ensures v1 autoscaler only sees simple ResourceDict format like {"CPU": 1} without nested structures, preventing the TypeError in freq_of_dicts().

…ctors

PR 54843 broke v1 autoscaler by writing new format with nested dicts to KV.
Fix by extracting 'resources' field for v1 (v2 uses GCS RPC with full format).

Signed-off-by: dragongu <andrewgu@vip.qq.com>
@dragongu dragongu requested a review from a team as a code owner January 5, 2026 12:53
Copy link
Contributor

@gemini-code-assist gemini-code-assist bot left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Code Review

This pull request effectively resolves a TypeError in the v1 autoscaler when request_resources is called with bundle_label_selectors. The issue stemmed from the v1 autoscaler's inability to process the newer resource request format. The implemented solution correctly introduces version-specific logic, using GCS RPC for v2 and writing a backward-compatible format to the KV store for v1. The changes are clear, logical, and well-contained. The accompanying test modifications are excellent, as they not only assert the correct data format for v1 but also verify that the operation which previously caused the crash now executes successfully. The code quality is high, and I have no further recommendations.

@ray-gardener ray-gardener bot added core Issues that should be addressed in Ray Core community-contribution Contributed by the community labels Jan 5, 2026
@dayshah
Copy link
Contributor

dayshah commented Jan 5, 2026

@rueian can you take look

Also @dragongu is there a reason you're still using autoscaler v1?

@dayshah dayshah added the go add ONLY when ready to merge, run all tests label Jan 6, 2026
@dragongu
Copy link
Contributor Author

dragongu commented Jan 6, 2026

@rueian can you take look

Also @dragongu is there a reason you're still using autoscaler v1?

@dayshah Historically we’ve been on autoscaler v1. We hit a few issues when moving to v2, so we’ll have to keep using v1 for now.

@edoakes
Copy link
Collaborator

edoakes commented Jan 6, 2026

@dragongu what are the issues you've run into? We'd like to fully migrate to v2 and deprecate/remove v1 soon, so definitely want to sort those out!

@dragongu
Copy link
Contributor Author

dragongu commented Jan 6, 2026

@dragongu what are the issues you've run into? We'd like to fully migrate to v2 and deprecate/remove v1 soon, so definitely want to sort those out!

@edoakes Yes, we’re planning to move to v2 soon. The main issue is that our KubeRay is on a very old version (0.5.6). We hit some problems during the upgrade and had to roll back in a hurry. I’ll dig into it more today, but the v1 fix is still pretty important since it’s blocking my Ray upgrade.

Copy link
Contributor

@rueian rueian left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM. Thanks @dragongu.

@dayshah dayshah merged commit 8d27704 into ray-project:master Jan 13, 2026
6 checks passed
rushikeshadhav pushed a commit to rushikeshadhav/ray that referenced this pull request Jan 14, 2026
jeffery4011 pushed a commit to jeffery4011/ray that referenced this pull request Jan 20, 2026
…ctors (ray-project#59850)

Signed-off-by: dragongu <andrewgu@vip.qq.com>
Signed-off-by: jeffery4011 <jefferyshen1015@gmail.com>
ryanaoleary pushed a commit to ryanaoleary/ray that referenced this pull request Feb 3, 2026
peterxcli pushed a commit to peterxcli/ray that referenced this pull request Feb 25, 2026
…ctors (ray-project#59850)

Signed-off-by: dragongu <andrewgu@vip.qq.com>
Signed-off-by: peterxcli <peterxcli@gmail.com>
peterxcli pushed a commit to peterxcli/ray that referenced this pull request Feb 25, 2026
…ctors (ray-project#59850)

Signed-off-by: dragongu <andrewgu@vip.qq.com>
Signed-off-by: peterxcli <peterxcli@gmail.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

community-contribution Contributed by the community core Issues that should be addressed in Ray Core go add ONLY when ready to merge, run all tests

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants