Skip to content

[Autoscaler][V2] Consider bundle_label_selector in Ray V2 Autoscaler#56826

Merged
edoakes merged 11 commits intoray-project:masterfrom
ryanaoleary:bundle-label-selector-e2e-test
Oct 7, 2025
Merged

[Autoscaler][V2] Consider bundle_label_selector in Ray V2 Autoscaler#56826
edoakes merged 11 commits intoray-project:masterfrom
ryanaoleary:bundle-label-selector-e2e-test

Conversation

@ryanaoleary
Copy link
Copy Markdown
Contributor

@ryanaoleary ryanaoleary commented Sep 23, 2025

Why are these changes needed?

This PR adds support to parse the GangResourceRequest.bundle_selectors.resource_requests field for gang resource requests in the V2 Autoscaler. This proto field replaces the deprecated GangResourceRequest.resource_requests (definition) in order to support repeated selectors for fallback strategy. This change is required for autoscaling to work with the bundle_label_selector placement group option.

This PR also adds an e2e test case for scaling up a placement group with bundle_label_selector specified. This tests verifies the behavior that the v2 scheduler will scale nodes satisfying the given label constraints, preferring nodes with the required labels over node types with sufficient resources, but lacking those labels.

Related issue number

Contributes to #51564

Checks

  • I've signed off every commit(by using the -s flag, i.e., git commit -s) in this PR.
  • I've run scripts/format.sh to lint the changes in this PR.
  • I've included any doc changes needed for https://docs.ray.io/en/master/.
    • I've added any new APIs to the API Reference. For example, if I added a
      method in Tune, I've added it in doc/source/tune/api/ under the
      corresponding .rst file.
  • I've made sure the tests are passing. Note that there might be a few flaky tests, see the recent failures at https://flakey-tests.ray.io/
  • Testing Strategy
    • Unit tests
    • Release tests
    • This PR is not tested :(

Note

Adds and validates e2e scheduling tests for label-based placement of tasks, actors, and placement groups (bundle_label_selector), including a helper for selector matching.

  • Tests (Autoscaler v2):
    • Label selector scheduling:
      • Add test_task_scheduled_on_node_with_label_selector: submits tasks with different label_selectors via .options(...), waits for autoscaled nodes, and verifies each task runs on a node whose Labels satisfy the selector.
      • Add test_actor_scheduled_on_node_with_label_selector: submits actors with label_selectors, verifies expected node types scale up and that each actor is placed on a matching-labeled node.
      • Add test_pg_scheduled_on_node_with_bundle_label_selector: creates a placement group with per-bundle bundle_label_selector, ensures correct node types scale and validates bundle-to-node label matching via placement_group_table.
    • Utilities: introduce _verify_node_labels_for_selector(...) to assert node Labels satisfy in(...), !in(...), equality, and negation forms.
    • Imports: include list_actors and placement_group_table for new validations.

Written by Cursor Bugbot for commit 48cbf73. This will update automatically on new commits. Configure here.

Signed-off-by: Ryan O'Leary <ryanaoleary@google.com>
@ryanaoleary ryanaoleary requested a review from a team as a code owner September 23, 2025 04:38
@gemini-code-assist
Copy link
Copy Markdown
Contributor

Warning

You have reached your daily quota limit. Please wait up to 24 hours and I will start processing your requests again!

@ryanaoleary
Copy link
Copy Markdown
Contributor Author

cc: @MengjinYan as discussed offline

@ray-gardener ray-gardener bot added core Issues that should be addressed in Ray Core community-contribution Contributed by the community labels Sep 23, 2025
Copy link
Copy Markdown
Contributor

@MengjinYan MengjinYan left a comment

Choose a reason for hiding this comment

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

Thanks for the followup!!

@edoakes edoakes added go add ONLY when ready to merge, run all tests and removed community-contribution Contributed by the community labels Sep 26, 2025
Signed-off-by: Ryan O'Leary <ryanaoleary@google.com>
Copy link
Copy Markdown
Contributor

@MengjinYan MengjinYan left a comment

Choose a reason for hiding this comment

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

Just minor and nit comments.

return len(status.active_nodes) == expected_nodes

wait_for_condition(all_nodes_launched, timeout=30)
proc.wait(timeout=30)
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

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

Curious why we need to wait the tasks to be done?

Just to want to make sure there is no unnecessary wait in the test cases.

Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

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

I think it's necessary or you can run into a race condition where the tasks/nodes have "launched' but the task["node_id"] is still None. When I removed the proc.wait this test became flaky with this issue, but with the wait it passes consistently.

@ryanaoleary
Copy link
Copy Markdown
Contributor Author

From testing with this PR I realize now that label_selectors are not included in the placement_group_table_data. Using this function I'm able to get the bundles, but when they're generated here we don't add the label selectors anywhere. Using list_placement_groups() to get the PlacementGroupState works though and includes the label_selector for each bundle.

Signed-off-by: Ryan O'Leary <ryanaoleary@google.com>
@MengjinYan
Copy link
Copy Markdown
Contributor

From testing with this PR I realize now that label_selectors are not included in the placement_group_table_data. Using this function I'm able to get the bundles, but when they're generated here we don't add the label selectors anywhere. Using list_placement_groups() to get the PlacementGroupState works though and includes the label_selector for each bundle.

We probably should add it also to the placement_group_table_data but probably in a followup PR.

Copy link
Copy Markdown
Contributor

@MengjinYan MengjinYan left a comment

Choose a reason for hiding this comment

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

Thanks!

cursor[bot]

This comment was marked as outdated.

Signed-off-by: Ryan O'Leary <ryanaoleary@google.com>
@ryanaoleary ryanaoleary changed the title [Autoscaler][V2] Add e2e test for bundle_label_selector [Autoscaler][V2] Consider bundle_label_selector in Ray V2 Autoscaler Oct 7, 2025
cursor[bot]

This comment was marked as outdated.

Signed-off-by: Ryan O'Leary <ryanaoleary@google.com>
@ryanaoleary
Copy link
Copy Markdown
Contributor Author

Manual testing process:

RayCluster CR:

apiVersion: ray.io/v1
kind: RayCluster
metadata:
 name: raycluster-label-based-scheduling
spec:
 enableInTreeAutoscaling: true
 autoscalerOptions:
   version: v2
   upscalingMode: Default
   idleTimeoutSeconds: 600
   imagePullPolicy: Always
   securityContext: {}
   env: []
   envFrom: []
   resources:
     limits:
       cpu: "500m"
       memory: "512Mi"
     requests:
       cpu: "500m"
       memory: "512Mi"
 headGroupSpec:
   rayStartParams:
     num-cpus: "0"
   template:
     spec:
       containers:
       - name: ray-head
         image: us-central1-docker.pkg.dev/tpu-vm-gke-testing/ryanaoleary-ray/ray:latest
         ports:
         - containerPort: 6379
           name: gcs
         - containerPort: 8265
           name: dashboard
         - containerPort: 10001
           name: client
         resources:
           limits:
             cpu: "1"
             memory: "2G"
           requests:
             cpu: "1"
             memory: "2G"
         volumeMounts:
         - mountPath: /home/ray/samples
           name: ray-example-configmap
       volumes:
       - name: ray-example-configmap
         configMap:
           name: ray-example
           defaultMode: 0777
           items:
           - key: test_task.py
             path: test_task.py
           - key: test_actor.py
             path: test_actor.py
           - key: test_placement_group.py
             path: test_placement_group.py
 workerGroupSpecs:
 - replicas: 0
   minReplicas: 0
   maxReplicas: 10
   groupName: group-1
   rayStartParams:
    labels: "test-label=test-value-1"
   template:
     spec:
       containers:
       - name: ray-worker
         image: us-central1-docker.pkg.dev/tpu-vm-gke-testing/ryanaoleary-ray/ray:latest
         resources:
           limits:
             cpu: "1"
             memory: "1G"
           requests:
             cpu: "1"
             memory: "1G"
 - replicas: 0
   minReplicas: 0
   maxReplicas: 10
   groupName: group-2
   rayStartParams:
    labels: "test-label=test-value-2"
   template:
     spec:
       containers:
       - name: ray-worker
         image: us-central1-docker.pkg.dev/tpu-vm-gke-testing/ryanaoleary-ray/ray:latest
         resources:
           limits:
             cpu: "1"
             memory: "1G"
           requests:
             cpu: "1"
             memory: "1G"
 - replicas: 0
   minReplicas: 0
   maxReplicas: 10
   groupName: group-3
   rayStartParams:
    labels: "test-label=test-value-3"
   template:
     spec:
       containers:
       - name: ray-worker
         image: us-central1-docker.pkg.dev/tpu-vm-gke-testing/ryanaoleary-ray/ray:latest
         resources:
           limits:
             cpu: "1"
             memory: "1G"
           requests:
             cpu: "1"
             memory: "1G"           
---
apiVersion: v1
kind: ConfigMap
metadata:
 name: ray-example
data:
 test_task.py: |
  import ray

  @ray.remote(num_cpus=1, label_selector={"test-label": "test-value-1"})
  def test_task():
    pass

  ray.init()
  ray.get(test_task.remote())

 test_actor.py: |
  import ray

  @ray.remote(num_cpus=1, label_selector={"test-label": "test-value-2"})
  class Actor:
  def ready(self):
    return True

  ray.init()
  my_actor = Actor.remote()
  ray.get(my_actor.ready.remote())

 test_placement_group.py: |
  import ray
  from ray.util.placement_group import placement_group
  
  ray.init()
  pg = placement_group([{"CPU": 1}] * 2, bundle_label_selector=[{"test-label": "test-value-3"}] * 2, strategy="SPREAD")
  ray.get(pg.ready())

Setup:

export HEAD_POD=$(kubectl get pods --selector=ray.io/node-type=head -o custom-columns=POD:[metadata.name](http://metadata.name/) --no-headers)
export RAYCLUSTER_NAME=raycluster-label-based-scheduling
export HEAD_SVC=$(kubectl get svc -l ray.io/cluster=$RAYCLUSTER_NAME,ray.io/node-type=head -o jsonpath='{.items[0].metadata.name}')

kubectl port-forward svc/$HEAD_SVC 8265:8265 2>&1 >/dev/null &

ray list nodes --address http://localhost:8265

2025-10-06 22:53:54,553 - INFO - Note: NumExpr detected 64 cores but "NUMEXPR_MAX_THREADS" not set, so enforcing safe limit of 8.
2025-10-06 22:53:54,553 - INFO - NumExpr defaulting to 8 threads.

======== List: 2025-10-06 22:53:55.232435 ========
Stats:
------------------------------
Total: 1

Table:
------------------------------
    NODE_ID                                                   NODE_IP      IS_HEAD_NODE    STATE    STATE_MESSAGE    NODE_NAME    RESOURCES_TOTAL                   LABELS
 0  0a7fc01c6b6c28737ca2c86bc6346d85560e74bcb50621b2ca6eb746  10.192.5.26  True            ALIVE                     10.192.5.26  memory: 1.863 GiB                 ray.io/node-group: headgroup
                                                                                                                                  node:10.192.5.26: 1.0             ray.io/node-id: 0a7fc01c6b6c28737ca2c86bc6346d85560e74bcb50621b2ca6eb746
                                                                                                                                  node:__internal_head__: 1.0
                                                                                                                                  object_store_memory: 318.635 MiB

Task test:

kubectl exec -it $HEAD_POD -- python3 /home/ray/samples/test_task.py

Defaulted container "ray-head" out of: ray-head, autoscaler
2025-10-06 15:54:21,329 INFO worker.py:1692 -- Using address 127.0.0.1:6379 set in the environment variable RAY_ADDRESS
2025-10-06 15:54:21,332 INFO worker.py:1833 -- Connecting to existing Ray cluster at address: 10.192.5.26:6379...
2025-10-06 15:54:21,343 INFO worker.py:2004 -- Connected to Ray cluster. View the dashboard at 10.192.5.26:8265 
/home/ray/anaconda3/lib/python3.9/site-packages/ray/_private/worker.py:2052: FutureWarning: Tip: In future versions of Ray, Ray will no longer override accelerator visible devices env var if num_gpus=0 or num_gpus=None (default). To enable this behavior and turn off this error message, set RAY_ACCEL_ENV_VAR_OVERRIDE_ON_ZERO=0
  warnings.warn(
(autoscaler +4s) Tip: use `ray status` to view detailed cluster status. To disable these messages, set RAY_SCHEDULER_EVENTS=0.
(autoscaler +4s) Adding 1 node(s) of type group-1.
(autoscaler +4s) Resized to 1 CPUs.

ray list nodes --address http://localhost:8265
2025-10-06 22:55:08,637 - INFO - Note: NumExpr detected 64 cores but "NUMEXPR_MAX_THREADS" not set, so enforcing safe limit of 8.
2025-10-06 22:55:08,637 - INFO - NumExpr defaulting to 8 threads.

======== List: 2025-10-06 22:55:09.296522 ========
Stats:
------------------------------
Total: 2

Table:
------------------------------
    NODE_ID                                                   NODE_IP      IS_HEAD_NODE    STATE    STATE_MESSAGE    NODE_NAME    RESOURCES_TOTAL                   LABELS
 0  0a7fc01c6b6c28737ca2c86bc6346d85560e74bcb50621b2ca6eb746  10.192.5.26  True            ALIVE                     10.192.5.26  memory: 1.863 GiB                 ray.io/node-group: headgroup
                                                                                                                                  node:10.192.5.26: 1.0             ray.io/node-id: 0a7fc01c6b6c28737ca2c86bc6346d85560e74bcb50621b2ca6eb746
                                                                                                                                  node:__internal_head__: 1.0
                                                                                                                                  object_store_memory: 318.635 MiB
 1  69d98912d3b6e39208fcdc7223ccf3148ee7da662c1feee9660e0ca0  10.192.4.31  False           ALIVE                     10.192.4.31  CPU: 1.0                          ray.io/node-group: group-1
                                                                                                                                  memory: 953.674 MiB               ray.io/node-id: 69d98912d3b6e39208fcdc7223ccf3148ee7da662c1feee9660e0ca0
                                                                                                                                  node:10.192.4.31: 1.0             test-label: test-value-1
                                                                                                                                  object_store_memory: 252.925 MiB

Actor test:

kubectl exec -it $HEAD_POD -- python3 /home/ray/samples/test_actor.py

kubectl exec -it $HEAD_POD -- python3 /home/ray/samples/test_actor.py
Defaulted container "ray-head" out of: ray-head, autoscaler
2025-10-06 17:28:28,205 INFO worker.py:1692 -- Using address 127.0.0.1:6379 set in the environment variable RAY_ADDRESS
2025-10-06 17:28:28,209 INFO worker.py:1833 -- Connecting to existing Ray cluster at address: 10.192.1.37:6379...
2025-10-06 17:28:28,229 INFO worker.py:2004 -- Connected to Ray cluster. View the dashboard at 10.192.1.37:8265 
/home/ray/anaconda3/lib/python3.9/site-packages/ray/_private/worker.py:2052: FutureWarning: Tip: In future versions of Ray, Ray will no longer override accelerator visible devices env var if num_gpus=0 or num_gpus=None (default). To enable this behavior and turn off this error message, set RAY_ACCEL_ENV_VAR_OVERRIDE_ON_ZERO=0
  warnings.warn(
(autoscaler +4s) Tip: use `ray status` to view detailed cluster status. To disable these messages, set RAY_SCHEDULER_EVENTS=0.
(autoscaler +4s) Adding 1 node(s) of type group-2.
(autoscaler +4s) Resized to 1 CPUs.

Placement Group Test:

kubectl exec -it $HEAD_POD -- python3 /home/ray/samples/test_placement_group.py

kubectl exec -it $HEAD_POD -- python3 /home/ray/samples/test_placement_group.py
Defaulted container "ray-head" out of: ray-head, autoscaler
2025-10-06 19:04:05,320 INFO worker.py:1692 -- Using address 127.0.0.1:6379 set in the environment variable RAY_ADDRESS
2025-10-06 19:04:05,325 INFO worker.py:1833 -- Connecting to existing Ray cluster at address: 10.192.5.31:6379...
2025-10-06 19:04:05,334 INFO worker.py:2004 -- Connected to Ray cluster. View the dashboard at 10.192.5.31:8265 
/home/ray/anaconda3/lib/python3.10/site-packages/ray/_private/worker.py:2052: FutureWarning: Tip: In future versions of Ray, Ray will no longer override accelerator visible devices env var if num_gpus=0 or num_gpus=None (default). To enable this behavior and turn off this error message, set RAY_ACCEL_ENV_VAR_OVERRIDE_ON_ZERO=0
  warnings.warn(
(raylet) There are tasks with infeasible resource requests that cannot be scheduled. See https://docs.ray.io/en/latest/ray-core/scheduling/index.html#ray-scheduling-resources for more details. Possible solutions: 1. Updating the ray cluster to include nodes with all required resources 2. To cause the tasks with infeasible requests to raise an error instead of hanging, set the 'RAY_enable_infeasible_task_early_exit=true'. This feature will be turned on by default in a future release of Ray.
(autoscaler +5s) Tip: use `ray status` to view detailed cluster status. To disable these messages, set RAY_SCHEDULER_EVENTS=0.
(autoscaler +5s) Adding 2 node(s) of type group-3.
(autoscaler +5s) Resized to 2 CPUs.
(autoscaler +5s) No available node types can fulfill resource requests {'bundle_group_6bbb5a54efa735d58782e60d265c01000000': 0.001}*1. Add suitable node types to this cluster to resolve this issue.
(autoscaler +10s) No available node types can fulfill resource requests {'bundle_group_6bbb5a54efa735d58782e60d265c01000000': 0.001}*1. Add suitable node types to this cluster to resolve this issue.
(autoscaler +15s) No available node types can fulfill resource requests {'bundle_group_6bbb5a54efa735d58782e60d265c01000000': 0.001}*1. Add suitable node types to this cluster to resolve this issue.

Copy link
Copy Markdown
Contributor

@MengjinYan MengjinYan left a comment

Choose a reason for hiding this comment

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

Thanks for the fix!

@edoakes edoakes merged commit 8dcd4c7 into ray-project:master Oct 7, 2025
6 checks passed
liulehui pushed a commit to liulehui/ray that referenced this pull request Oct 9, 2025
ray-project#56826)

This PR adds support to parse the
`GangResourceRequest.bundle_selectors.resource_requests` field for gang
resource requests in the V2 Autoscaler. This proto field replaces the
deprecated `GangResourceRequest.resource_requests`
([definition](https://github.com/ray-project/ray/blob/3408fe94a687e0ed03f6861ab8f9e8708a68763a/src/ray/protobuf/autoscaler.proto#L85))
in order to support repeated selectors for fallback strategy. This
change is required for autoscaling to work with the
`bundle_label_selector` placement group option.

This PR also adds an e2e test case for scaling up a placement group with
`bundle_label_selector` specified. This tests verifies the behavior that
the v2 scheduler will scale nodes satisfying the given label
constraints, preferring nodes with the required `labels` over node types
with sufficient resources, but lacking those labels.

ray-project#51564

---------

Signed-off-by: Ryan O'Leary <ryanaoleary@google.com>
Co-authored-by: Mengjin Yan <mengjinyan3@gmail.com>
joshkodi pushed a commit to joshkodi/ray that referenced this pull request Oct 13, 2025
ray-project#56826)

This PR adds support to parse the
`GangResourceRequest.bundle_selectors.resource_requests` field for gang
resource requests in the V2 Autoscaler. This proto field replaces the
deprecated `GangResourceRequest.resource_requests`
([definition](https://github.com/ray-project/ray/blob/3408fe94a687e0ed03f6861ab8f9e8708a68763a/src/ray/protobuf/autoscaler.proto#L85))
in order to support repeated selectors for fallback strategy. This
change is required for autoscaling to work with the
`bundle_label_selector` placement group option.

This PR also adds an e2e test case for scaling up a placement group with
`bundle_label_selector` specified. This tests verifies the behavior that
the v2 scheduler will scale nodes satisfying the given label
constraints, preferring nodes with the required `labels` over node types
with sufficient resources, but lacking those labels.

ray-project#51564

---------

Signed-off-by: Ryan O'Leary <ryanaoleary@google.com>
Co-authored-by: Mengjin Yan <mengjinyan3@gmail.com>
Signed-off-by: Josh Kodi <joshkodi@gmail.com>
ArturNiederfahrenhorst pushed a commit to ArturNiederfahrenhorst/ray that referenced this pull request Oct 13, 2025
ray-project#56826)

This PR adds support to parse the
`GangResourceRequest.bundle_selectors.resource_requests` field for gang
resource requests in the V2 Autoscaler. This proto field replaces the
deprecated `GangResourceRequest.resource_requests`
([definition](https://github.com/ray-project/ray/blob/3408fe94a687e0ed03f6861ab8f9e8708a68763a/src/ray/protobuf/autoscaler.proto#L85))
in order to support repeated selectors for fallback strategy. This
change is required for autoscaling to work with the
`bundle_label_selector` placement group option.

This PR also adds an e2e test case for scaling up a placement group with
`bundle_label_selector` specified. This tests verifies the behavior that
the v2 scheduler will scale nodes satisfying the given label
constraints, preferring nodes with the required `labels` over node types
with sufficient resources, but lacking those labels.

ray-project#51564

---------

Signed-off-by: Ryan O'Leary <ryanaoleary@google.com>
Co-authored-by: Mengjin Yan <mengjinyan3@gmail.com>
justinyeh1995 pushed a commit to justinyeh1995/ray that referenced this pull request Oct 20, 2025
ray-project#56826)

This PR adds support to parse the
`GangResourceRequest.bundle_selectors.resource_requests` field for gang
resource requests in the V2 Autoscaler. This proto field replaces the
deprecated `GangResourceRequest.resource_requests`
([definition](https://github.com/ray-project/ray/blob/3408fe94a687e0ed03f6861ab8f9e8708a68763a/src/ray/protobuf/autoscaler.proto#L85))
in order to support repeated selectors for fallback strategy. This
change is required for autoscaling to work with the
`bundle_label_selector` placement group option.

This PR also adds an e2e test case for scaling up a placement group with
`bundle_label_selector` specified. This tests verifies the behavior that
the v2 scheduler will scale nodes satisfying the given label
constraints, preferring nodes with the required `labels` over node types
with sufficient resources, but lacking those labels.

ray-project#51564

---------

Signed-off-by: Ryan O'Leary <ryanaoleary@google.com>
Co-authored-by: Mengjin Yan <mengjinyan3@gmail.com>
xinyuangui2 pushed a commit to xinyuangui2/ray that referenced this pull request Oct 22, 2025
ray-project#56826)

This PR adds support to parse the
`GangResourceRequest.bundle_selectors.resource_requests` field for gang
resource requests in the V2 Autoscaler. This proto field replaces the
deprecated `GangResourceRequest.resource_requests`
([definition](https://github.com/ray-project/ray/blob/3408fe94a687e0ed03f6861ab8f9e8708a68763a/src/ray/protobuf/autoscaler.proto#L85))
in order to support repeated selectors for fallback strategy. This
change is required for autoscaling to work with the
`bundle_label_selector` placement group option.

This PR also adds an e2e test case for scaling up a placement group with
`bundle_label_selector` specified. This tests verifies the behavior that
the v2 scheduler will scale nodes satisfying the given label
constraints, preferring nodes with the required `labels` over node types
with sufficient resources, but lacking those labels.

ray-project#51564

---------

Signed-off-by: Ryan O'Leary <ryanaoleary@google.com>
Co-authored-by: Mengjin Yan <mengjinyan3@gmail.com>
Signed-off-by: xgui <xgui@anyscale.com>
landscapepainter pushed a commit to landscapepainter/ray that referenced this pull request Nov 17, 2025
ray-project#56826)

This PR adds support to parse the
`GangResourceRequest.bundle_selectors.resource_requests` field for gang
resource requests in the V2 Autoscaler. This proto field replaces the
deprecated `GangResourceRequest.resource_requests`
([definition](https://github.com/ray-project/ray/blob/3408fe94a687e0ed03f6861ab8f9e8708a68763a/src/ray/protobuf/autoscaler.proto#L85))
in order to support repeated selectors for fallback strategy. This
change is required for autoscaling to work with the
`bundle_label_selector` placement group option.

This PR also adds an e2e test case for scaling up a placement group with
`bundle_label_selector` specified. This tests verifies the behavior that
the v2 scheduler will scale nodes satisfying the given label
constraints, preferring nodes with the required `labels` over node types
with sufficient resources, but lacking those labels.

ray-project#51564

---------

Signed-off-by: Ryan O'Leary <ryanaoleary@google.com>
Co-authored-by: Mengjin Yan <mengjinyan3@gmail.com>
Aydin-ab pushed a commit to Aydin-ab/ray-aydin that referenced this pull request Nov 19, 2025
ray-project#56826)

This PR adds support to parse the
`GangResourceRequest.bundle_selectors.resource_requests` field for gang
resource requests in the V2 Autoscaler. This proto field replaces the
deprecated `GangResourceRequest.resource_requests`
([definition](https://github.com/ray-project/ray/blob/3408fe94a687e0ed03f6861ab8f9e8708a68763a/src/ray/protobuf/autoscaler.proto#L85))
in order to support repeated selectors for fallback strategy. This
change is required for autoscaling to work with the
`bundle_label_selector` placement group option.

This PR also adds an e2e test case for scaling up a placement group with
`bundle_label_selector` specified. This tests verifies the behavior that
the v2 scheduler will scale nodes satisfying the given label
constraints, preferring nodes with the required `labels` over node types
with sufficient resources, but lacking those labels.

ray-project#51564

---------

Signed-off-by: Ryan O'Leary <ryanaoleary@google.com>
Co-authored-by: Mengjin Yan <mengjinyan3@gmail.com>
Signed-off-by: Aydin Abiar <aydin@anyscale.com>
Future-Outlier pushed a commit to Future-Outlier/ray that referenced this pull request Dec 7, 2025
ray-project#56826)

This PR adds support to parse the
`GangResourceRequest.bundle_selectors.resource_requests` field for gang
resource requests in the V2 Autoscaler. This proto field replaces the
deprecated `GangResourceRequest.resource_requests`
([definition](https://github.com/ray-project/ray/blob/3408fe94a687e0ed03f6861ab8f9e8708a68763a/src/ray/protobuf/autoscaler.proto#L85))
in order to support repeated selectors for fallback strategy. This
change is required for autoscaling to work with the
`bundle_label_selector` placement group option.

This PR also adds an e2e test case for scaling up a placement group with
`bundle_label_selector` specified. This tests verifies the behavior that
the v2 scheduler will scale nodes satisfying the given label
constraints, preferring nodes with the required `labels` over node types
with sufficient resources, but lacking those labels.

ray-project#51564

---------

Signed-off-by: Ryan O'Leary <ryanaoleary@google.com>
Co-authored-by: Mengjin Yan <mengjinyan3@gmail.com>
Signed-off-by: Future-Outlier <eric901201@gmail.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

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.

3 participants