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/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.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) 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..0e2b66bd930d 100644 --- a/python/ray/serve/_private/api.py +++ b/python/ray/serve/_private/api.py @@ -14,37 +14,54 @@ 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) - ) - 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 maybe_restore_proxy_location(prev_value, new_value) -> (str, str): + if isinstance(prev_value, DeploymentMode) and 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 not new_http_options: + return + + 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(): + 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_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." + ) def _start_controller( @@ -136,7 +153,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 +227,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/api.py b/python/ray/serve/api.py index d18bbd3d0e69..ba783e818fdd 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..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 @@ -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/exceptions.py b/python/ray/serve/exceptions.py index 10033b28f9cd..9db5ffb6d942 100644 --- a/python/ray/serve/exceptions.py +++ b/python/ray/serve/exceptions.py @@ -10,6 +10,13 @@ class RayServeException(Exception): pass +@PublicAPI(stability="alpha") +class RayServeConfigException(Exception): + """Raised when a Serve config failed to be applied.""" + + pass + + @PublicAPI(stability="alpha") class BackPressureError(RayServeException): """Raised when max_queued_requests is exceeded on a DeploymentHandle.""" 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_api_2.py b/python/ray/serve/tests/test_api_2.py index 65fd4c63e28e..9f90507b3dcd 100644 --- a/python/ray/serve/tests/test_api_2.py +++ b/python/ray/serve/tests/test_api_2.py @@ -3,8 +3,12 @@ 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 def test_get_application_urls(serve_instance): @@ -61,6 +65,59 @@ 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"}) + + +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() + 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: + _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 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, 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..85eb6e4e9f06 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 @@ -702,11 +703,11 @@ 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) ) -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, 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) 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,