Skip to content
Merged
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
61 changes: 60 additions & 1 deletion python/ray/serve/_private/config.py
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,14 @@
MAX_REPLICAS_PER_NODE_MAX_VALUE,
)
from ray.serve._private.utils import DEFAULT, DeploymentOptionUpdateType
from ray.serve.config import AggregationFunction, AutoscalingConfig, RequestRouterConfig
from ray.serve.config import (
AggregationFunction,
AutoscalingConfig,
DeploymentMode,
HTTPOptions,
ProxyLocation,
RequestRouterConfig,
)
from ray.serve.generated.serve_pb2 import (
AutoscalingConfig as AutoscalingConfigProto,
DeploymentConfig as DeploymentConfigProto,
Expand Down Expand Up @@ -799,3 +806,55 @@ def to_proto(self):

def to_proto_bytes(self):
return self.to_proto().SerializeToString()


def prepare_imperative_http_options(
proxy_location: Union[None, str, ProxyLocation],
http_options: Union[None, dict, HTTPOptions],
) -> HTTPOptions:
"""Prepare `HTTPOptions` with a resolved `location` based on `proxy_location` and `http_options`.

Precedence:
- If `proxy_location` is provided, it overrides any `location` in `http_options`.
- Else if `http_options` specifies a `location` explicitly (HTTPOptions(...) or dict with 'location'), keep it.
- Else (no `proxy_location` and no explicit `location`) set `location` to `DeploymentMode.EveryNode`.
A bare `HTTPOptions()` counts as an explicit default (`HeadOnly`).

Args:
proxy_location: Optional ProxyLocation (or its string representation).
http_options: Optional HTTPOptions instance or dict. If None, a new HTTPOptions() is created.

Returns:
HTTPOptions: New instance with resolved location.

Note:
1. Default ProxyLocation (when unspecified) resolves to DeploymentMode.EveryNode.
2. Default HTTPOptions() location is DeploymentMode.HeadOnly.
3. `HTTPOptions` is used in `imperative` mode (Python API) cluster set-up.
`Declarative` mode (CLI / REST) uses `HTTPOptionsSchema`.

Raises:
ValueError: If http_options is not None, dict, or HTTPOptions.
"""
if http_options is None:
location_set_explicitly = False
http_options = HTTPOptions()
elif isinstance(http_options, dict):
location_set_explicitly = "location" in http_options
http_options = HTTPOptions(**http_options)
elif isinstance(http_options, HTTPOptions):
# empty `HTTPOptions()` is considered as user specified the default location value `HeadOnly` explicitly
location_set_explicitly = True
http_options = HTTPOptions(**http_options.dict(exclude_unset=True))
else:
raise ValueError(
f"Unexpected type for http_options: `{type(http_options).__name__}`"
)

if proxy_location is None:
if not location_set_explicitly:
http_options.location = DeploymentMode.EveryNode
else:
http_options.location = ProxyLocation._to_deployment_mode(proxy_location)

return http_options
17 changes: 2 additions & 15 deletions python/ray/serve/api.py
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@
DeploymentConfig,
ReplicaConfig,
handle_num_replicas_auto,
prepare_imperative_http_options,
)
from ray.serve._private.constants import (
RAY_SERVE_FORCE_LOCAL_TESTING_MODE,
Expand All @@ -39,7 +40,6 @@
)
from ray.serve.config import (
AutoscalingConfig,
DeploymentMode,
HTTPOptions,
ProxyLocation,
RequestRouterConfig,
Expand Down Expand Up @@ -96,20 +96,7 @@ class See `gRPCOptions` for supported options.
logging_config: logging config options for the serve component (
controller & proxy).
"""
if proxy_location is None:
if http_options is None:
http_options = HTTPOptions(location=DeploymentMode.EveryNode)
else:
if http_options is None:
http_options = HTTPOptions()
elif isinstance(http_options, dict):
http_options = HTTPOptions(**http_options)

if isinstance(proxy_location, str):
proxy_location = ProxyLocation(proxy_location)

http_options.location = ProxyLocation._to_deployment_mode(proxy_location)

http_options = prepare_imperative_http_options(proxy_location, http_options)
_private_api.serve_start(
http_options=http_options,
grpc_options=grpc_options,
Expand Down
9 changes: 8 additions & 1 deletion python/ray/serve/scripts.py
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,11 @@
SERVE_DEFAULT_APP_NAME,
SERVE_NAMESPACE,
)
from ray.serve.config import DeploymentMode, ProxyLocation, gRPCOptions
from ray.serve.config import (
DeploymentMode,
ProxyLocation,
gRPCOptions,
)
from ray.serve.deployment import Application, deployment_to_schema
from ray.serve.schema import (
LoggingConfig,
Expand Down Expand Up @@ -533,6 +537,9 @@ def run(
grpc_options = gRPCOptions()
# Merge http_options and grpc_options with the ones on ServeDeploySchema.
if is_config and isinstance(config, ServeDeploySchema):
http_options["location"] = ProxyLocation._to_deployment_mode(
config.proxy_location
).value
config_http_options = config.http_options.dict()
http_options = {**config_http_options, **http_options}
grpc_options = gRPCOptions(**config.grpc_options.dict())
Expand Down
3 changes: 2 additions & 1 deletion 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, gRPCOptions
from ray.serve.config import HTTPOptions, ProxyLocation, gRPCOptions
from ray.serve.context import _get_global_client
from ray.tests.conftest import ( # noqa
external_redis,
Expand Down Expand Up @@ -148,6 +148,7 @@ def _shared_serve_instance():
_system_config={"metrics_report_interval_ms": 1000, "task_retry_delay_ms": 50},
)
serve.start(
proxy_location=ProxyLocation.HeadOnly,
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 current value which is used implicitly.

Copy link
Contributor

Choose a reason for hiding this comment

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

just confirming, even if we remove this, serve.start will still use HeadOnly?

Copy link
Contributor Author

@axreldable axreldable Oct 26, 2025

Choose a reason for hiding this comment

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

It was true before the change. In the current master, the result proxy_location if not provided explicitly depends on the presence of http_options in serve.start parameters. Empty http_options gives EveryNode, but non-empty http_options gives HeadOnly.

This PR changes this discrepancy. After the change serve.start runs cluster with default EveryNode proxy_location in case of empty proxy_location parameter.

See the example in the description:

  1. Fix discrepancy for 'proxy_location' in the Python API 'start' method

serve.start function in Python API sets different http_options.location depending on if http_options is provided.

Steps to reproduce:

  • have a script:
# discrepancy.py
import time

from ray import serve
from ray.serve.context import _get_global_client

if __name__ == '__main__':
    serve.start()
    client = _get_global_client()
    print(f"Empty http_options: `{client.http_config.location}`")

    serve.shutdown()
    time.sleep(5)

    serve.start(http_options={"host": "0.0.0.0"})
    client = _get_global_client()
    print(f"Non empty http_options: `{client.http_config.location}`")

Execute:

ray stop
ray start --head
python -m discrepancy

Output:

Before change:
Empty http_options: `EveryNode`
Non empty http_options: `HeadOnly`
After change:
Empty http_options: `EveryNode`
Non empty http_options: `EveryNode`

So, I use now

serve.start(
        proxy_location=ProxyLocation.HeadOnly,

to have the same behavior in tests as before 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.

makes sense :)

http_options={"host": "0.0.0.0"},
grpc_options={
"port": 9000,
Expand Down
55 changes: 55 additions & 0 deletions python/ray/serve/tests/test_cli_3.py
Original file line number Diff line number Diff line change
Expand Up @@ -4,6 +4,7 @@
import subprocess
import sys
import time
from typing import Union

import httpx
import pytest
Expand Down Expand Up @@ -138,6 +139,60 @@ def build_echo_app_typed(args: TypedArgs):


class TestRun:
@pytest.mark.skipif(
sys.platform == "win32", reason="File path incorrect on Windows."
)
@pytest.mark.parametrize(
"proxy_location,expected",
[
(
None,
"EveryNode",
), # default ProxyLocation `EveryNode` is used as http_options.location is not specified
("EveryNode", "EveryNode"),
("HeadOnly", "HeadOnly"),
("Disabled", "Disabled"),
],
)
def test_proxy_location(self, ray_start_stop, tmp_path, proxy_location, expected):
# when the `serve run` cli command is executed
# without serve already running (for the first time)
# `proxy_location` should be set from the config file if specified
def is_proxy_location_correct(expected_proxy_location: str) -> bool:
try:
response = httpx.get(
"http://localhost:8265/api/serve/applications/"
).text
response_json = json.loads(response)
print("response_json")
print(response_json)
return response_json["proxy_location"] == expected_proxy_location
except httpx.HTTPError:
return False

def arithmetic_config(with_proxy_location: Union[str, None]) -> str:
config_file_name = os.path.join(
os.path.dirname(__file__), "test_config_files", "arithmetic.yaml"
)
with open(config_file_name, "r") as config_file:
arithmetic_config_dict = yaml.safe_load(config_file)

config_path = tmp_path / "config.yaml"
if with_proxy_location:
arithmetic_config_dict["proxy_location"] = with_proxy_location
with open(config_path, "w") as f:
yaml.dump(arithmetic_config_dict, f)
return str(config_path)

config_path = arithmetic_config(with_proxy_location=proxy_location)
p = subprocess.Popen(["serve", "run", config_path])
wait_for_condition(
lambda: is_proxy_location_correct(expected_proxy_location=expected),
timeout=10,
)
p.send_signal(signal.SIGINT)
p.wait()

@pytest.mark.parametrize("number_of_kill_signals", (1, 2))
@pytest.mark.skipif(
sys.platform == "win32", reason="File path incorrect on Windows."
Expand Down
26 changes: 26 additions & 0 deletions python/ray/serve/tests/test_standalone.py
Original file line number Diff line number Diff line change
Expand Up @@ -876,6 +876,32 @@ def test_build_app_task_uses_zero_cpus(ray_shutdown):
"http_options": None,
"expected": HTTPOptions(location=DeploymentMode.EveryNode),
},
{
"proxy_location": None,
"http_options": {"test": "test"}, # location is not specified
"expected": HTTPOptions(
location=DeploymentMode.EveryNode
), # using default proxy_location (to align with the case when `http_options` are None)
},
{
"proxy_location": None,
"http_options": {
"location": "NoServer"
}, # `location` is specified, but `proxy_location` is not
"expected": HTTPOptions(
location=DeploymentMode.NoServer
), # using `location` value
},
{
"proxy_location": None,
"http_options": HTTPOptions(location=None),
"expected": HTTPOptions(location=DeploymentMode.NoServer),
},
{
"proxy_location": None,
"http_options": HTTPOptions(),
"expected": HTTPOptions(location=DeploymentMode.HeadOnly),
}, # using default location from HTTPOptions
{
"proxy_location": None,
"http_options": HTTPOptions(location="NoServer"),
Expand Down
72 changes: 71 additions & 1 deletion python/ray/serve/tests/unit/test_config.py
Original file line number Diff line number Diff line change
Expand Up @@ -6,7 +6,12 @@
from ray import cloudpickle, serve
from ray._common.pydantic_compat import ValidationError
from ray._common.utils import import_attr
from ray.serve._private.config import DeploymentConfig, ReplicaConfig, _proto_to_dict
from ray.serve._private.config import (
DeploymentConfig,
ReplicaConfig,
_proto_to_dict,
prepare_imperative_http_options,
)
from ray.serve._private.constants import (
DEFAULT_AUTOSCALING_POLICY_NAME,
DEFAULT_GRPC_PORT,
Expand Down Expand Up @@ -655,6 +660,71 @@ def test_http_options():
assert HTTPOptions(location=DeploymentMode.EveryNode).location == "EveryNode"


def test_prepare_imperative_http_options():
assert prepare_imperative_http_options(
proxy_location=None,
http_options=None,
) == HTTPOptions(location=DeploymentMode.EveryNode)

assert prepare_imperative_http_options(
proxy_location=None,
http_options={},
) == HTTPOptions(location=DeploymentMode.EveryNode)

assert prepare_imperative_http_options(
proxy_location=None,
http_options=HTTPOptions(**{}),
) == HTTPOptions(
location=DeploymentMode.HeadOnly
) # 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)

assert prepare_imperative_http_options(
proxy_location=None,
http_options={"test": "test"},
) == HTTPOptions(location=DeploymentMode.EveryNode)

assert prepare_imperative_http_options(
proxy_location=None,
http_options={"host": "0.0.0.0"},
) == HTTPOptions(location=DeploymentMode.EveryNode, host="0.0.0.0")

assert prepare_imperative_http_options(
proxy_location=None,
http_options={"location": "NoServer"},
) == HTTPOptions(location=DeploymentMode.NoServer)

assert prepare_imperative_http_options(
proxy_location=ProxyLocation.Disabled,
http_options=None,
) == HTTPOptions(location=DeploymentMode.NoServer)

assert prepare_imperative_http_options(
proxy_location=ProxyLocation.HeadOnly,
http_options={"host": "0.0.0.0"},
) == HTTPOptions(location=DeploymentMode.HeadOnly, host="0.0.0.0")

assert prepare_imperative_http_options(
proxy_location=ProxyLocation.HeadOnly,
http_options={"location": "NoServer"},
) == HTTPOptions(location=DeploymentMode.HeadOnly)

with pytest.raises(ValueError, match="not a valid ProxyLocation"):
prepare_imperative_http_options(proxy_location="wrong", http_options=None)

with pytest.raises(ValueError, match="not a valid enumeration"):
prepare_imperative_http_options(
proxy_location=None, http_options={"location": "123"}
)

with pytest.raises(ValueError, match="Unexpected type"):
prepare_imperative_http_options(proxy_location=None, http_options="wrong")


def test_with_proto():
# Test roundtrip
config = DeploymentConfig(num_replicas=100, max_ongoing_requests=16)
Expand Down