Skip to content

[Serve] Add label_selector and bundle_label_selector to Serve API#57694

Merged
abrarsheikh merged 88 commits intoray-project:masterfrom
ryanaoleary:add-label-selector-serve-option
Jan 28, 2026
Merged

[Serve] Add label_selector and bundle_label_selector to Serve API#57694
abrarsheikh merged 88 commits intoray-project:masterfrom
ryanaoleary:add-label-selector-serve-option

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 :(

Signed-off-by: Ryan O'Leary <ryanaoleary@google.com>
@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 <ryanaoleary@google.com>
@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

[c225938](/ray-project/ray/pull/57694/commits/c22593822c02c4b36e63b53e640e3a66190eb3c3)

Sorry should be fixed with c225938

Copy link

@cursor cursor bot left a comment

Choose a reason for hiding this comment

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

Cursor Bugbot has reviewed your changes and found 1 potential issue.

Signed-off-by: ryanaoleary <ryanaoleary@google.com>
Signed-off-by: ryanaoleary <ryanaoleary@google.com>
Copy link

@cursor cursor bot left a comment

Choose a reason for hiding this comment

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

Cursor Bugbot has reviewed your changes and found 1 potential issue.

ryanaoleary and others added 3 commits January 22, 2026 15:47
@ryanaoleary
Copy link
Contributor Author

ryanaoleary commented Jan 23, 2026

I believe I fixed the CI error with 561f814, test_e2e_serve_label_selector_unschedulable[True-PACK] was hanging and timing out because the PACK and SPREAD policies would infinitely retry impossible placement group requests (such as missing labels) that the scheduler returned Failed() for, starving the GCS thread and preventing new test nodes from registering (which the test that timed out relied on). The fix adds an IsRequestFeasible() pre-check that returns Infeasible() for impossible constraints - preventing the infinite looping issue I saw in the CI logs. cc: @MengjinYan this required another core change which would need a re-review, I could put it in a separate PR but I left it here for now because I found it useful to verify the tests pass.

Copy link

@cursor cursor bot left a comment

Choose a reason for hiding this comment

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

Cursor Bugbot has reviewed your changes and found 1 potential issue.

Signed-off-by: Ryan O'Leary <113500783+ryanaoleary@users.noreply.github.com>
@ryanaoleary
Copy link
Contributor Author

cc: @abrarsheikh for merge

@abrarsheikh
Copy link
Contributor

@MengjinYan want to take another pass at the core changes?

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.

Thanks!

ryanaoleary and others added 2 commits January 27, 2026 11:34
Co-authored-by: Mengjin Yan <mengjinyan3@gmail.com>
Signed-off-by: Ryan O'Leary <113500783+ryanaoleary@users.noreply.github.com>
Signed-off-by: ryanaoleary <ryanaoleary@google.com>
Copy link

@cursor cursor bot left a comment

Choose a reason for hiding this comment

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

Cursor Bugbot has reviewed your changes and found 1 potential issue.

Signed-off-by: ryanaoleary <ryanaoleary@google.com>
@abrarsheikh abrarsheikh merged commit 829c85c into ray-project:master Jan 28, 2026
6 checks passed
@abrarsheikh
Copy link
Contributor

solid work on this @ryanaoleary 👍

@ryanaoleary
Copy link
Contributor Author

solid work on this @ryanaoleary 👍

Thank you again for your help!!

jinbum-kim pushed a commit to jinbum-kim/ray that referenced this pull request Jan 29, 2026
…ray-project#57694)

<!-- Thank you for your contribution! Please review
https://github.com/ray-project/ray/blob/master/CONTRIBUTING.rst before
opening a pull request. -->

<!-- Please add a reviewer to the assignee section when you create a PR.
If you don't have the access to it, we will shortly find a reviewer and
assign them to your PR. -->

## 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

ray-project#51564

## Checks

- [x] I've signed off every commit(by using the -s flag, i.e., `git
commit -s`) in this PR.
- [x] I've run pre-commit jobs to lint the changes in this PR.
([pre-commit
setup](https://docs.ray.io/en/latest/ray-contribute/getting-involved.html#lint-and-formatting))
- [ ] 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.
- [x] 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
   - [x] Unit tests
   - [ ] Release tests
   - [ ] This PR is not tested :(

---------

Signed-off-by: Ryan O'Leary <ryanaoleary@google.com>
Signed-off-by: ryanaoleary <ryanaoleary@google.com>
Signed-off-by: Ryan O'Leary <113500783+ryanaoleary@users.noreply.github.com>
Co-authored-by: Cindy Zhang <cindyzyx9@gmail.com>
Co-authored-by: Abrar Sheikh <abrar2002as@gmail.com>
Co-authored-by: Mengjin Yan <mengjinyan3@gmail.com>
Signed-off-by: jinbum-kim <jinbum9958@gmail.com>
limarkdcunha pushed a commit to limarkdcunha/ray that referenced this pull request Jan 29, 2026
…ray-project#57694)

<!-- Thank you for your contribution! Please review
https://github.com/ray-project/ray/blob/master/CONTRIBUTING.rst before
opening a pull request. -->

<!-- Please add a reviewer to the assignee section when you create a PR.
If you don't have the access to it, we will shortly find a reviewer and
assign them to your PR. -->

## 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

ray-project#51564

## Checks

- [x] I've signed off every commit(by using the -s flag, i.e., `git
commit -s`) in this PR.
- [x] I've run pre-commit jobs to lint the changes in this PR.
([pre-commit
setup](https://docs.ray.io/en/latest/ray-contribute/getting-involved.html#lint-and-formatting))
- [ ] 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.
- [x] 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
   - [x] Unit tests
   - [ ] Release tests
   - [ ] This PR is not tested :(

---------

Signed-off-by: Ryan O'Leary <ryanaoleary@google.com>
Signed-off-by: ryanaoleary <ryanaoleary@google.com>
Signed-off-by: Ryan O'Leary <113500783+ryanaoleary@users.noreply.github.com>
Co-authored-by: Cindy Zhang <cindyzyx9@gmail.com>
Co-authored-by: Abrar Sheikh <abrar2002as@gmail.com>
Co-authored-by: Mengjin Yan <mengjinyan3@gmail.com>
400Ping pushed a commit to 400Ping/ray that referenced this pull request Feb 1, 2026
…ray-project#57694)

<!-- Thank you for your contribution! Please review
https://github.com/ray-project/ray/blob/master/CONTRIBUTING.rst before
opening a pull request. -->

<!-- Please add a reviewer to the assignee section when you create a PR.
If you don't have the access to it, we will shortly find a reviewer and
assign them to your PR. -->

## 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

ray-project#51564

## Checks

- [x] I've signed off every commit(by using the -s flag, i.e., `git
commit -s`) in this PR.
- [x] I've run pre-commit jobs to lint the changes in this PR.
([pre-commit
setup](https://docs.ray.io/en/latest/ray-contribute/getting-involved.html#lint-and-formatting))
- [ ] 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.
- [x] 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
   - [x] Unit tests
   - [ ] Release tests
   - [ ] This PR is not tested :(

---------

Signed-off-by: Ryan O'Leary <ryanaoleary@google.com>
Signed-off-by: ryanaoleary <ryanaoleary@google.com>
Signed-off-by: Ryan O'Leary <113500783+ryanaoleary@users.noreply.github.com>
Co-authored-by: Cindy Zhang <cindyzyx9@gmail.com>
Co-authored-by: Abrar Sheikh <abrar2002as@gmail.com>
Co-authored-by: Mengjin Yan <mengjinyan3@gmail.com>
Signed-off-by: 400Ping <jiekaichang@apache.org>
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.

8 participants