Skip to content

Commit 263c7e1

Browse files
authored
[serve] configure http options in controller (#54331)
We set a bunch of defaults for http and grpc options in the proxy, but we should set them in the controller. --------- Signed-off-by: Cindy Zhang <[email protected]>
1 parent c0179ab commit 263c7e1

File tree

6 files changed

+45
-31
lines changed

6 files changed

+45
-31
lines changed

python/ray/serve/_private/controller.py

Lines changed: 9 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -36,6 +36,10 @@
3636
from ray.serve._private.deployment_info import DeploymentInfo
3737
from ray.serve._private.deployment_state import DeploymentStateManager
3838
from ray.serve._private.endpoint_state import EndpointState
39+
from ray.serve._private.grpc_util import set_proxy_default_grpc_options
40+
from ray.serve._private.http_util import (
41+
configure_http_options_with_defaults,
42+
)
3943
from ray.serve._private.logging_utils import (
4044
configure_component_cpu_profiler,
4145
configure_component_logger,
@@ -156,13 +160,16 @@ async def __init__(
156160
self.cluster_node_info_cache = create_cluster_node_info_cache(self.gcs_client)
157161
self.cluster_node_info_cache.update()
158162

163+
# Configure proxy default HTTP and gRPC options.
159164
self.proxy_state_manager = ProxyStateManager(
160-
http_options=http_options,
165+
http_options=configure_http_options_with_defaults(http_options),
161166
head_node_id=self._controller_node_id,
162167
cluster_node_info_cache=self.cluster_node_info_cache,
163168
logging_config=self.global_logging_config,
164-
grpc_options=grpc_options,
169+
grpc_options=set_proxy_default_grpc_options(grpc_options),
165170
)
171+
# We modify the HTTP and gRPC options above, so delete them to avoid
172+
del http_options, grpc_options
166173

167174
self.endpoint_state = EndpointState(self.kv_store, self.long_poll_host)
168175

python/ray/serve/_private/grpc_util.py

Lines changed: 17 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1,13 +1,18 @@
11
import asyncio
22
import logging
3+
from copy import deepcopy
34
from typing import Callable, List, Optional, Sequence, Tuple
45
from unittest.mock import Mock
56

67
import grpc
78
from grpc.aio._server import Server
89

910
from ray.exceptions import RayActorError, RayTaskError
10-
from ray.serve._private.constants import DEFAULT_GRPC_SERVER_OPTIONS, SERVE_LOGGER_NAME
11+
from ray.serve._private.constants import (
12+
DEFAULT_GRPC_SERVER_OPTIONS,
13+
RAY_SERVE_REQUEST_PROCESSING_TIMEOUT_S,
14+
SERVE_LOGGER_NAME,
15+
)
1116
from ray.serve._private.proxy_request_response import ResponseStatus
1217
from ray.serve.config import gRPCOptions
1318
from ray.serve.exceptions import BackPressureError, DeploymentUnavailableError
@@ -155,3 +160,14 @@ def set_grpc_code_and_details(
155160
context.set_code(status.code)
156161
if not context.details():
157162
context.set_details(status.message)
163+
164+
165+
def set_proxy_default_grpc_options(grpc_options) -> gRPCOptions:
166+
grpc_options = deepcopy(grpc_options) or gRPCOptions()
167+
168+
if grpc_options.request_timeout_s or RAY_SERVE_REQUEST_PROCESSING_TIMEOUT_S:
169+
grpc_options.request_timeout_s = (
170+
grpc_options.request_timeout_s or RAY_SERVE_REQUEST_PROCESSING_TIMEOUT_S
171+
)
172+
173+
return grpc_options

python/ray/serve/_private/http_util.py

Lines changed: 10 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -809,12 +809,19 @@ def configure_http_options_with_defaults(http_options: HTTPOptions) -> HTTPOptio
809809
http_options.keep_alive_timeout_s = RAY_SERVE_HTTP_KEEP_ALIVE_TIMEOUT_S
810810

811811
# TODO: Deprecate SERVE_REQUEST_PROCESSING_TIMEOUT_S env var
812-
http_options.request_timeout_s = (
813-
http_options.request_timeout_s or RAY_SERVE_REQUEST_PROCESSING_TIMEOUT_S
814-
)
812+
if http_options.request_timeout_s or RAY_SERVE_REQUEST_PROCESSING_TIMEOUT_S:
813+
http_options.request_timeout_s = (
814+
http_options.request_timeout_s or RAY_SERVE_REQUEST_PROCESSING_TIMEOUT_S
815+
)
815816

816817
http_options.middlewares = http_options.middlewares or []
817818

819+
return http_options
820+
821+
822+
def configure_http_middlewares(http_options: HTTPOptions) -> HTTPOptions:
823+
http_options = deepcopy(http_options)
824+
818825
# Add environment variable middleware
819826
if RAY_SERVE_HTTP_PROXY_CALLBACK_IMPORT_PATH:
820827
logger.info(

python/ray/serve/_private/proxy.py

Lines changed: 3 additions & 23 deletions
Original file line numberDiff line numberDiff line change
@@ -6,7 +6,6 @@
66
import pickle
77
import time
88
from abc import ABC, abstractmethod
9-
from copy import deepcopy
109
from typing import Any, Callable, Dict, Generator, Optional, Set, Tuple
1110

1211
import grpc
@@ -30,7 +29,6 @@
3029
RAY_SERVE_ENABLE_PROXY_GC_OPTIMIZATIONS,
3130
RAY_SERVE_PROXY_GC_THRESHOLD,
3231
RAY_SERVE_REQUEST_PATH_LOG_BUFFER_SIZE,
33-
RAY_SERVE_REQUEST_PROCESSING_TIMEOUT_S,
3432
REQUEST_LATENCY_BUCKETS_MS,
3533
SERVE_CONTROLLER_NAME,
3634
SERVE_HTTP_REQUEST_ID_HEADER,
@@ -46,7 +44,7 @@
4644
)
4745
from ray.serve._private.http_util import (
4846
MessageQueue,
49-
configure_http_options_with_defaults,
47+
configure_http_middlewares,
5048
convert_object_to_asgi_messages,
5149
get_http_response_status,
5250
receive_http_body,
@@ -1007,40 +1005,22 @@ async def send_request_to_replica(
10071005
yield status
10081006

10091007

1010-
def _set_proxy_default_grpc_options(grpc_options) -> gRPCOptions:
1011-
grpc_options = deepcopy(grpc_options) or gRPCOptions()
1012-
1013-
grpc_options.request_timeout_s = (
1014-
grpc_options.request_timeout_s or RAY_SERVE_REQUEST_PROCESSING_TIMEOUT_S
1015-
)
1016-
1017-
return grpc_options
1018-
1019-
10201008
@ray.remote(num_cpus=0)
10211009
class ProxyActor:
10221010
def __init__(
10231011
self,
10241012
http_options: HTTPOptions,
1013+
grpc_options: gRPCOptions,
10251014
*,
1026-
grpc_options: Optional[gRPCOptions] = None,
10271015
node_id: NodeId,
10281016
node_ip_address: str,
10291017
logging_config: LoggingConfig,
10301018
long_poll_client: Optional[LongPollClient] = None,
10311019
): # noqa: F821
10321020
self._node_id = node_id
10331021
self._node_ip_address = node_ip_address
1034-
1035-
# Configure proxy default HTTP and gRPC options.
1036-
http_options = configure_http_options_with_defaults(http_options)
1037-
grpc_options = _set_proxy_default_grpc_options(grpc_options)
1038-
self._http_options = http_options
1022+
self._http_options = configure_http_middlewares(http_options)
10391023
self._grpc_options = grpc_options
1040-
1041-
# We modify the HTTP and gRPC options above, so delete them to avoid
1042-
del http_options, grpc_options
1043-
10441024
grpc_enabled = is_grpc_enabled(self._grpc_options)
10451025

10461026
event_loop = get_or_create_event_loop()

python/ray/serve/tests/test_callback.py

Lines changed: 4 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -14,7 +14,7 @@
1414
from ray.exceptions import RayActorError
1515
from ray.serve._private.test_utils import get_application_url
1616
from ray.serve._private.utils import call_function_from_import_path
17-
from ray.serve.config import HTTPOptions
17+
from ray.serve.config import HTTPOptions, gRPCOptions
1818
from ray.serve.context import _get_global_client
1919
from ray.serve.schema import LoggingConfig, ProxyStatus, ServeInstanceDetails
2020

@@ -170,6 +170,7 @@ def test_callback_fail(ray_instance):
170170
actor_def = ray.serve._private.proxy.ProxyActor
171171
handle = actor_def.remote(
172172
http_options=HTTPOptions(host="http_proxy", root_path="/", port=123),
173+
grpc_options=gRPCOptions(),
173174
node_ip_address="127.0.0.1",
174175
node_id="123",
175176
logging_config=LoggingConfig(),
@@ -182,6 +183,7 @@ def test_callback_fail(ray_instance):
182183
actor_def = ray.actor._make_actor(serve_controller, {})
183184
handle = actor_def.remote(
184185
http_options=HTTPOptions(),
186+
grpc_options=gRPCOptions(),
185187
global_logging_config=LoggingConfig(),
186188
)
187189
with pytest.raises(RayActorError, match="cannot be imported"):
@@ -203,6 +205,7 @@ def test_http_proxy_return_aribitary_objects(ray_instance):
203205
actor_def = ray.serve._private.proxy.ProxyActor
204206
handle = actor_def.remote(
205207
http_options=HTTPOptions(host="http_proxy", root_path="/", port=123),
208+
grpc_options=gRPCOptions(),
206209
node_ip_address="127.0.0.1",
207210
node_id="123",
208211
logging_config=LoggingConfig(),

python/ray/serve/tests/unit/test_http_util.py

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -13,6 +13,7 @@
1313
from ray.serve._private.http_util import (
1414
ASGIReceiveProxy,
1515
MessageQueue,
16+
configure_http_middlewares,
1617
configure_http_options_with_defaults,
1718
)
1819

@@ -359,7 +360,7 @@ async def dispatch(self, request, call_next):
359360
] # Return list of wrapped middleware
360361

361362
# Act
362-
result = configure_http_options_with_defaults(base_http_options)
363+
result = configure_http_middlewares(base_http_options)
363364

364365
# Assert
365366
mock_call_function.assert_called_once_with(

0 commit comments

Comments
 (0)