Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
1 change: 1 addition & 0 deletions doc/source/serve/api/index.md
Original file line number Diff line number Diff line change
Expand Up @@ -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
```
Expand Down
2 changes: 2 additions & 0 deletions doc/source/serve/production-guide/config.md
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
21 changes: 0 additions & 21 deletions python/ray/dashboard/modules/serve/serve_head.py
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down Expand Up @@ -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.

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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)

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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
Copy link
Contributor

Choose a reason for hiding this comment

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

shouldn't we send 409 Conflict in such scenario? @abrar thoughts?


# Fetch Serve status and confirm that HTTP options are unchanged
get_response = requests.get(SERVE_HEAD_URL, timeout=5)
Expand Down
63 changes: 40 additions & 23 deletions python/ray/serve/_private/api.py
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Copy link

Choose a reason for hiding this comment

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

Bug: Bug

The _check_http_options function can raise an AttributeError if new_http_options is a dictionary containing keys that aren't valid HTTPOptions attributes. This happens because the code directly uses getattr without validating the keys, leading to runtime failures and unhelpful error messages for users.

Fix in Cursor Fix in Web

Copy link
Contributor

Choose a reason for hiding this comment

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

+1

Copy link
Contributor Author

Choose a reason for hiding this comment

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

fixed.

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(
Expand Down Expand Up @@ -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
Expand Down Expand Up @@ -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
Expand Down
1 change: 0 additions & 1 deletion python/ray/serve/api.py
Original file line number Diff line number Diff line change
Expand Up @@ -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,
)

Expand Down
6 changes: 3 additions & 3 deletions python/ray/serve/config.py
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand All @@ -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
Copy link
Contributor Author

Choose a reason for hiding this comment

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

@abrarsheikh , please also notice that I change this default value. Aiming to align it with Proxy-location default. Without this if I remember correctly it's challenging to align all of the tests and can be confusing for users either.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I'd also align default host between http configs as I mentioned in the description, but this is a bigger change and I wasn't sure what value to choose and it was easy to fix tests for that discrepancy which became explicit after the change.

num_cpus: int = 0
root_url: str = ""
root_path: str = ""
Expand Down
7 changes: 7 additions & 0 deletions python/ray/serve/exceptions.py
Original file line number Diff line number Diff line change
Expand Up @@ -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."""
Expand Down
3 changes: 1 addition & 2 deletions python/ray/serve/tests/conftest.py
Original file line number Diff line number Diff line change
Expand Up @@ -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,
Expand Down Expand Up @@ -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,
Expand Down
57 changes: 57 additions & 0 deletions python/ray/serve/tests/test_api_2.py
Original file line number Diff line number Diff line change
Expand Up @@ -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):
Expand Down Expand Up @@ -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

Expand Down
34 changes: 34 additions & 0 deletions python/ray/serve/tests/test_cli.py
Original file line number Diff line number Diff line change
Expand Up @@ -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."""
Expand Down
6 changes: 3 additions & 3 deletions python/ray/serve/tests/test_cli_2.py
Original file line number Diff line number Diff line change
Expand Up @@ -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"

Expand Down Expand Up @@ -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,
Expand All @@ -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"})
Copy link
Contributor

Choose a reason for hiding this comment

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

why change this?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

this is the same discrepancy in default values.

wait_for_condition(
lambda: len(list_actors(filters=[("state", "=", "ALIVE")])) == 4,
timeout=15,
Expand Down
2 changes: 1 addition & 1 deletion python/ray/serve/tests/test_controller.py
Original file line number Diff line number Diff line change
Expand Up @@ -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,
Expand Down
Loading