From fa2cf172bff2a713a63e255280e892368d740925 Mon Sep 17 00:00:00 2001 From: axreldable Date: Sat, 13 Sep 2025 19:44:19 +0200 Subject: [PATCH 01/12] [serve] Fail on the change of 'proxy_location' and 'http_options' parameters for the 'serve' API Signed-off-by: axreldable --- doc/source/serve/production-guide/config.md | 2 + .../ray/dashboard/modules/serve/serve_head.py | 21 -------- .../serve/tests/test_serve_dashboard_2.py | 9 ++-- python/ray/serve/_private/api.py | 52 +++++++++++-------- python/ray/serve/exceptions.py | 5 ++ python/ray/serve/tests/test_api_2.py | 40 ++++++++++++++ python/ray/serve/tests/test_cli.py | 34 ++++++++++++ python/ray/serve/tests/test_cli_2.py | 6 +-- 8 files changed, 117 insertions(+), 52 deletions(-) diff --git a/doc/source/serve/production-guide/config.md b/doc/source/serve/production-guide/config.md index fa5eff346fe5..fe385f2c3697 100644 --- a/doc/source/serve/production-guide/config.md +++ b/doc/source/serve/production-guide/config.md @@ -60,6 +60,8 @@ The `proxy_location` field configures where to run proxies to handle traffic to - HeadOnly: Only run a single proxy on the head node. - Disabled: Don't run proxies at all. Set this value if you are only making calls to your applications using deployment handles. +Note that the Proxy config is global to your Ray cluster, and you can't update it during runtime. + (http-config)= ## HTTP config diff --git a/python/ray/dashboard/modules/serve/serve_head.py b/python/ray/dashboard/modules/serve/serve_head.py index 205648d74dc3..21851f4de4b7 100644 --- a/python/ray/dashboard/modules/serve/serve_head.py +++ b/python/ray/dashboard/modules/serve/serve_head.py @@ -170,11 +170,6 @@ async def put_all_applications(self, req: Request) -> Response: global_logging_config=config.logging_config, ) - # Serve ignores HTTP options if it was already running when - # serve_start_async() is called. Therefore we validate that no - # existing HTTP options are updated and print warning in case they are - self.validate_http_options(client, full_http_options) - try: if config.logging_config: client.update_global_logging_config(config.logging_config) @@ -269,22 +264,6 @@ async def scale_deployment(self, req: Request) -> Response: {"error": "Internal Server Error"}, 503 ) - def validate_http_options(self, client, http_options): - divergent_http_options = [] - - for option, new_value in http_options.items(): - prev_value = getattr(client.http_config, option) - if prev_value != new_value: - divergent_http_options.append(option) - - if divergent_http_options: - logger.warning( - "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 " - f"updated: {divergent_http_options}." - ) - async def get_serve_controller(self): """Gets the ServeController to the this cluster's Serve app. diff --git a/python/ray/dashboard/modules/serve/tests/test_serve_dashboard_2.py b/python/ray/dashboard/modules/serve/tests/test_serve_dashboard_2.py index a21cad86ad32..52ce2a16563c 100644 --- a/python/ray/dashboard/modules/serve/tests/test_serve_dashboard_2.py +++ b/python/ray/dashboard/modules/serve/tests/test_serve_dashboard_2.py @@ -76,11 +76,10 @@ def test_serve_namespace(ray_start_stop): def test_put_with_http_options(ray_start_stop, option, override): """Submits a config with HTTP options specified. - Trying to submit a config to the serve agent with the HTTP options modified should - NOT fail: + Trying to submit a config to the serve agent with the HTTP options modified should fail: - If Serve is NOT running, HTTP options will be honored when starting Serve - - If Serve is running, HTTP options will be ignored, and warning will be logged - urging users to restart Serve if they want their options to take effect + - If Serve is running and HTTP options are attempted to be changed, deployment will fail, + forcing users to restart Serve if they want their options to take effect. """ pizza_import_path = "ray.serve.tests.test_config_files.pizza.serve_dag" @@ -126,7 +125,7 @@ def test_put_with_http_options(ray_start_stop, option, override): put_response = requests.put( SERVE_HEAD_URL, json=updated_serve_config_json, timeout=5 ) - assert put_response.status_code == 200 + assert put_response.status_code == 500 # Fetch Serve status and confirm that HTTP options are unchanged get_response = requests.get(SERVE_HEAD_URL, timeout=5) diff --git a/python/ray/serve/_private/api.py b/python/ray/serve/_private/api.py index 60b097d4c001..69b5c96ff959 100644 --- a/python/ray/serve/_private/api.py +++ b/python/ray/serve/_private/api.py @@ -14,37 +14,43 @@ SERVE_NAMESPACE, ) from ray.serve._private.default_impl import get_controller_impl -from ray.serve.config import HTTPOptions, gRPCOptions +from ray.serve.config import DeploymentMode, HTTPOptions, ProxyLocation, gRPCOptions from ray.serve.context import _get_global_client, _set_global_client from ray.serve.deployment import Application -from ray.serve.exceptions import RayServeException +from ray.serve.exceptions import RayServeConfigException, RayServeException from ray.serve.schema import LoggingConfig logger = logging.getLogger(SERVE_LOGGER_NAME) def _check_http_options( - client: ServeControllerClient, http_options: Union[dict, HTTPOptions] + curr_http_options: HTTPOptions, new_http_options: Union[dict, HTTPOptions] ) -> None: - if http_options: - client_http_options = client.http_config - new_http_options = ( - http_options - if isinstance(http_options, HTTPOptions) - else HTTPOptions.parse_obj(http_options) + def maybe_restore_proxy_location(prev_value, new_value) -> (str, str): + if isinstance(prev_value, DeploymentMode) or isinstance( + new_value, DeploymentMode + ): + # restore ProxyLocation as this is the property user configured + prev_value = ProxyLocation._from_deployment_mode(prev_value).value + new_value = ProxyLocation._from_deployment_mode(new_value).value + return prev_value, new_value + + if isinstance(new_http_options, HTTPOptions): + new_http_options = new_http_options.dict(exclude_unset=True) + + diff_http_options = {} + for option, new_value in new_http_options.items(): + prev_value = getattr(curr_http_options, option) + if prev_value != new_value: + prev_value, new_value = maybe_restore_proxy_location(prev_value, new_value) + diff_http_options[option] = {"previous": prev_value, "new": new_value} + if diff_http_options: + 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." ) - different_fields = [] - all_http_option_fields = new_http_options.__dict__ - for field in all_http_option_fields: - if getattr(new_http_options, field) != getattr(client_http_options, field): - different_fields.append(field) - - if len(different_fields): - logger.warning( - "The new client HTTP config differs from the existing one " - f"in the following fields: {different_fields}. " - "The new HTTP config is ignored." - ) def _start_controller( @@ -136,7 +142,7 @@ async def serve_start_async( " New http options will not be applied." ) if http_options: - _check_http_options(client, http_options) + _check_http_options(client.http_config, http_options) return client except RayServeException: pass @@ -210,7 +216,7 @@ def serve_start( " New http options will not be applied." ) if http_options: - _check_http_options(client, http_options) + _check_http_options(client.http_config, http_options) return client except RayServeException: pass diff --git a/python/ray/serve/exceptions.py b/python/ray/serve/exceptions.py index 10033b28f9cd..e96519bbc059 100644 --- a/python/ray/serve/exceptions.py +++ b/python/ray/serve/exceptions.py @@ -10,6 +10,11 @@ class RayServeException(Exception): pass +@PublicAPI(stability="alpha") +class RayServeConfigException(Exception): + pass + + @PublicAPI(stability="alpha") class BackPressureError(RayServeException): """Raised when max_queued_requests is exceeded on a DeploymentHandle.""" diff --git a/python/ray/serve/tests/test_api_2.py b/python/ray/serve/tests/test_api_2.py index 65fd4c63e28e..80849bc34b87 100644 --- a/python/ray/serve/tests/test_api_2.py +++ b/python/ray/serve/tests/test_api_2.py @@ -5,6 +5,9 @@ from ray._common.network_utils import build_address from ray.serve._private.common import RequestProtocol from ray.serve._private.test_utils import get_application_urls +from ray.serve.api import _prepare_http_options +from ray.serve.config import DeploymentMode, ProxyLocation +from ray.serve.exceptions import RayServeConfigException def test_get_application_urls(serve_instance): @@ -61,6 +64,43 @@ def f(): ] +def test_serve_start(serve_instance): + serve.start() # success with default params + with pytest.raises( + RayServeConfigException, + match=r"{'host': {'previous': '0.0.0.0', 'new': '127.0.0.1'}}", + ): + serve.start(http_options={"host": "127.0.0.1"}) + + +class TestPrepHttpOptions: + def test_none_http_options_and_none_proxy_location(self): + http_options = _prepare_http_options(proxy_location=None, http_options=None) + assert http_options.location == DeploymentMode.EveryNode + assert http_options.host == "127.0.0.1" + + def test_none_proxy_location(self): + http_options = _prepare_http_options( + proxy_location=None, http_options={"host": "0.0.0.0"} + ) + assert http_options.location == DeploymentMode.EveryNode + assert http_options.host == "0.0.0.0" + + def test_proxy_location_from_user_set(self): + http_options = _prepare_http_options(proxy_location=None, http_options=None) + assert http_options.location == DeploymentMode.EveryNode + + http_options = _prepare_http_options( + proxy_location="HeadOnly", http_options=None + ) + assert http_options.location == DeploymentMode.HeadOnly + + http_options = _prepare_http_options( + proxy_location=ProxyLocation.Disabled, http_options=None + ) + assert http_options.location == DeploymentMode.NoServer + + if __name__ == "__main__": import sys diff --git a/python/ray/serve/tests/test_cli.py b/python/ray/serve/tests/test_cli.py index 0c57e0197942..6a0f1272540d 100644 --- a/python/ray/serve/tests/test_cli.py +++ b/python/ray/serve/tests/test_cli.py @@ -275,6 +275,40 @@ def test_deploy_bad_v2_config(serve_instance): assert "Please ensure each application's route_prefix is unique" in output +@pytest.mark.skipif(sys.platform == "win32", reason="File path incorrect on Windows.") +def test_deploy_with_different_http_options(serve_instance, tmp_path): + config_path = tmp_path / "config.yaml" + config = { + "proxy_location": "HeadOnly", + "http_options": { + "host": "0.0.0.1", + "port": 8001, + }, + "applications": [], + } + with open(config_path, "w") as f: + yaml.dump(config, f) + + with pytest.raises(subprocess.CalledProcessError) as e: + subprocess.check_output( + ["serve", "deploy", str(config_path)], stderr=subprocess.STDOUT + ) + error = e.value.output.decode("utf-8") + assert "RayServeConfigException" in error + assert ( + "Attempt to update `http_options` or `proxy_location` has been detected!" + in error + ) + assert "'host': {'previous': '0.0.0.0', 'new': '0.0.0.1'}" in error + assert "'port': {'previous': 8000, 'new': 8001}" in error + assert "'location': {'previous': 'EveryNode', 'new': 'HeadOnly'}" in error + assert ( + "HTTP config is global to your Ray cluster, and you can't update it during runtime." + in error + ) + assert "Please restart Ray Serve to apply the change." in error + + @pytest.mark.skipif(sys.platform == "win32", reason="File path incorrect on Windows.") def test_deploy_multi_app_builder_with_args(serve_instance): """Deploys a config file containing multiple applications that take arguments.""" diff --git a/python/ray/serve/tests/test_cli_2.py b/python/ray/serve/tests/test_cli_2.py index df267c69d8d0..9c806cfac30c 100644 --- a/python/ray/serve/tests/test_cli_2.py +++ b/python/ray/serve/tests/test_cli_2.py @@ -58,7 +58,7 @@ def check_http_response(expected_text: str, json: Optional[Dict] = None): def test_start_shutdown(ray_start_stop): - subprocess.check_output(["serve", "start"]) + subprocess.check_output(["serve", "start", "--http-host", "0.0.0.0"]) # deploy a simple app import_path = "ray.serve.tests.test_config_files.arg_builders.build_echo_app" @@ -253,7 +253,7 @@ def test_idempotence_after_controller_death(ray_start_stop, use_command: bool): deploy_response = subprocess.check_output(["serve", "deploy", config_file_name]) assert success_message_fragment in deploy_response - serve.start() + serve.start(http_options={"host": "0.0.0.0"}) wait_for_condition( lambda: len(list_actors(filters=[("state", "=", "ALIVE")])) == 4, timeout=15, @@ -274,7 +274,7 @@ def test_idempotence_after_controller_death(ray_start_stop, use_command: bool): assert success_message_fragment in deploy_response # Restore testing controller - serve.start() + serve.start(http_options={"host": "0.0.0.0"}) wait_for_condition( lambda: len(list_actors(filters=[("state", "=", "ALIVE")])) == 4, timeout=15, From 30a9c201335ec9417a3b322bef1ebbde38f0a641 Mon Sep 17 00:00:00 2001 From: axreldable Date: Thu, 30 Oct 2025 18:08:07 +0100 Subject: [PATCH 02/12] [serve] Fix test: remove serve.start because of different default host in HTTPOptions and HTTPOptionsSchema Signed-off-by: axreldable --- .../ray/dashboard/modules/serve/tests/test_serve_dashboard.py | 2 -- 1 file changed, 2 deletions(-) diff --git a/python/ray/dashboard/modules/serve/tests/test_serve_dashboard.py b/python/ray/dashboard/modules/serve/tests/test_serve_dashboard.py index 47be34577b47..05a9cb13eb91 100644 --- a/python/ray/dashboard/modules/serve/tests/test_serve_dashboard.py +++ b/python/ray/dashboard/modules/serve/tests/test_serve_dashboard.py @@ -887,8 +887,6 @@ def test_scale_deployment_retention_during_serve_controller_restart( self, ray_start_stop ): """Test that replica counts set via /scale are retained after serve controller restart.""" - serve.start() - with tempfile.TemporaryDirectory() as tmpdir: tmp_path = Path(tmpdir) From d55b10e3922d245b79502a6f637ccce7d87de5a4 Mon Sep 17 00:00:00 2001 From: axreldable Date: Thu, 30 Oct 2025 19:48:07 +0100 Subject: [PATCH 03/12] [serve] Change default HTTPOptions.location HeadOnly -> EveryNode Signed-off-by: axreldable --- python/ray/serve/api.py | 1 - python/ray/serve/config.py | 2 +- python/ray/serve/tests/conftest.py | 3 +-- python/ray/serve/tests/test_controller.py | 2 +- python/ray/serve/tests/test_standalone.py | 16 +++++++--------- 5 files changed, 10 insertions(+), 14 deletions(-) diff --git a/python/ray/serve/api.py b/python/ray/serve/api.py index 30560f99502a..8e83e225ed14 100644 --- a/python/ray/serve/api.py +++ b/python/ray/serve/api.py @@ -588,7 +588,6 @@ def _run_many( return [b.deployment_handles[b.ingress_deployment_name] for b in built_apps] else: client = _private_api.serve_start( - http_options={"location": "EveryNode"}, global_logging_config=None, ) diff --git a/python/ray/serve/config.py b/python/ray/serve/config.py index 6aa7ac54317b..f6b664189f89 100644 --- a/python/ray/serve/config.py +++ b/python/ray/serve/config.py @@ -584,7 +584,7 @@ class HTTPOptions(BaseModel): host: Optional[str] = DEFAULT_HTTP_HOST port: int = DEFAULT_HTTP_PORT middlewares: List[Any] = [] - location: Optional[DeploymentMode] = DeploymentMode.HeadOnly + location: Optional[DeploymentMode] = DeploymentMode.EveryNode num_cpus: int = 0 root_url: str = "" root_path: str = "" diff --git a/python/ray/serve/tests/conftest.py b/python/ray/serve/tests/conftest.py index 6bf5243eabb8..69e7392e4b91 100644 --- a/python/ray/serve/tests/conftest.py +++ b/python/ray/serve/tests/conftest.py @@ -21,7 +21,7 @@ check_ray_stopped, start_telemetry_app, ) -from ray.serve.config import HTTPOptions, ProxyLocation, gRPCOptions +from ray.serve.config import HTTPOptions, gRPCOptions from ray.serve.context import _get_global_client from ray.tests.conftest import ( # noqa external_redis, @@ -148,7 +148,6 @@ def _shared_serve_instance(): _system_config={"metrics_report_interval_ms": 1000, "task_retry_delay_ms": 50}, ) serve.start( - proxy_location=ProxyLocation.HeadOnly, http_options={"host": "0.0.0.0"}, grpc_options={ "port": 9000, diff --git a/python/ray/serve/tests/test_controller.py b/python/ray/serve/tests/test_controller.py index 01f410999930..df091a960cc9 100644 --- a/python/ray/serve/tests/test_controller.py +++ b/python/ray/serve/tests/test_controller.py @@ -124,7 +124,7 @@ def autoscaling_app(): "worker_id": controller_details.worker_id, "log_file_path": controller_details.log_file_path, }, - "proxy_location": "HeadOnly", + "proxy_location": "EveryNode", "http_options": {"host": "0.0.0.0"}, "grpc_options": { "port": 9000, diff --git a/python/ray/serve/tests/test_standalone.py b/python/ray/serve/tests/test_standalone.py index dfa84420c30f..989ed473d7a0 100644 --- a/python/ray/serve/tests/test_standalone.py +++ b/python/ray/serve/tests/test_standalone.py @@ -31,6 +31,7 @@ from ray.serve._private.utils import block_until_http_ready, format_actor_name from ray.serve.config import DeploymentMode, HTTPOptions, ProxyLocation from ray.serve.context import _get_global_client +from ray.serve.exceptions import RayServeConfigException from ray.serve.schema import ServeApplicationSchema, ServeDeploySchema from ray.util.state import list_actors @@ -706,7 +707,7 @@ class A: ) -def test_serve_start_different_http_checkpoint_options_warning( +def test_serve_start_different_http_checkpoint_options_error( ray_shutdown, propagate_logs, caplog ): logger = logging.getLogger("ray.serve") @@ -726,13 +727,10 @@ def emit(self, record): # create a different config test_http = dict(host="127.1.1.8", port=_get_random_port()) - serve.start(http_options=test_http) - - for test_config, msg in zip([["host", "port"]], warning_msg): - for test_msg in test_config: - if "Autoscaling metrics pusher thread" in msg: - continue - assert test_msg in msg + with pytest.raises( + RayServeConfigException, match="Attempt to update `http_options`" + ): + serve.start(http_options=test_http) def test_recovering_controller_no_redeploy(): @@ -900,7 +898,7 @@ def test_build_app_task_uses_zero_cpus(ray_shutdown): { "proxy_location": None, "http_options": HTTPOptions(), - "expected": HTTPOptions(location=DeploymentMode.HeadOnly), + "expected": HTTPOptions(location=DeploymentMode.EveryNode), }, # using default location from HTTPOptions { "proxy_location": None, From beb9b62c4c8532bf84ad645eb0e1373bb45ecee2 Mon Sep 17 00:00:00 2001 From: axreldable Date: Thu, 30 Oct 2025 20:54:17 +0100 Subject: [PATCH 04/12] [serve] Remove unused tests + change condition Signed-off-by: axreldable --- python/ray/serve/_private/api.py | 2 +- python/ray/serve/tests/test_api_2.py | 30 ---------------------------- 2 files changed, 1 insertion(+), 31 deletions(-) diff --git a/python/ray/serve/_private/api.py b/python/ray/serve/_private/api.py index 69b5c96ff959..98e15a5c01a4 100644 --- a/python/ray/serve/_private/api.py +++ b/python/ray/serve/_private/api.py @@ -27,7 +27,7 @@ def _check_http_options( curr_http_options: HTTPOptions, new_http_options: Union[dict, HTTPOptions] ) -> None: def maybe_restore_proxy_location(prev_value, new_value) -> (str, str): - if isinstance(prev_value, DeploymentMode) or isinstance( + if isinstance(prev_value, DeploymentMode) and isinstance( new_value, DeploymentMode ): # restore ProxyLocation as this is the property user configured diff --git a/python/ray/serve/tests/test_api_2.py b/python/ray/serve/tests/test_api_2.py index 80849bc34b87..b8b8665b32b5 100644 --- a/python/ray/serve/tests/test_api_2.py +++ b/python/ray/serve/tests/test_api_2.py @@ -5,8 +5,6 @@ from ray._common.network_utils import build_address from ray.serve._private.common import RequestProtocol from ray.serve._private.test_utils import get_application_urls -from ray.serve.api import _prepare_http_options -from ray.serve.config import DeploymentMode, ProxyLocation from ray.serve.exceptions import RayServeConfigException @@ -73,34 +71,6 @@ def test_serve_start(serve_instance): serve.start(http_options={"host": "127.0.0.1"}) -class TestPrepHttpOptions: - def test_none_http_options_and_none_proxy_location(self): - http_options = _prepare_http_options(proxy_location=None, http_options=None) - assert http_options.location == DeploymentMode.EveryNode - assert http_options.host == "127.0.0.1" - - def test_none_proxy_location(self): - http_options = _prepare_http_options( - proxy_location=None, http_options={"host": "0.0.0.0"} - ) - assert http_options.location == DeploymentMode.EveryNode - assert http_options.host == "0.0.0.0" - - def test_proxy_location_from_user_set(self): - http_options = _prepare_http_options(proxy_location=None, http_options=None) - assert http_options.location == DeploymentMode.EveryNode - - http_options = _prepare_http_options( - proxy_location="HeadOnly", http_options=None - ) - assert http_options.location == DeploymentMode.HeadOnly - - http_options = _prepare_http_options( - proxy_location=ProxyLocation.Disabled, http_options=None - ) - assert http_options.location == DeploymentMode.NoServer - - if __name__ == "__main__": import sys From bd398345e8c20ffe19d55f7140f80f015b29d8ef Mon Sep 17 00:00:00 2001 From: axreldable Date: Thu, 30 Oct 2025 21:42:38 +0100 Subject: [PATCH 05/12] [serve] Remove unused tests + change condition Signed-off-by: axreldable --- python/ray/serve/tests/unit/test_config.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/python/ray/serve/tests/unit/test_config.py b/python/ray/serve/tests/unit/test_config.py index 5f2d3902e2c6..ad4e8293e017 100644 --- a/python/ray/serve/tests/unit/test_config.py +++ b/python/ray/serve/tests/unit/test_config.py @@ -675,13 +675,13 @@ def test_prepare_imperative_http_options(): proxy_location=None, http_options=HTTPOptions(**{}), ) == HTTPOptions( - location=DeploymentMode.HeadOnly + location=DeploymentMode.EveryNode ) # in this case we can't know whether location was provided or not assert prepare_imperative_http_options( proxy_location=None, http_options=HTTPOptions(), - ) == HTTPOptions(location=DeploymentMode.HeadOnly) + ) == HTTPOptions(location=DeploymentMode.EveryNode) assert prepare_imperative_http_options( proxy_location=None, From dbd08257a9f96c55501aeebc86a54e78a1b4a1a3 Mon Sep 17 00:00:00 2001 From: axreldable Date: Fri, 31 Oct 2025 12:04:38 +0100 Subject: [PATCH 06/12] [serve] Use correct host in test Signed-off-by: axreldable --- python/ray/serve/tests/test_telemetry_1.py | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/python/ray/serve/tests/test_telemetry_1.py b/python/ray/serve/tests/test_telemetry_1.py index 6524e9679c9d..93203533dc56 100644 --- a/python/ray/serve/tests/test_telemetry_1.py +++ b/python/ray/serve/tests/test_telemetry_1.py @@ -105,13 +105,14 @@ def test_rest_api(manage_ray_with_telemetry, tmp_dir): check_telemetry(ServeUsageTag.REST_API_VERSION, expected=None) config = { + "http_options": {"host": "127.0.0.1"}, "applications": [ { "name": "stub_app", "import_path": "ray.serve.tests.test_telemetry_1.stub_app", "route_prefix": "/stub", }, - ] + ], } config_file_path = f"{tmp_dir}/config.yaml" with open(config_file_path, "w+") as f: @@ -149,7 +150,7 @@ def test_rest_api(manage_ray_with_telemetry, tmp_dir): assert ServeUsageTag.AUTO_NUM_REPLICAS_USED.get_value_from_report(report) is None # Check that app deletions are tracked. - new_config = {"applications": []} + new_config = {"http_options": {"host": "127.0.0.1"}, "applications": []} with open(config_file_path, "w+") as f: yaml.safe_dump(new_config, f) From 50dfc4557977f5324f33214cbfec0095f09f0175 Mon Sep 17 00:00:00 2001 From: axreldable Date: Fri, 31 Oct 2025 13:45:22 +0100 Subject: [PATCH 07/12] [serve] Use correct port in test Signed-off-by: axreldable --- python/ray/serve/tests/test_standalone.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/serve/tests/test_standalone.py b/python/ray/serve/tests/test_standalone.py index 989ed473d7a0..85eb6e4e9f06 100644 --- a/python/ray/serve/tests/test_standalone.py +++ b/python/ray/serve/tests/test_standalone.py @@ -703,7 +703,7 @@ class A: driver_template.format(address=address, namespace="test_namespace1", port=8000) ) run_string_as_driver( - driver_template.format(address=address, namespace="test_namespace2", port=8001) + driver_template.format(address=address, namespace="test_namespace2", port=8000) ) From 52b5d442b0150fe24b204d52f6c58f73c99fdedd Mon Sep 17 00:00:00 2001 From: axreldable Date: Sat, 1 Nov 2025 15:28:18 +0100 Subject: [PATCH 08/12] [serve] Add RayServeConfigException to docs Signed-off-by: axreldable --- doc/source/serve/api/index.md | 1 + python/ray/serve/exceptions.py | 2 ++ 2 files changed, 3 insertions(+) diff --git a/doc/source/serve/api/index.md b/doc/source/serve/api/index.md index af6540bd5af5..b4dd44adb1e2 100644 --- a/doc/source/serve/api/index.md +++ b/doc/source/serve/api/index.md @@ -141,6 +141,7 @@ See the [model composition guide](serve-model-composition) for how to update cod serve.grpc_util.RayServegRPCContext serve.exceptions.BackPressureError serve.exceptions.RayServeException + serve.exceptions.RayServeConfigException serve.exceptions.RequestCancelledError serve.exceptions.DeploymentUnavailableError ``` diff --git a/python/ray/serve/exceptions.py b/python/ray/serve/exceptions.py index e96519bbc059..9db5ffb6d942 100644 --- a/python/ray/serve/exceptions.py +++ b/python/ray/serve/exceptions.py @@ -12,6 +12,8 @@ class RayServeException(Exception): @PublicAPI(stability="alpha") class RayServeConfigException(Exception): + """Raised when a Serve config failed to be applied.""" + pass From e59e19aa6ef5b8da2255f3fb48569fc6d434e6b6 Mon Sep 17 00:00:00 2001 From: axreldable Date: Sun, 2 Nov 2025 15:02:38 +0100 Subject: [PATCH 09/12] [serve] Add todo Signed-off-by: axreldable --- python/ray/serve/_private/api.py | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/python/ray/serve/_private/api.py b/python/ray/serve/_private/api.py index 98e15a5c01a4..306d3119bd47 100644 --- a/python/ray/serve/_private/api.py +++ b/python/ray/serve/_private/api.py @@ -26,6 +26,12 @@ def _check_http_options( curr_http_options: HTTPOptions, new_http_options: Union[dict, HTTPOptions] ) -> None: + # todo: + # - review logix of this function + # - consider to have the same defaults for http_options in HTTPOptions and HTTPOptionsSchema + # - test different scenarios of starting/deploying to cluster: + # - CLI start - CLI run/deploy + # - CLI start - python run, etc def maybe_restore_proxy_location(prev_value, new_value) -> (str, str): if isinstance(prev_value, DeploymentMode) and isinstance( new_value, DeploymentMode From 53e1561b930c4c28edca1ecf71b9c4d9d5dca47b Mon Sep 17 00:00:00 2001 From: axreldable Date: Mon, 3 Nov 2025 20:44:27 +0100 Subject: [PATCH 10/12] [serve] Add tests Signed-off-by: axreldable --- python/ray/serve/_private/api.py | 11 +++----- python/ray/serve/tests/test_api_2.py | 40 ++++++++++++++++++++++++++++ 2 files changed, 44 insertions(+), 7 deletions(-) diff --git a/python/ray/serve/_private/api.py b/python/ray/serve/_private/api.py index 306d3119bd47..ac91290d3a4b 100644 --- a/python/ray/serve/_private/api.py +++ b/python/ray/serve/_private/api.py @@ -26,12 +26,6 @@ def _check_http_options( curr_http_options: HTTPOptions, new_http_options: Union[dict, HTTPOptions] ) -> None: - # todo: - # - review logix of this function - # - consider to have the same defaults for http_options in HTTPOptions and HTTPOptionsSchema - # - test different scenarios of starting/deploying to cluster: - # - CLI start - CLI run/deploy - # - CLI start - python run, etc def maybe_restore_proxy_location(prev_value, new_value) -> (str, str): if isinstance(prev_value, DeploymentMode) and isinstance( new_value, DeploymentMode @@ -41,6 +35,9 @@ def maybe_restore_proxy_location(prev_value, new_value) -> (str, str): new_value = ProxyLocation._from_deployment_mode(new_value).value return prev_value, new_value + if not new_http_options: + return + if isinstance(new_http_options, HTTPOptions): new_http_options = new_http_options.dict(exclude_unset=True) @@ -53,7 +50,7 @@ def maybe_restore_proxy_location(prev_value, new_value) -> (str, str): if diff_http_options: raise RayServeConfigException( "Attempt to update `http_options` or `proxy_location` has been detected! " - f"Attempted updates: {diff_http_options}. " + 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." ) diff --git a/python/ray/serve/tests/test_api_2.py b/python/ray/serve/tests/test_api_2.py index b8b8665b32b5..474c75e7f51b 100644 --- a/python/ray/serve/tests/test_api_2.py +++ b/python/ray/serve/tests/test_api_2.py @@ -3,8 +3,11 @@ import ray from ray import serve from ray._common.network_utils import build_address +from ray.serve import HTTPOptions +from ray.serve._private.api import _check_http_options from ray.serve._private.common import RequestProtocol from ray.serve._private.test_utils import get_application_urls +from ray.serve.config import DeploymentMode from ray.serve.exceptions import RayServeConfigException @@ -71,6 +74,43 @@ def test_serve_start(serve_instance): serve.start(http_options={"host": "127.0.0.1"}) +def test_check_http_options(): + curr = HTTPOptions() + _check_http_options(curr, None) + _check_http_options(curr, {}) + + curr = HTTPOptions() + new = HTTPOptions() + _check_http_options(curr, new) + _check_http_options({}, {}) + _check_http_options(None, None) + + curr = HTTPOptions(host="127.0.0.1", port=8000) + new = {"host": "0.0.0.0"} + with pytest.raises(RayServeConfigException) as exc: + _check_http_options(curr, new) + msg = str(exc.value) + assert "Attempted updates:" in msg + assert "`{'host': {'previous': '127.0.0.1', 'new': '0.0.0.0'}}`" in msg + + curr = HTTPOptions(host="127.0.0.1", port=8000, root_path="") + new = {"host": "0.0.0.0", "port": 8001} + with pytest.raises(RayServeConfigException) as exc: + _check_http_options(curr, new) + msg = str(exc.value) + assert ( + "`{'host': {'previous': '127.0.0.1', 'new': '0.0.0.0'}, " + "'port': {'previous': 8000, 'new': 8001}}`" + ) in msg + + curr = HTTPOptions(location=DeploymentMode.HeadOnly) + new = HTTPOptions(location=DeploymentMode.EveryNode) + with pytest.raises(RayServeConfigException) as exc: + _check_http_options(curr, new) + msg = str(exc.value) + assert "`{'location': {'previous': 'HeadOnly', 'new': 'EveryNode'}}`" in msg + + if __name__ == "__main__": import sys From 2f1f4c7148bcfac0297ecb955341c7d9ac5202e5 Mon Sep 17 00:00:00 2001 From: axreldable Date: Wed, 12 Nov 2025 11:19:02 +0100 Subject: [PATCH 11/12] [serve] Check valid keys Signed-off-by: axreldable --- python/ray/serve/_private/api.py | 8 ++++++++ python/ray/serve/tests/test_api_2.py | 7 +++++++ 2 files changed, 15 insertions(+) diff --git a/python/ray/serve/_private/api.py b/python/ray/serve/_private/api.py index ac91290d3a4b..0e2b66bd930d 100644 --- a/python/ray/serve/_private/api.py +++ b/python/ray/serve/_private/api.py @@ -43,6 +43,14 @@ def maybe_restore_proxy_location(prev_value, new_value) -> (str, str): diff_http_options = {} for option, new_value in new_http_options.items(): + if not hasattr(curr_http_options, option): + valid_fields = ( + getattr(HTTPOptions, "__fields__", None) + or getattr(HTTPOptions, "model_fields", {}).keys() + ) + raise RayServeConfigException( + f"Invalid http_options key '{option}'. Valid keys: {sorted(valid_fields)}" + ) prev_value = getattr(curr_http_options, option) if prev_value != new_value: prev_value, new_value = maybe_restore_proxy_location(prev_value, new_value) diff --git a/python/ray/serve/tests/test_api_2.py b/python/ray/serve/tests/test_api_2.py index 474c75e7f51b..9f90507b3dcd 100644 --- a/python/ray/serve/tests/test_api_2.py +++ b/python/ray/serve/tests/test_api_2.py @@ -85,6 +85,13 @@ def test_check_http_options(): _check_http_options({}, {}) _check_http_options(None, None) + curr = HTTPOptions() + new = {"foo": 123} + with pytest.raises(RayServeConfigException) as exc: + _check_http_options(curr, new) + msg = str(exc.value) + assert "Invalid http_options key 'foo'." in msg + curr = HTTPOptions(host="127.0.0.1", port=8000) new = {"host": "0.0.0.0"} with pytest.raises(RayServeConfigException) as exc: From 312f3543c8f87e70e99c93184bcc0f762efc1265 Mon Sep 17 00:00:00 2001 From: axreldable Date: Wed, 12 Nov 2025 22:12:34 +0100 Subject: [PATCH 12/12] [serve] Update docstring Signed-off-by: axreldable --- python/ray/serve/config.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/python/ray/serve/config.py b/python/ray/serve/config.py index f6b664189f89..540424642981 100644 --- a/python/ray/serve/config.py +++ b/python/ray/serve/config.py @@ -573,8 +573,8 @@ class HTTPOptions(BaseModel): - "HeadOnly": start one HTTP server on the head node. Serve assumes the head node is the node you executed serve.start - on. This is the default. - - "EveryNode": start one HTTP server per node. + on. + - "EveryNode": start one HTTP server per node. This is the default. - "NoServer": disable HTTP server. - num_cpus: [DEPRECATED] The number of CPU cores to reserve for each