Skip to content

Conversation

@ryanaoleary
Copy link
Contributor

@ryanaoleary ryanaoleary commented Oct 14, 2025

Why are these changes needed?

This PR adds the label_selector option to the supported list of Actor options for a Serve deployment. Additionally, we add bundle_label_selector to specify label selectors for bundles when placement_group_bundles are specified for the deployment. These two options are already supported for Tasks/Actors and placement groups respectively.

Example use case:

llm_config = LLMConfig(
    model_loading_config={
        "model_id": "meta-llama/Meta-Llama-3-70B-Instruct",
        "model_source": "huggingface",
    },
    engine_kwargs=tpu_engine_config,
    resources_per_bundle={"TPU": 4},
    runtime_env={"env_vars": {"VLLM_USE_V1": "1"}},
    deployment_config={
        "num_replicas": 4,
        "ray_actor_options": {
            # In a GKE cluster with multiple TPU node-pools, schedule
            # only to the desired slice.
            "label_selector": {
                "ray.io/tpu-topology": "4x4" # added by default by Ray
            }
        }
    }
)

The expected behaviors of these new fields is as follows:

Pack scheduling enabled

PACK/STRICT_PACK PG strategy:

  • Standard PG without bundle_label_selector or fallback:

    • Sorts replicas by resource size (descending). Attempts to find the "best fit" node (minimizing fragmentation) that has available resources. Creates a Placement Group on that target node.
  • PG node label selector provided:

    • Same behavior as regular placement group but filters the list of candidate nodes to only those matching the label selector before finding the best fit
  • PG node label selector and fallback:
    Same as above but when scheduling tries the following:

    1. Tries to find a node matching the primary placement_group_bundles and bundle_label_selector.
    2. If no node fits, iterates through the placement_group_fallback_strategy. For each fallback entry, tries to find a node matching that entry's bundles and labels.
    3. If a node is found, creates a PG on it.

SPREAD/STRICT_SPREAD PG strategy:

  • If any deployment uses these strategies, the global logic falls back to "Spread Scheduling" (see below)

Spread scheduling enabled

  • Standard PG without bundle_label_selector or fallback:
    • Creates a Placement Group via Ray Core without specifying a target_node_id. Ray Core decides placement based on the strategy.
  • PG node label selector provided:
    • Serve passes the bundle_label_selector to the CreatePlacementGroupRequest. Ray Core handles the soft/hard constraint logic during PG creation.
  • PG node label selector and fallback:
    • Serve passes the bundle_label_selector to the CreatePlacementGroupRequest, fallback_strategy is not yet supported in the placement group options so this field isn't passed / considered. It's only used in the "best fit" node selection logic which is skipped for Spread scheduling.

Related issue number

#51564

Checks

  • I've signed off every commit(by using the -s flag, i.e., git commit -s) in this PR.
  • I've run pre-commit jobs to lint the changes in this PR. (pre-commit setup)
  • 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 :(

@ryanaoleary ryanaoleary marked this pull request as ready for review October 14, 2025 07:08
@ryanaoleary ryanaoleary requested a review from a team as a code owner October 14, 2025 07:08
@ryanaoleary
Copy link
Contributor Author

cc: @MengjinYan @eicherseiji I think this change can help enable TPU use-cases with Ray LLM, since it'll allow users to target the desired slice/topology based on labels like these:

ray.io/accelerator-type: TPU-V6E
ray.io/node-group: tpu-group
ray.io/node-id: 0870a6a06413aed6079c15eeaa4f61e8a1413fa6140fc70c93608505
ray.io/tpu-pod-type: v6e-16
ray.io/tpu-slice-name: tpu-group-0
ray.io/tpu-topology: 4x4
ray.io/tpu-worker-id: '2'

cursor[bot]

This comment was marked as outdated.

@ray-gardener ray-gardener bot added serve Ray Serve Related Issue community-contribution Contributed by the community labels Oct 14, 2025
@eicherseiji eicherseiji self-requested a review October 14, 2025 23:21
@MengjinYan MengjinYan added the go add ONLY when ready to merge, run all tests label Oct 15, 2025
Copy link
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.

The current change looks good.

At the same time, I have a general question, probably also to @eicherseiji as well: I'm not familiar with the serve codebase. But from the look at the code, the change in the PR seems not cover the whole code path from the replica config to actually creating the placement group where we need to apply the bundle label selector (e.g. CreatePlacementGroupRequest, DeploymentTargetState, DeploymentVersion, etc.).

Wondering if we should include the change to the rest of the code path in this PR as well?

@eicherseiji
Copy link
Contributor

Hi @ryanaoleary! Seems like it may be useful, but could you go into more detail about the problem this solved for you? I.e. is it typical to have multiple TPU node pools/heterogeneous compute in a TPU cluster?

Signed-off-by: Ryan O'Leary <[email protected]>
@ryanaoleary
Copy link
Contributor Author

Hi @ryanaoleary! Seems like it may be useful, but could you go into more detail about the problem this solved for you? I.e. is it typical to have multiple TPU node pools/heterogeneous compute in a TPU cluster?

The use-case would be for TPU multi-slice, where it's important that the Ray nodes reserved with some group of resources (i.e. a bundle of TPUs) are a part of the same actual TPU slice so that they can benefit from the high speed ICI interconnects. There are GKE webhooks that set some TPU information as env vars when the Pods are created, including topology and a unique identifier for the slice which we set as Ray node labels. They also inject podAffinities so that the TPU Pods are scheduled to the same node-pool (i.e. TPU slice) and co-located. So if we then use those labels for scheduling in the application code, we can guarantee that the workers are running on co-located TPU devices.

For a RayCluster with multiple TPU slices (of the same topology or different), we currently only schedule using TPU: <number-resources> and/or the TPU generation (i.e. TPU-V6E) which can result in the placement group spanning across TPU slices.

@eicherseiji
Copy link
Contributor

eicherseiji commented Oct 16, 2025

@ryanaoleary I see, thanks! Is there a reason we can't extend/re-use the resources_per_bundle concept for this instead?

@MengjinYan
Copy link
Contributor

Hi @ryanaoleary! Seems like it may be useful, but could you go into more detail about the problem this solved for you? I.e. is it typical to have multiple TPU node pools/heterogeneous compute in a TPU cluster?

In addition to the TPU support, in general, we want to have label support in all library APIs so that user can do scheduling based on node labels as well.

@ryanaoleary
Copy link
Contributor Author

@ryanaoleary I see, thanks! Is there a reason we can't extend/re-use the resources_per_bundle concept for this instead?

I think just configuring resources_per_bundle with a TPU resource alone wouldn't work, we'd have to add some matching custom resource that denotes the TPU slice name to Pods of the same slice and use it as a workaround. That would also work, but moving away from this workaround was one of the rationales for adding label selectors. Additionally, we already set these Ray node labels for TPUs by default so it'd be less work for the user to get it working.

@github-actions
Copy link

This pull request has been automatically marked as stale because it has not had
any activity for 14 days. It will be closed in another 14 days if no further activity occurs.
Thank you for your contributions.

You can always ask for help on our discussion forum or Ray's public slack channel.

If you'd like to keep this open, just leave any comment, and the stale label will be removed.

@github-actions github-actions bot added the stale The issue is stale. It will be closed within 7 days unless there are further conversation label Oct 31, 2025
@github-actions github-actions bot added unstale A PR that has been marked unstale. It will not get marked stale again if this label is on it. and removed stale The issue is stale. It will be closed within 7 days unless there are further conversation labels Nov 4, 2025
@abrarsheikh
Copy link
Contributor

@ryanaoleary I will review this PR next week

@ktyxx
Copy link
Contributor

ktyxx commented Dec 9, 2025

thanks for working on this! Really looking forward to using label selectors in Serve.
One thing I was wondering - would it be possible to also expose fallback_strategy from ray core?
My use case is that I'd like to use labels more as a "preference" rather than a strict requirement. For example, I want to prioritize deploying to nodes with a specific label, but if none are available, just fall back to any node (like the default scheduling behavior). I actually opened an issue about this before (#59055) - initially thinking about using NodeLabelSchedulingStrategy with hard/soft params, but was told it's being deprecated in favor of label selectors. So it'd be great if Serve could also expose fallback_strategy to support soft constraints via fallback_strategy=[{"label_selector": {}}] pattern.
What do you think?

@ryanaoleary
Copy link
Contributor Author

the def schedule function is hard to follow/review right now, probably also because I am not 100% sure what the expected behavior is under various situations. could you fill out the following situations and add it to the PR description

  1. Pack scheduling enabled
  • PG strategy pack/strict-pack/spread/strict-spread
  • PG node label selector provided vs when not provided
  • PG node label fallback provided vs when not provided
  1. spread scheduling enabled
  • PG strategy pack/strict-pack/spread/strict-spread
  • PG node label selector provided vs when not provided
  • PG node label fallback provided vs when not provided

@abrarsheikh Done, I updated the PR description with an outline that I think covers all these situations.

@abrarsheikh
Copy link
Contributor

abrarsheikh commented Jan 8, 2026

I think the logic of your implementation is correct afaict, I am writing down my understanding of how this stuff is supposed to work, so that we both are on the same page

Decision Flow Chart For Scheduler

Is RAY_SERVE_USE_PACK_SCHEDULING_STRATEGY=1?
├── No → Use Spread Scheduling (Ray Core decides placement)
└── Yes
    └── Any deployment uses non-STRICT_PACK PG?
        ├── Yes → Use Spread Scheduling (Ray Core decides placement)
        └── No → Use Pack Scheduling
            └── For each replica:
                ├── Has PG bundles?
                │   ├── Yes (must be STRICT_PACK)
                │   │   ├── Has bundle_label_selector?
                │   │   │   ├── Yes → Filter nodes by ALL selectors, find best-fit
                │   │   │   └── No → Find best-fit node for sum of bundles
                │   │   └── Create PG with _soft_target_node_id + bundle_label_selector
                │   └── No (Actor only)
                │       ├── Has label_selector?
                │       │   ├── Yes → Filter nodes, find best-fit
                │       │   └── No → Find best-fit node
                │       ├── Has fallback_strategy?
                │       │   └── Yes → Try each fallback's labels until match
                │       └── Create actor with NodeAffinitySchedulingStrategy
                └── Pass to Ray Core for final placement
Configuration Scheduling Mode Serve Node Selection Ray Core Strategy
Actor only Pack Best-fit NodeAffinitySchedulingStrategy
Actor + label_selector Pack Best-fit with label filter NodeAffinitySchedulingStrategy
Actor + label_selector + fallback Pack Try labels in order NodeAffinitySchedulingStrategy
STRICT_PACK PG Pack Best-fit for sum of bundles PlacementGroupSchedulingStrategy with _soft_target_node_id
STRICT_PACK PG + bundle_label_selector Pack Best-fit for sum of bundles, with label filter PlacementGroupSchedulingStrategy with _soft_target_node_id + bundle_label_selector
PACK/SPREAD/STRICT_SPREAD PG Spread None PlacementGroupSchedulingStrategy (Ray Core decides)
Any + placement_group_fallback_strategy N/A N/A NotImplementedError

ryanaoleary and others added 4 commits January 7, 2026 23:41
ryanaoleary and others added 5 commits January 8, 2026 03:03
Co-authored-by: Abrar Sheikh <[email protected]>
Signed-off-by: Ryan O'Leary <[email protected]>
Co-authored-by: Abrar Sheikh <[email protected]>
Signed-off-by: Ryan O'Leary <[email protected]>
Co-authored-by: Abrar Sheikh <[email protected]>
Signed-off-by: Ryan O'Leary <[email protected]>
Co-authored-by: Abrar Sheikh <[email protected]>
Signed-off-by: Ryan O'Leary <[email protected]>
Co-authored-by: Abrar Sheikh <[email protected]>
Signed-off-by: Ryan O'Leary <[email protected]>
@ryanaoleary ryanaoleary requested a review from a team as a code owner January 9, 2026 01:56
Signed-off-by: ryanaoleary <[email protected]>
return False

# !in operator
if selector_value.startswith("!in(") and selector_value.endswith(")"):
Copy link
Contributor

Choose a reason for hiding this comment

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

should we trim the selector_value before passing through the string parsers?

# Actor: Use Actor label selector
if "label_selector" in scheduling_request.actor_options:
primary_labels = [
scheduling_request.actor_options["label_selector"] or {}
Copy link
Contributor

Choose a reason for hiding this comment

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

is None allowed in actor_options["label_selector"]?

Comment on lines +819 to +822
fallback_labels = [fallback.get("label_selector", {}) or {}]
placement_candidates.append(
(scheduling_request.required_resources, fallback_labels)
)
Copy link
Contributor

Choose a reason for hiding this comment

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

can there be a fallback with label_selector == None?

Also should we skip adding to placement_candidates if fallback_labels is empty?

Comment on lines +373 to +429
@pytest.mark.parametrize(
"node_value, selector_value, expected",
[
# Equals operator
("us-west", "us-west", True),
("us-east", "us-west", False),
(None, "us-west", False),
# Not equals (!) operator
("us-west", "!us-east", True),
("us-east", "!us-east", False),
(None, "!us-east", True),
# In operator
("A100", "in(A100, H100)", True),
("T4", "in(A100, H100)", False),
(None, "in(A100, H100)", False),
("value", "in(value)", True),
# Not in operator
("T4", "!in(A100, H100)", True),
("A100", "!in(A100, H100)", False),
(None, "!in(A100, H100)", True),
# Invalid types
("A100", None, False),
("A100", 123, False),
],
)
def test_match_label_selector_value(node_value, selector_value, expected):
assert match_label_selector_value(node_value, selector_value) == expected


@pytest.mark.parametrize(
"node_labels, selector, expected",
[
# Match all
(
{"region": "us-west", "gpu": "A100"},
{"region": "us-west", "gpu": "A100"},
True,
),
# Partial match
(
{"region": "us-west", "gpu": "T4"},
{"region": "us-west", "gpu": "A100"},
False,
),
# Multiple operators
(
{"region": "us-west", "env": "prod"},
{"region": "in(us-west, us-east)", "env": "!dev"},
True,
),
# Missing keys in node labels
({"region": "us-west"}, {"gpu": "!in(H100)"}, True),
({"region": "us-west"}, {"gpu": "in(H100)"}, False),
],
)
def test_match_label_selector(node_labels, selector, expected):
assert match_label_selector(node_labels, selector) == expected
Copy link
Contributor

Choose a reason for hiding this comment

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

Suggested change
@pytest.mark.parametrize(
"node_value, selector_value, expected",
[
# Equals operator
("us-west", "us-west", True),
("us-east", "us-west", False),
(None, "us-west", False),
# Not equals (!) operator
("us-west", "!us-east", True),
("us-east", "!us-east", False),
(None, "!us-east", True),
# In operator
("A100", "in(A100, H100)", True),
("T4", "in(A100, H100)", False),
(None, "in(A100, H100)", False),
("value", "in(value)", True),
# Not in operator
("T4", "!in(A100, H100)", True),
("A100", "!in(A100, H100)", False),
(None, "!in(A100, H100)", True),
# Invalid types
("A100", None, False),
("A100", 123, False),
],
)
def test_match_label_selector_value(node_value, selector_value, expected):
assert match_label_selector_value(node_value, selector_value) == expected
@pytest.mark.parametrize(
"node_labels, selector, expected",
[
# Match all
(
{"region": "us-west", "gpu": "A100"},
{"region": "us-west", "gpu": "A100"},
True,
),
# Partial match
(
{"region": "us-west", "gpu": "T4"},
{"region": "us-west", "gpu": "A100"},
False,
),
# Multiple operators
(
{"region": "us-west", "env": "prod"},
{"region": "in(us-west, us-east)", "env": "!dev"},
True,
),
# Missing keys in node labels
({"region": "us-west"}, {"gpu": "!in(H100)"}, True),
({"region": "us-west"}, {"gpu": "in(H100)"}, False),
],
)
def test_match_label_selector(node_labels, selector, expected):
assert match_label_selector(node_labels, selector) == expected
@pytest.mark.parametrize(
"node_value, selector_value, expected",
[
# Equals operator
("us-west", "us-west", True),
("us-east", "us-west", False),
(None, "us-west", False),
# Equals with empty strings
("", "", True),
("value", "", False),
("", "value", False),
# Not equals (!) operator
("us-west", "!us-east", True),
("us-east", "!us-east", False),
(None, "!us-east", True),
# Not equals with empty string
("value", "!", True), # !<empty> means not equal to empty
("", "!", False), # empty string equals empty, so !empty is False
("", "!value", True), # empty string != "value"
# In operator
("A100", "in(A100, H100)", True),
("T4", "in(A100, H100)", False),
(None, "in(A100, H100)", False),
("value", "in(value)", True),
# In operator without spaces
("A100", "in(A100,H100)", True),
("H100", "in(A100,H100)", True),
("T4", "in(A100,H100)", False),
# In operator with many values
("c", "in(a, b, c, d, e)", True),
("z", "in(a, b, c, d, e)", False),
# In operator with empty string value not in list
("", "in(A100, H100)", False),
# Not in operator
("T4", "!in(A100, H100)", True),
("A100", "!in(A100, H100)", False),
(None, "!in(A100, H100)", True),
# Not in operator without spaces
("T4", "!in(A100,H100)", True),
("A100", "!in(A100,H100)", False),
# Not in with single value
("T4", "!in(A100)", True),
("A100", "!in(A100)", False),
# Not in with empty string
("", "!in(A100, H100)", True), # empty string not in list
# Invalid selector types
("A100", None, False),
("A100", 123, False),
(None, None, False),
("value", [], False),
("value", {}, False),
# Node value that looks like operator syntax (edge case)
("in(x)", "in(x)", True),
("!value", "!value", True),
# Case sensitivity
("A100", "a100", False),
("A100", "in(a100, h100)", False),
("A100", "!a100", True),
],
)
def test_match_label_selector_value(node_value, selector_value, expected):
assert match_label_selector_value(node_value, selector_value) == expected
@pytest.mark.parametrize(
"node_labels, selector, expected",
[
# Empty cases
({}, {}, True), # Empty selector matches everything
({"region": "us-west"}, {}, True), # Empty selector matches any node
({}, {"region": "us-west"}, False), # Missing key with equals fails
({}, {"region": "!us-west"}, True), # Missing key with not-equals succeeds
# Match all
(
{"region": "us-west", "gpu": "A100"},
{"region": "us-west", "gpu": "A100"},
True,
),
# Single key match
({"region": "us-west"}, {"region": "us-west"}, True),
({"region": "us-east"}, {"region": "us-west"}, False),
# Partial match (one key matches, one doesn't)
(
{"region": "us-west", "gpu": "T4"},
{"region": "us-west", "gpu": "A100"},
False,
),
# Multiple operators
(
{"region": "us-west", "env": "prod"},
{"region": "in(us-west, us-east)", "env": "!dev"},
True,
),
# Multiple operators - failure case
(
{"region": "us-west", "env": "dev"},
{"region": "in(us-west, us-east)", "env": "!dev"},
False,
),
# Missing keys in node labels
({"region": "us-west"}, {"gpu": "!in(H100)"}, True),
({"region": "us-west"}, {"gpu": "in(H100)"}, False),
({"region": "us-west"}, {"gpu": "!H100"}, True), # Missing key with ! succeeds
# Extra keys in node_labels that selector doesn't care about
(
{"region": "us-west", "gpu": "A100", "env": "prod", "zone": "a"},
{"region": "us-west"},
True,
),
# All keys matching with mixed operators
(
{"region": "us-west", "gpu": "A100", "env": "prod"},
{"region": "us-west", "gpu": "in(A100, H100)", "env": "!dev"},
True,
),
# First key matches, second doesn't
(
{"region": "us-west", "gpu": "T4"},
{"region": "in(us-west, us-east)", "gpu": "in(A100, H100)"},
False,
),
# First key doesn't match
(
{"region": "eu-west", "gpu": "A100"},
{"region": "in(us-west, us-east)", "gpu": "in(A100, H100)"},
False,
),
# Empty string values
({"region": ""}, {"region": ""}, True),
({"region": "us-west"}, {"region": ""}, False),
# Complex multi-key scenario
(
{"cloud": "aws", "region": "us-west", "gpu": "A100", "spot": "true"},
{"cloud": "!gcp", "region": "in(us-west, us-east)", "gpu": "A100"},
True,
),
],
)
def test_match_label_selector(node_labels, selector, expected):
assert match_label_selector(node_labels, selector) == expected

Copy link
Contributor

Choose a reason for hiding this comment

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

just add more params

"labels": {"region": "us-east", "gpu-type": "H100"},
}
cluster.add_node(**node2_config)
node_2_id = ray.get(_get_node_id.options(resources={"worker2": 1}).remote())
Copy link
Contributor

Choose a reason for hiding this comment

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

this is clever, nice :)


serve.shutdown()


Copy link
Contributor

Choose a reason for hiding this comment

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

let's add a few more tests here

  1. STRICT_PACK PG + bundle_label_selector (Pack Scheduling Mode)
  2. STRICT_SPREAD PG Strategy
  3. placement_group_fallback_strategy → NotImplementedError
  4. Non-STRICT_PACK PG Forces Spread Mode (when RAY_SERVE_USE_PACK_SCHEDULING_STRATEGY=1)
  5. Multiple Bundles with bundle_label_selector
  6. Actor label_selector + fallback with Multiple Fallbacks

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 go add ONLY when ready to merge, run all tests serve Ray Serve Related Issue unstale A PR that has been marked unstale. It will not get marked stale again if this label is on it.

Projects

None yet

Development

Successfully merging this pull request may close these issues.

7 participants