Skip to content

Conversation

axreldable
Copy link
Contributor

@axreldable axreldable commented Sep 13, 2025

Why are these changes needed?

Global Serve configuration parameters such as proxy_location and http_options are not supposed to be updated via the Serve API: serve deploy and REST API PUT "/api/serve/applications/". When users try to update them, REST API PUT "/api/serve/applications/" ignores the change with the warning in the /tmp/ray/session_latest/logs/dashboard_ServeHead.log file. The warning has the following format:

2025-09-14 17:06:04,061 WARNING serve_head.py:177 -- Serve is already running on this Ray cluster and it's not possible to update its HTTP options without restarting it. Following options are attempted to be updated: ['location', 'host', 'port'].

While serve deploy silently ignores the changes, which may cause confusion for users.

This PR introduces:

  • change warning to RayServeConfigException in the _check_http_options function in serve._private.api
  • eliminate validate_http_options function in the dashboard.modules.serve.serve_head as it's not needed anymore
  • fix bug in serve.api of using wrong default value for DeploymentMode in case of empty proxy_location and non empty http_options (old implementation fell back to HeadOnly instead of the current default - EveryNode)
  • add note to Proxy config doc that the parameter is global and can't be updated at runtime.

User scenario:

  • have a file hello_world.py:
# hello_world.py
from ray.serve import deployment

@deployment
async def hello_world():
    return "Hello, world!"

hello_world_app = hello_world.bind()
  • execute commands:
ray stop
ray start --head
serve build -o config.yaml hello_world:hello_world_app  # generate `config.yaml`
serve deploy config.yaml
# update proxy_location: EveryNode -> HeadOnly and host and port in the `config.yaml` file
serve deploy config.yaml

Output before the change:

# stdout:
bash$ serve deploy config.yaml
2025-09-14 17:19:15,606 INFO scripts.py:239 -- Deploying from config file: 'config.yaml'.
2025-09-14 17:19:15,619 SUCC scripts.py:359 -- 
Sent deploy request successfully.
 * Use `serve status` to check applications' statuses.
 * Use `serve config` to see the current application config(s).

# /tmp/ray/session_latest/logs/dashboard_ServeHead.log
2025-09-14 17:19:15,615 WARNING serve_head.py:177 -- Serve is already running on this Ray cluster and it's not possible to update its HTTP options without restarting it. Following options are attempted to be updated: ['location', 'host', 'port'].

Output after the change:

# stdout:
bash$ serve deploy config.yaml
2025-09-25 20:40:59,614 INFO scripts.py:239 -- Deploying from config file: 'config.yaml'.
Traceback (most recent call last):
  File "~/ray/.venv/bin/serve", line 7, in <module>
    sys.exit(cli())
  File "~/ray/.venv/lib/python3.9/site-packages/click/core.py", line 1157, in __call__
    return self.main(*args, **kwargs)
  File "~/ray/.venv/lib/python3.9/site-packages/click/core.py", line 1078, in main
    rv = self.invoke(ctx)
  File "~/ray/.venv/lib/python3.9/site-packages/click/core.py", line 1688, in invoke
    return _process_result(sub_ctx.command.invoke(sub_ctx))
  File "~/ray/.venv/lib/python3.9/site-packages/click/core.py", line 1434, in invoke
    return ctx.invoke(self.callback, **ctx.params)
  File "~/ray/.venv/lib/python3.9/site-packages/click/core.py", line 783, in invoke
    return __callback(*args, **kwargs)
  File "~/ray/python/ray/serve/scripts.py", line 356, in deploy
    ServeSubmissionClient(address).deploy_applications(
  File "~/ray/python/ray/dashboard/modules/serve/sdk.py", line 80, in deploy_applications
    self._raise_error(response)
  File "~/ray/python/ray/dashboard/modules/dashboard_sdk.py", line 283, in _raise_error
    raise RuntimeError(
RuntimeError: Request failed with status code 500: Traceback (most recent call last):
  File "~/ray/python/ray/dashboard/optional_utils.py", line 187, in decorator
    return await f(self, *args, **kwargs)
  File "~/ray/python/ray/dashboard/modules/serve/serve_head.py", line 36, in check
    return await func(self, *args, **kwargs)
  File "~/ray/python/ray/dashboard/modules/serve/serve_head.py", line 144, in put_all_applications
    client = await serve_start_async(
  File "~/ray/python/ray/serve/_private/api.py", line 148, in serve_start_async
    _check_http_options(client, http_options)
  File "~/ray/python/ray/serve/_private/api.py", line 51, in _check_http_options
    raise RayServeConfigException(
ray.serve.exceptions.RayServeConfigException: Attempt to update `http_options` or `proxy_location` has been detected! Attempted updates: {'host': {'previous': '0.0.0.0', 'new': '127.0.0.1'}, 'port': {'previous': 8000, 'new': 8001}, 'location': {'previous': <ProxyLocation.EveryNode: 'EveryNode'>, 'new': <ProxyLocation.HeadOnly: 'HeadOnly'>}}. HTTP config is global to your Ray cluster, and you can't update it during runtime. Please restart Ray Serve to apply the change.

The same behavior for serve REST API:

bash$ curl -X PUT "http://localhost:8265/api/serve/applications/"   -H "Accept: application/json"   -H "Content-Type: application/json"   -d '{
>     "applications": [
>       {
>         "name": "app1",
>         "route_prefix": "/",
>         "import_path": "hello_world:hello_world_app",
>         "runtime_env": {},
>         "deployments": [
>           { "name": "hello_world" }
>         ]
>       }
>     ]
>   }'
bash$ serve config
name: app1
route_prefix: /
import_path: hello_world:hello_world_app
runtime_env: {}
deployments:
- name: hello_world
bash$ curl -X PUT "http://localhost:8265/api/serve/applications/"   -H "Accept: application/json"   -H "Content-Type: application/json"   -d '{
>     "proxy_location": "EveryNode",
>     "http_options": {
>         "host": "127.0.0.1",
>         "port": 8001,
>         "root_path": "",
>         "request_timeout_s": null,
>         "keep_alive_timeout_s": 5
>     },
>     "applications": [
>       {
>         "name": "app1",
>         "route_prefix": "/",
>         "import_path": "hello_world:hello_world_app",
>         "runtime_env": {},
>         "deployments": [
>           { "name": "hello_world" }
>         ]
>       }
>     ]
> }'
Traceback (most recent call last):
  File "~/ray/python/ray/dashboard/optional_utils.py", line 187, in decorator
    return await f(self, *args, **kwargs)
  File "~/ray/python/ray/dashboard/modules/serve/serve_head.py", line 36, in check
    return await func(self, *args, **kwargs)
  File "~/ray/python/ray/dashboard/modules/serve/serve_head.py", line 144, in put_all_applications
    client = await serve_start_async(
  File "~/ray/python/ray/serve/_private/api.py", line 148, in serve_start_async
    _check_http_options(client, http_options)
  File "~/ray/python/ray/serve/_private/api.py", line 51, in _check_http_options
    raise RayServeConfigException(
ray.serve.exceptions.RayServeConfigException: Attempt to update `http_options` or `proxy_location` has been detected! Attempted updates: {'host': {'previous': '0.0.0.0', 'new': '127.0.0.1'}, 'port': {'previous': 8000, 'new': 8001}, 'keep_alive_timeout_s': {'previous': 90, 'new': 5}}. HTTP config is global to your Ray cluster, and you can't update it during runtime. Please restart Ray Serve to apply the change.

A thing worth to mention here is this change makes explicit the discrepancy between default host value in serve.config.HTTPOptions (host="127.0.0.1") vs serve.schema.HTTPOptionsSchema (host="0.0.0.0"). serve.config.HTTPOptions is primarily used in serve start CLI and Python APIs and serve.schema.HTTPOptionsSchema is used in deploy/run/build APIs.
Previously, when users use default start and then deploy or run - http_options from start were used.
Now we explicitly failing in this scenario:

ray stop
ray start --head
serve build -o config.yaml hello_world:hello_world_app  # generate `config.yaml` with default values

serve start
serve deploy config.yaml
...
ray.serve.exceptions.RayServeConfigException: Attempt to update `http_options` or `proxy_location` has been detected! Attempted updates: {'host': {'previous': '127.0.0.1', 'new': '0.0.0.0'}}. HTTP config is global to your Ray cluster, and you can't update it during runtime. Please restart Ray Serve to apply the change.

This behavior can be annoying for users, but now it's explicit. Maybe we can consider to align default values in HTTPOptions and HTTPOptionsSchema.

Related issue number

Closes #56163

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

Disallow runtime updates to global http_options/proxy_location by raising RayServeConfigException, fix default proxy location handling, refactor validation, and update docs/tests/CLI accordingly.

  • Serve API (backend)
    • Raise RayServeConfigException on attempts to change global http_options/proxy_location via _check_http_options (now compares against client.http_config, normalizes ProxyLocation/DeploymentMode).
    • Fix default proxy location handling: _prepare_http_options sets DeploymentMode.EveryNode when proxy_location=None; start() uses it.
    • Adjust serve_start/serve_start_async to pass client.http_config into _check_http_options.
    • Add RayServeConfigException in ray.serve.exceptions.
  • Dashboard REST
    • Remove validate_http_options warning logic from serve_head.py; rely on backend check.
    • PUT with changed HTTP/proxy config now fails (500) instead of warning.
  • CLI/Tests
    • Update tests to expect failure on config changes and to use explicit HTTP host where needed.
    • Add tests for _prepare_http_options and serve.start rejection on changed HTTP config; add CLI test verifying detailed diff in error.
  • Docs
    • Note that proxy_location and HTTP/gRPC configs are cluster-global and cannot be updated at runtime.

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

@axreldable
Copy link
Contributor Author

/gemini review

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 introduces valuable validation for immutable configuration parameters (proxy_location and http_options) within the serve deploy command. By failing fast with a clear error message, it significantly improves the user experience over the previous behavior of silently ignoring changes. The implementation is straightforward, and the new logic is accompanied by a solid set of unit tests. I have a couple of suggestions to enhance the code's conciseness and test coverage.

@axreldable
Copy link
Contributor Author

/gemini review

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 introduces validation to prevent runtime updates to immutable Serve configurations like proxy_location and http_options via the serve deploy CLI. This is a great improvement as it provides explicit and fast feedback to the user. The new validation logic in _validate_deployment_config is clear and correct. The accompanying tests are comprehensive, covering various scenarios including successful updates, and failures for both proxy_location and http_options changes. I've added a couple of minor suggestions to improve the readability of the error messages.

@axreldable
Copy link
Contributor Author

/gemini review

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 introduces important validation for proxy_location and http_options in the serve deploy command, preventing runtime changes to these immutable configurations. The implementation is solid, with a new validation function that fails fast and provides clear error messages. The changes are well-tested with both unit and integration-style tests. I have one suggestion to improve the test code's maintainability by marking helper methods as static.

@axreldable
Copy link
Contributor Author

/gemini review

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 introduces an important validation step in the serve deploy command to prevent runtime updates to immutable global configurations like proxy_location and http_options. By raising an exception, it provides clear, immediate feedback to the user, which is a significant improvement over the previous silent failure. The implementation is well-structured, and the accompanying tests are thorough. I have one suggestion to improve the consistency of the validation logic for proxy_location to better align with the handling of http_options, making the behavior more intuitive for users.

@axreldable
Copy link
Contributor Author

/gemini review

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 introduces important validation to the serve deploy command, preventing users from attempting to update immutable global configurations like proxy_location and http_options at runtime. Instead of silently ignoring these changes, Serve will now raise a RayServeException with a clear error message, which significantly improves the user experience and reduces confusion. The implementation is solid, with a new validation function that correctly compares the new configuration against the running one. The accompanying documentation update and comprehensive unit tests are also well-executed. I have a minor suggestion to improve consistency in an error message.

@axreldable
Copy link
Contributor Author

/gemini review

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 introduces important validation to prevent runtime updates of immutable Serve configurations like proxy_location and http_options via serve deploy. The changes provide clear, immediate feedback to the user by raising an exception, which is a significant improvement over the previous silent failure. The implementation is solid, and the addition of unit tests for the new validation logic is great. I've suggested a couple of improvements to the tests to ensure full coverage for the new http_options validation at the command level. Overall, this is a valuable change that will improve user experience and prevent confusion.

@axreldable
Copy link
Contributor Author

/gemini review

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 introduces an important validation step in the serve deploy command. It prevents users from attempting to update immutable global configurations like proxy_location and http_options at runtime, which previously failed silently. Instead, it now raises a RayServeException with a clear error message. The changes include a new validation function, its integration into the deployment workflow, and comprehensive unit tests. Additionally, the documentation has been updated to reflect this limitation. My review focuses on improving the implementation of the validation logic for better code consistency.

@axreldable
Copy link
Contributor Author

/gemini review

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 introduces important validation to prevent runtime updates of global Serve configuration parameters like proxy_location and http_options via serve deploy. The change from silently ignoring these updates to raising a RayServeException is a significant improvement for user experience, providing clear and immediate feedback. The accompanying documentation update and comprehensive test suite are well-executed. I've included one minor suggestion to refactor the validation logic for better consistency and efficiency.

@axreldable axreldable marked this pull request as ready for review September 15, 2025 18:44
@axreldable axreldable requested review from a team as code owners September 15, 2025 18:44
@axreldable
Copy link
Contributor Author

Hi @abrarsheikh!
Could you please check this?

@ray-gardener ray-gardener bot added serve Ray Serve Related Issue docs An issue or change related to documentation community-contribution Contributed by the community labels Sep 15, 2025
Comment on lines 417 to 421
submission_client = ServeSubmissionClient(address)
serve_details = ServeInstanceDetails(**submission_client.get_serve_details())
_validate_deployment_config(serve_details, config)

submission_client.deploy_applications(
Copy link
Contributor

Choose a reason for hiding this comment

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

if we make change at this layer, then still PUT API call will be ignoring the warning errors(this change works for serve deploy only), shouldn't we put this validation behind the API? as soon as we intercept the PUT request

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Now there are 2 ways of updating deployment. serve deploy cli command and PUT "/api/serve/applications/" REST API (both ways eventually go to serve_head) where warning is logged in case of changes in http_options (which includes proxy_location there). This warning is logged to /tmp/ray/session_latest/logs/dashboard_ServeHead.log.

Do you think I need to replace that warning with error? Could you please let me know what would be the right place to add this check?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

if we make change at this layer, then still PUT API call will be ignoring the warning errors(this change works for serve deploy only), shouldn't we put this validation behind the API? as soon as we intercept the PUT request

@harshit-anyscale , do you think I can add the check to the controller after this line to compare the deploying config with the curr_config from checkpoint ? Then, we'll fail for both cli and rest APIs. Will it be the proper place for this check?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Hi @harshit-anyscale! I checked that and looks like I can introduce this check in 2 places:

  1. ray/python/ray/dashboard/modules/serve/serve_head.py. I can fail here instead of warning. Maybe it makes more sense, but it will be in the dashboard module.
  2. python/ray/serve/_private/controller.py or I can add the check here in apply_config in controller as I mentioned in my previous comment.
    I prefer the option 1 as it fails earlier, but not 100% sure if the dashboard module is the proper place to control it.
    Please let me know what you think.

Copy link
Contributor

Choose a reason for hiding this comment

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

first option seems appropriate, as we are already performing other validations there
@abrarsheikh let us know if you think otherwise

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Thank you, Harshit! I ended up changing warning to error in the serve._private.api on the serve side. I'll resolve conflicts, fix build, and ask for review.

Copy link
Contributor Author

@axreldable axreldable Sep 29, 2025

Choose a reason for hiding this comment

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

Hi @harshit-anyscale , @abrarsheikh !

It turns out that it's a bit trickier than I thought. There are 3 places where users can set/update http_options now:

  • Python API
  • CLI
  • REST API
Python API:
serve:
- start -> _private_api.serve_start
- run -> _private_api.serve_start

Command Line Interface (CLI)
serve start -> serve.start
serve deploy -> REST API PUT "/api/serve/applications/" 
serve run -> _private_api.serve_start

Serve REST API
PUT "/api/serve/applications/" -> _private_api.serve_start_async

All of those methods end up in the _private_api.serve_start(s) methods and I believe the _check_http_options(client, http_options) function is the correct place to introduce this change.

The problem now is that, it's challenging to distinguish what http_options were entered by users and what not (across of all of the APIs). I'll try to refactor the API a bit to be able to do that and verify the difference properly.

But while I was looking into it, I identified a bug and a discrepancy/bug which relate to proxy_location parameter. It would be good to resolve them first to reduce the complexity of this change. I opened a separate PR #56993 for it.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

The fix of the bug and discrepancy mentioned in the previous comment is ready for review in the #57622.

@axreldable
Copy link
Contributor Author

/gemini review

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 introduces important validation to prevent runtime updates of proxy_location and http_options via serve deploy, which addresses a source of user confusion. The implementation is solid, with a new validation function that provides clear, actionable error messages. The accompanying tests are thorough, and the documentation update is a welcome addition. I have a few suggestions to enhance code readability and improve the clarity of the tests.

@axreldable
Copy link
Contributor Author

Hi @harshit-anyscale , I addressed comments and asked a question. Could you kindly take a look?

cursor[bot]

This comment was marked as outdated.

@axreldable axreldable changed the title [serve] Add validation for 'proxy_location' and 'http_options' config parameters for the 'serve deploy' command [serve] Fail on the change of 'proxy_location' or 'http_options' parameters for the 'serve' API Sep 25, 2025
@axreldable axreldable requested a review from a team as a code owner September 25, 2025 19:17
… parameters for the 'serve deploy' command

Signed-off-by: axreldable <aleksei.starikov.ax@gmail.com>
…ameters for the 'serve' API

Signed-off-by: axreldable <aleksei.starikov.ax@gmail.com>
@axreldable
Copy link
Contributor Author

/gemini review

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 is a great improvement for user experience. It changes the behavior of Ray Serve to fail explicitly with a RayServeConfigException when a user attempts to modify global configurations like proxy_location or http_options at runtime, instead of silently ignoring the changes. This prevents confusion and makes the system's behavior more predictable. The changes are well-implemented across the API, dashboard, and CLI, and are accompanied by corresponding documentation and test updates. My review includes a suggestion to further improve the readability of the new exception's error message and a minor cleanup in one of the new tests.

Comment on lines 51 to 56
raise RayServeConfigException(
"Attempt to update `http_options` or `proxy_location` has been detected! "
f"Attempted updates: {diff_http_options}. "
"HTTP config is global to your Ray cluster, and you can't update it during runtime. "
"Please restart Ray Serve to apply the change."
)
Copy link
Contributor

Choose a reason for hiding this comment

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

medium

The current error message for RayServeConfigException includes the repr() of ProxyLocation enum, which is a bit verbose (e.g., <ProxyLocation.EveryNode: 'EveryNode'>). We can format the dictionary of differences to use the enum's string value for a cleaner and more readable error message. This will also require updating the corresponding test assertion in python/ray/serve/tests/test_cli.py.

            from enum import Enum

            # Format the diff for a cleaner error message.
            # Enums are converted to their string values.
            formatted_diff = {}
            for field, diff in diff_http_options.items():
                prev = diff["previous"]
                new = diff["new"]
                formatted_diff[field] = {
                    "previous": prev.value if isinstance(prev, Enum) else prev,
                    "new": new.value if isinstance(new, Enum) else new,
                }

            raise RayServeConfigException(
                "Attempt to update `http_options` or `proxy_location` has been detected! "
                f"Attempted updates: {formatted_diff}. "
                "HTTP config is global to your Ray cluster, and you can't update it during runtime. "
                "Please restart Ray Serve to apply the change."
            )

Comment on lines 302 to 307
assert (
"Attempted updates: {"
"'host': {'previous': '0.0.0.0', 'new': '0.0.0.1'}, "
"'port': {'previous': 8000, 'new': 8001}, "
"'location': {'previous': <ProxyLocation.EveryNode: 'EveryNode'>, 'new': <ProxyLocation.HeadOnly: 'HeadOnly'>}}"
) in error
Copy link
Contributor

Choose a reason for hiding this comment

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

medium

This assertion checks for the specific format of the error message. To accompany the suggested improvement to the error message format in _private/api.py, this test assertion should be updated to match the new, more readable format that uses string values for enums.

Suggested change
assert (
"Attempted updates: {"
"'host': {'previous': '0.0.0.0', 'new': '0.0.0.1'}, "
"'port': {'previous': 8000, 'new': 8001}, "
"'location': {'previous': <ProxyLocation.EveryNode: 'EveryNode'>, 'new': <ProxyLocation.HeadOnly: 'HeadOnly'>}}"
) in error
assert (
"Attempted updates: {"
"'host': {'previous': '0.0.0.0', 'new': '0.0.0.1'}, "
"'port': {'previous': 8000, 'new': 8001}, "
"'location': {'previous': 'EveryNode', 'new': 'HeadOnly'}}"
) in error

…meters for the 'serve' API

Signed-off-by: axreldable <aleksei.starikov.ax@gmail.com>
…meters for the 'serve' API

Signed-off-by: axreldable <aleksei.starikov.ax@gmail.com>
Copy link
Collaborator

@jjyao jjyao left a comment

Choose a reason for hiding this comment

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

CORE stamp.

@axreldable
Copy link
Contributor Author

CORE stamp.

Hi @jjyao ! Thank the approval. This PR is still WIP. Could you please clarify what CORE stamp. means?

@axreldable axreldable changed the title [serve] Fail on the change of 'proxy_location' or 'http_options' parameters for the 'serve' API [WIP][serve] Fail on the change of 'proxy_location' or 'http_options' parameters for the 'serve' API Sep 29, 2025
@jjyao
Copy link
Collaborator

jjyao commented Oct 2, 2025

Thank the approval. This PR is still WIP. Could you please clarify what CORE stamp. means?

Means core side change looks good and approved.

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 docs An issue or change related to documentation serve Ray Serve Related Issue

Projects

None yet

Development

Successfully merging this pull request may close these issues.

[Serve] proxy_location change is ignored on redeploy unless Serve/cluster is restarted

3 participants