From 690551ce1fdf0ae8c3d481565d2d18cd7d6fbb3d Mon Sep 17 00:00:00 2001 From: Seiji Eicher Date: Mon, 16 Jun 2025 17:54:24 -0700 Subject: [PATCH 01/36] Pass parameters to custom routers through LLMConfig Signed-off-by: Seiji Eicher --- python/ray/serve/_private/config.py | 3 +++ python/ray/serve/_private/router.py | 2 ++ 2 files changed, 5 insertions(+) diff --git a/python/ray/serve/_private/config.py b/python/ray/serve/_private/config.py index 6a5ed3655b7f..c3182be5f9b5 100644 --- a/python/ray/serve/_private/config.py +++ b/python/ray/serve/_private/config.py @@ -205,6 +205,9 @@ class DeploymentConfig(BaseModel): default=DEFAULT_REQUEST_ROUTER_PATH ) + # Keyword arguments that will be passed to the request router class. + request_router_kwargs: Dict[str, Any] = Field(default={}) + class Config: validate_assignment = True arbitrary_types_allowed = True diff --git a/python/ray/serve/_private/router.py b/python/ray/serve/_private/router.py index 32d10c8549eb..a39239238546 100644 --- a/python/ray/serve/_private/router.py +++ b/python/ray/serve/_private/router.py @@ -502,6 +502,7 @@ def request_router(self) -> Optional[RequestRouter]: prefer_local_node_routing=self._prefer_local_node_routing, prefer_local_az_routing=RAY_SERVE_PROXY_PREFER_LOCAL_AZ_ROUTING, self_availability_zone=self._availability_zone, + **self._request_router_kwargs, ) # Populate the running replicas if they are already available. @@ -538,6 +539,7 @@ def update_deployment_targets(self, deployment_target_info: DeploymentTargetInfo def update_deployment_config(self, deployment_config: DeploymentConfig): self._request_router_class = deployment_config.get_request_router_class() + self._request_router_kwargs = deployment_config.request_router_kwargs self._metrics_manager.update_deployment_config( deployment_config, curr_num_replicas=len(self.request_router.curr_replicas), From 5783bff64fb04f40db061ef2cd1aff2e6212a434 Mon Sep 17 00:00:00 2001 From: Seiji Eicher Date: Tue, 17 Jun 2025 18:30:41 -0700 Subject: [PATCH 02/36] Add request_router_kwargs to protobuf Signed-off-by: Seiji Eicher --- python/ray/serve/_private/config.py | 47 +++++++++++++++++++++++------ src/ray/protobuf/serve.proto | 2 ++ 2 files changed, 40 insertions(+), 9 deletions(-) diff --git a/python/ray/serve/_private/config.py b/python/ray/serve/_private/config.py index c3182be5f9b5..2d8ca1e2ea06 100644 --- a/python/ray/serve/_private/config.py +++ b/python/ray/serve/_private/config.py @@ -224,6 +224,20 @@ def user_config_json_serializable(cls, v): return v + @validator("request_router_kwargs", always=True) + def request_router_kwargs_json_serializable(cls, v): + if isinstance(v, bytes): + return v + if v is not None: + try: + json.dumps(v) + except TypeError as e: + raise ValueError( + f"request_router_kwargs is not JSON-serializable: {str(e)}." + ) + + return v + @validator("logging_config", always=True) def logging_config_valid(cls, v): if v is None: @@ -286,6 +300,11 @@ def to_proto(self): if data.get("user_config") is not None: if self.needs_pickle(): data["user_config"] = cloudpickle.dumps(data["user_config"]) + if data.get("request_router_kwargs") is not None: + if self.needs_pickle(): + data["request_router_kwargs"] = cloudpickle.dumps( + data["request_router_kwargs"] + ) if data.get("autoscaling_config"): data["autoscaling_config"] = AutoscalingConfigProto( **data["autoscaling_config"] @@ -308,23 +327,33 @@ def to_proto_bytes(self): @classmethod def from_proto(cls, proto: DeploymentConfigProto): data = _proto_to_dict(proto) + deployment_language = ( + data["deployment_language"] + if "deployment_language" in data + else DeploymentLanguage.PYTHON + ) + is_cross_language = ( + data["is_cross_language"] if "is_cross_language" in data else False + ) + needs_pickle = _needs_pickle(deployment_language, is_cross_language) if "user_config" in data: if data["user_config"] != b"": - deployment_language = ( - data["deployment_language"] - if "deployment_language" in data - else DeploymentLanguage.PYTHON - ) - is_cross_language = ( - data["is_cross_language"] if "is_cross_language" in data else False - ) - needs_pickle = _needs_pickle(deployment_language, is_cross_language) if needs_pickle: data["user_config"] = cloudpickle.loads(proto.user_config) else: data["user_config"] = proto.user_config else: data["user_config"] = None + if "request_router_kwargs" in data: + if data["request_router_kwargs"] != b"": + if needs_pickle: + data["request_router_kwargs"] = cloudpickle.loads( + proto.request_router_kwargs + ) + else: + data["request_router_kwargs"] = proto.request_router_kwargs + else: + data["request_router_kwargs"] = {} if "autoscaling_config" in data: if not data["autoscaling_config"].get("upscale_smoothing_factor"): data["autoscaling_config"]["upscale_smoothing_factor"] = None diff --git a/src/ray/protobuf/serve.proto b/src/ray/protobuf/serve.proto index 62f6145680e5..f18f83005c80 100644 --- a/src/ray/protobuf/serve.proto +++ b/src/ray/protobuf/serve.proto @@ -148,6 +148,8 @@ message DeploymentConfig { // Timeout after which a replica started a record routing stats without a response. double request_routing_stats_timeout_s = 18; + + bytes request_router_kwargs = 19; } // Deployment language. From 410fe608f56d6bccd8145876d070563dacb50c65 Mon Sep 17 00:00:00 2001 From: Seiji Eicher Date: Tue, 17 Jun 2025 19:01:49 -0700 Subject: [PATCH 03/36] Remove unnecessary lock from eviction loop Signed-off-by: Seiji Eicher --- .../deployments/test_prefix_aware_request_router.py | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/python/ray/llm/tests/serve/cpu/deployments/test_prefix_aware_request_router.py b/python/ray/llm/tests/serve/cpu/deployments/test_prefix_aware_request_router.py index 4ecd45f0dceb..14037e097a4c 100644 --- a/python/ray/llm/tests/serve/cpu/deployments/test_prefix_aware_request_router.py +++ b/python/ray/llm/tests/serve/cpu/deployments/test_prefix_aware_request_router.py @@ -124,7 +124,7 @@ async def test_fallback_when_no_prompt(self, prefix_request_router): req = fake_pending_request() for _ in range(10): - chosen = await prefix_request_router.choose_replica_for_request(req) + chosen = await prefix_request_router._choose_replica_for_request(req) assert chosen == r1 @pytest.mark.asyncio @@ -161,7 +161,7 @@ async def test_fallback_when_imbalanced(self, prefix_request_router): req = fake_pending_request(prompt="hello world") for _ in range(10): - chosen = await prefix_request_router.choose_replica_for_request(req) + chosen = await prefix_request_router._choose_replica_for_request(req) # Even though r2 has a higher match rate, it is not chosen because the load is imbalanced assert chosen == r1 @@ -199,13 +199,13 @@ async def test_high_match_rate_selects_matching_replica( prompt_req = fake_pending_request(prompt="Hello world") for _ in range(10): - chosen = await prefix_request_router.choose_replica_for_request(prompt_req) + chosen = await prefix_request_router._choose_replica_for_request(prompt_req) assert chosen == r2 chat_req = fake_pending_request( messages=[{"content": "Hello"}, {"content": " world"}] ) for _ in range(10): - chosen = await prefix_request_router.choose_replica_for_request(chat_req) + chosen = await prefix_request_router._choose_replica_for_request(chat_req) assert chosen == r2 @pytest.mark.asyncio @@ -240,14 +240,15 @@ async def test_low_match_rate_uses_smallest_tree(self, prefix_request_router): for _ in range(10): # Both tenants have 0% match rate, so the smaller tenant (r1) is chosen assert ( - await prefix_request_router.choose_replica_for_request(prompt_req) == r1 + await prefix_request_router._choose_replica_for_request(prompt_req) + == r1 ) chat_req = fake_pending_request(messages=[{"content": "z"}]) for _ in range(10): # Both tenants have 0% match rate, so the smaller tenant (r1) is chosen assert ( - await prefix_request_router.choose_replica_for_request(chat_req) == r1 + await prefix_request_router._choose_replica_for_request(chat_req) == r1 ) From b4c03f4960bb7c67a15987ed17911834fcf94b19 Mon Sep 17 00:00:00 2001 From: Seiji Eicher Date: Wed, 18 Jun 2025 11:47:38 -0700 Subject: [PATCH 04/36] Add request_router_kwargs to deployment options Signed-off-by: Seiji Eicher --- python/ray/serve/deployment.py | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/python/ray/serve/deployment.py b/python/ray/serve/deployment.py index 68139f5b5bb4..52358ae2628c 100644 --- a/python/ray/serve/deployment.py +++ b/python/ray/serve/deployment.py @@ -238,6 +238,7 @@ def options( health_check_timeout_s: Default[float] = DEFAULT.VALUE, logging_config: Default[Union[Dict, LoggingConfig, None]] = DEFAULT.VALUE, request_router_class: Default[Union[str, RequestRouter, None]] = DEFAULT.VALUE, + request_router_kwargs: Default[Union[Dict, None]] = DEFAULT.VALUE, request_routing_stats_period_s: Default[float] = DEFAULT.VALUE, request_routing_stats_timeout_s: Default[float] = DEFAULT.VALUE, _init_args: Default[Tuple[Any]] = DEFAULT.VALUE, @@ -375,6 +376,9 @@ def options( if request_router_class is not DEFAULT.VALUE: new_deployment_config.request_router_class = request_router_class + if request_router_kwargs is not DEFAULT.VALUE: + new_deployment_config.request_router_kwargs = request_router_kwargs + if request_routing_stats_period_s is not DEFAULT.VALUE: new_deployment_config.request_routing_stats_period_s = ( request_routing_stats_period_s From f61543d8834a764154ac848e4c3ab23de57a1f4e Mon Sep 17 00:00:00 2001 From: Seiji Eicher <58963096+eicherseiji@users.noreply.github.com> Date: Wed, 18 Jun 2025 12:04:01 -0700 Subject: [PATCH 05/36] Apply suggestions from code review Co-authored-by: Copilot <175728472+Copilot@users.noreply.github.com> Signed-off-by: Seiji Eicher <58963096+eicherseiji@users.noreply.github.com> --- python/ray/serve/_private/config.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/ray/serve/_private/config.py b/python/ray/serve/_private/config.py index 2d8ca1e2ea06..07b11fed776a 100644 --- a/python/ray/serve/_private/config.py +++ b/python/ray/serve/_private/config.py @@ -206,7 +206,7 @@ class DeploymentConfig(BaseModel): ) # Keyword arguments that will be passed to the request router class. - request_router_kwargs: Dict[str, Any] = Field(default={}) + request_router_kwargs: Dict[str, Any] = Field(default_factory=dict) class Config: validate_assignment = True From 4dfeabab27d3f333cb7e1030f50b5701be3f5cdc Mon Sep 17 00:00:00 2001 From: Seiji Eicher Date: Wed, 18 Jun 2025 12:05:52 -0700 Subject: [PATCH 06/36] Address code review Signed-off-by: Seiji Eicher --- python/ray/serve/_private/router.py | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/python/ray/serve/_private/router.py b/python/ray/serve/_private/router.py index a39239238546..b7c93b74c061 100644 --- a/python/ray/serve/_private/router.py +++ b/python/ray/serve/_private/router.py @@ -377,6 +377,7 @@ def __init__( prefer_local_node_routing: bool, resolve_request_arg_func: Coroutine = resolve_deployment_response, request_router_class: Optional[Callable] = None, + request_router_kwargs: Optional[Dict[str, Any]] = None, request_router: Optional[RequestRouter] = None, _request_router_initialized_event: Optional[asyncio.Event] = None, ): @@ -391,6 +392,9 @@ def __init__( self._handle_source = handle_source self._event_loop = event_loop self._request_router_class = request_router_class + self._request_router_kwargs = ( + request_router_kwargs if request_router_kwargs else {} + ) self._enable_strict_max_ongoing_requests = enable_strict_max_ongoing_requests self._node_id = node_id self._availability_zone = availability_zone From f62375e88c8ab72d30024f002435dc72b13c019f Mon Sep 17 00:00:00 2001 From: Seiji Eicher Date: Wed, 18 Jun 2025 16:26:36 -0700 Subject: [PATCH 07/36] Update api docs Signed-off-by: Seiji Eicher --- python/ray/serve/_private/config.py | 3 +++ python/ray/serve/api.py | 3 +++ python/ray/serve/schema.py | 4 ++++ 3 files changed, 10 insertions(+) diff --git a/python/ray/serve/_private/config.py b/python/ray/serve/_private/config.py index 07b11fed776a..28c8aac375c0 100644 --- a/python/ray/serve/_private/config.py +++ b/python/ray/serve/_private/config.py @@ -129,6 +129,9 @@ class DeploymentConfig(BaseModel): logging_config: Configuration for deployment logs. user_configured_option_names: The names of options manually configured by the user. + request_router_class: Custom router class to use for routing requests. + request_router_kwargs: Keyword arguments that will be passed to the + request router class __init__ method. """ num_replicas: Optional[NonNegativeInt] = Field( diff --git a/python/ray/serve/api.py b/python/ray/serve/api.py index aea80fe88ce6..cacbc970db63 100644 --- a/python/ray/serve/api.py +++ b/python/ray/serve/api.py @@ -339,6 +339,7 @@ def deployment( health_check_timeout_s: Default[float] = DEFAULT.VALUE, logging_config: Default[Union[Dict, LoggingConfig, None]] = DEFAULT.VALUE, request_router_class: Default[Union[str, RequestRouter, None]] = DEFAULT.VALUE, + request_router_kwargs: Default[Union[Dict, None]] = DEFAULT.VALUE, request_routing_stats_period_s: Default[float] = DEFAULT.VALUE, request_routing_stats_timeout_s: Default[float] = DEFAULT.VALUE, ) -> Callable[[Callable], Deployment]: @@ -410,6 +411,8 @@ class MyDeployment: handle created for this deployment will use the routing policy defined by the request router. Default to Serve's PowerOfTwoChoicesRequestRouter. + request_router_kwargs: Keyword arguments that will be passed to the + request router class __init__ method. request_routing_stats_period_s: Duration between record scheduling stats calls for the replica. Defaults to 10s. The health check is by default a no-op Actor call to the replica, but you can define your own request diff --git a/python/ray/serve/schema.py b/python/ray/serve/schema.py index 80904f875765..040477aced73 100644 --- a/python/ray/serve/schema.py +++ b/python/ray/serve/schema.py @@ -409,6 +409,10 @@ class DeploymentSchema(BaseModel, allow_population_by_field_name=True): default=DEFAULT.VALUE, description="The path pointing to the custom request router class to use for this deployment.", ) + request_router_kwargs: Dict[str, Any] = Field( + default=DEFAULT.VALUE, + description="Keyword arguments that will be passed to the request router class __init__ method.", + ) request_routing_stats_period_s: float = Field( default=DEFAULT.VALUE, description=( From bea9baaab00358e64a0b69f3b681d319ad8b31da Mon Sep 17 00:00:00 2001 From: Seiji Eicher Date: Wed, 18 Jun 2025 17:27:06 -0700 Subject: [PATCH 08/36] Add comment to Protobuf Signed-off-by: Seiji Eicher --- python/ray/serve/api.py | 9 ++++----- src/ray/protobuf/serve.proto | 1 + 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/python/ray/serve/api.py b/python/ray/serve/api.py index cacbc970db63..4788f8304a00 100644 --- a/python/ray/serve/api.py +++ b/python/ray/serve/api.py @@ -55,11 +55,7 @@ from ray.serve.exceptions import RayServeException from ray.serve.handle import DeploymentHandle from ray.serve.multiplex import _ModelMultiplexWrapper -from ray.serve.schema import ( - LoggingConfig, - ServeInstanceDetails, - ServeStatus, -) +from ray.serve.schema import LoggingConfig, ServeInstanceDetails, ServeStatus from ray.util.annotations import DeveloperAPI, PublicAPI from ray.serve._private import api as _private_api # isort:skip @@ -494,6 +490,9 @@ class MyDeployment: if request_router_class is not DEFAULT.VALUE: deployment_config.request_router_class = request_router_class + if request_router_kwargs is not DEFAULT.VALUE: + deployment_config.request_router_kwargs = request_router_kwargs + def decorator(_func_or_class): replica_config = ReplicaConfig.create( _func_or_class, diff --git a/src/ray/protobuf/serve.proto b/src/ray/protobuf/serve.proto index f18f83005c80..bfbd59113dbf 100644 --- a/src/ray/protobuf/serve.proto +++ b/src/ray/protobuf/serve.proto @@ -149,6 +149,7 @@ message DeploymentConfig { // Timeout after which a replica started a record routing stats without a response. double request_routing_stats_timeout_s = 18; + // kwargs which will be passed to the router class' __init__ method bytes request_router_kwargs = 19; } From de9561add741a8968411cecfdd0ebb9bb42559e0 Mon Sep 17 00:00:00 2001 From: Seiji Eicher Date: Thu, 19 Jun 2025 13:43:56 -0700 Subject: [PATCH 09/36] Remove prefix tree changes from this PR Signed-off-by: Seiji Eicher --- .../prefix_aware/prefix_tree.py | 60 +++++++++++-------- 1 file changed, 34 insertions(+), 26 deletions(-) diff --git a/python/ray/llm/_internal/serve/request_router/prefix_aware/prefix_tree.py b/python/ray/llm/_internal/serve/request_router/prefix_aware/prefix_tree.py index b3765e6e6813..ceed508f9e3f 100644 --- a/python/ray/llm/_internal/serve/request_router/prefix_aware/prefix_tree.py +++ b/python/ray/llm/_internal/serve/request_router/prefix_aware/prefix_tree.py @@ -1,5 +1,6 @@ from __future__ import annotations +import asyncio import logging import os import threading @@ -96,8 +97,7 @@ def __init__(self) -> None: # LRU tracking - root is always the head, tail is the least recently used. self.tenant_to_lru_tail: Dict[str, Optional[Node]] = {} - self._eviction_thread: Optional[threading.Thread] = None - self._eviction_stop_event: threading.Event = threading.Event() + self._eviction_task: Optional[asyncio.Task] = None @staticmethod def _shared_prefix_count(a: str, b: str) -> int: @@ -140,7 +140,9 @@ def _insert_node_into_linked_list( """ with self.lock: if tenant not in self.tenant_to_char_count: - logger.debug(f"Tenant '{tenant}' does not exist. No action taken.") + logger.debug( + f"[_insert_node_into_linked_list] Tenant '{tenant}' does not exist. No action taken." + ) return # Skip if node is the root @@ -165,7 +167,9 @@ def _remove_node_from_linked_list(self, node: Node, tenant: str) -> None: """ with self.lock: if tenant not in self.tenant_to_char_count: - logger.debug(f"Tenant '{tenant}' does not exist. No action taken.") + logger.debug( + f"[_remove_node_from_linked_list] Tenant '{tenant}' does not exist. No action taken." + ) return # Skip if node is the root @@ -203,11 +207,13 @@ def _remove_tenant_single_node(self, tenant: str, node: Node) -> int: """ with self.lock: if tenant not in self.tenant_to_char_count: - logger.debug(f"Tenant '{tenant}' does not exist. No action taken.") + logger.debug( + f"[_remove_tenant_single_node] Tenant '{tenant}' does not exist. No action taken." + ) return 0 if tenant not in node.tenant_to_last_access_time: logger.debug( - f"Tenant '{tenant}' does not have node '{node.text}'. No action taken." + f"[_remove_tenant_single_node] Tenant '{tenant}' does not have node '{node.text}'. No action taken." ) return 0 @@ -239,7 +245,9 @@ def add_tenants(self, tenants: List[str], time_s: float) -> None: with self.lock: for tenant in tenants: if tenant in self.tenant_to_char_count: - logger.debug(f"Tenant '{tenant}' already exists. Skipping.") + logger.debug( + f"[_add_tenants] Tenant '{tenant}' already exists. Skipping." + ) continue self.tenant_to_char_count[tenant] = 0 @@ -276,7 +284,7 @@ def insert(self, text: str, tenant: str, time_s: float) -> None: with self.lock: if tenant not in self.tenant_to_char_count: logger.debug( - f"Tenant '{tenant}' does not exist. Use add_tenants() first." + f"[_insert] Tenant '{tenant}' does not exist. Use add_tenants() first." ) return @@ -461,7 +469,9 @@ def remove_tenants(self, tenants: List[str]) -> Dict[str, int]: with self.lock: for tenant in tenants: if tenant not in self.tenant_to_char_count: - logger.debug(f"Tenant '{tenant}' does not exist. Skipping.") + logger.debug( + f"[_remove_tenants] Tenant '{tenant}' does not exist. Skipping." + ) chars_removed[tenant] = 0 continue @@ -504,13 +514,13 @@ def evict_tenant_by_lru(self, tenant: str, min_remove_size: int) -> int: with self.lock: if tenant not in self.tenant_to_char_count: logger.debug( - f"Cannot evict tenant '{tenant}': tenant does not exist. No action taken." + f"[_evict_tenant_by_lru] Cannot evict tenant '{tenant}': tenant does not exist. No action taken." ) return 0 if self.tenant_to_char_count[tenant] < min_remove_size: logger.debug( - f"Cannot evict {min_remove_size} characters from tenant '{tenant}', which has only " + f"[_evict_tenant_by_lru] Cannot evict {min_remove_size} characters from tenant '{tenant}', which has only " f"{self.tenant_to_char_count[tenant]} characters. Will remove all available characters." ) min_remove_size = self.tenant_to_char_count[tenant] @@ -564,9 +574,8 @@ def get_smallest_tenants(self) -> Optional[List[str]]: def start_eviction_loop( self, eviction_threshold: int, eviction_target: int, interval_secs: float ) -> bool: - """Start a single eviction loop within the actor itself. - - Args: + """Start a single eviction loop within the actor itself + Parameters: eviction_threshold: Minimum number of characters a tenant must have to be evicted eviction_target: The maximum number of characters a tenant should have after eviction interval_secs: Number of seconds between eviction checks @@ -574,18 +583,16 @@ def start_eviction_loop( Returns: True if the loop was started, False if it was already running """ - self._eviction_stop_event.clear() with self.lock: - if self._eviction_thread is None: - self._eviction_thread = threading.Thread( - target=self._run_eviction_loop, - args=(eviction_threshold, eviction_target, interval_secs), - daemon=True, + if self._eviction_task is None: + self._eviction_task = asyncio.create_task( + self._run_eviction_loop( + eviction_threshold, eviction_target, interval_secs + ) ) - self._eviction_thread.start() return True else: - logger.debug("Eviction loop already running") + logger.debug("[_start_eviction_loop] Eviction loop already running") return False def _run_eviction_loop(self, eviction_threshold, eviction_target, interval_secs): @@ -601,10 +608,11 @@ def _run_eviction_loop(self, eviction_threshold, eviction_target, interval_secs) self.evict_tenant_by_lru(tenant, excess) def stop_eviction_loop(self): - self._eviction_stop_event.set() - if self._eviction_thread: - self._eviction_thread.join() - self._eviction_thread = None + with self.lock: + if self._eviction_task: + self._eviction_task.cancel() + # self._eviction_task.close() + self._eviction_task = None @ray.remote From b363565d96e899cf790fc3a30951fbff2f4c8036 Mon Sep 17 00:00:00 2001 From: Seiji Eicher Date: Mon, 23 Jun 2025 16:26:38 -0700 Subject: [PATCH 10/36] Create initialize_state() to avoid **kwargs in RequestRouter swallowing user typos Signed-off-by: Seiji Eicher --- .../test_prefix_aware_request_router.py | 14 ++++++++------ .../request_router/prefix_aware_router.py | 15 +++++++++------ .../_private/request_router/request_router.py | 11 +++++++++++ python/ray/serve/_private/router.py | 2 +- 4 files changed, 29 insertions(+), 13 deletions(-) diff --git a/python/ray/llm/tests/serve/cpu/deployments/test_prefix_aware_request_router.py b/python/ray/llm/tests/serve/cpu/deployments/test_prefix_aware_request_router.py index 14037e097a4c..0cf7d123686f 100644 --- a/python/ray/llm/tests/serve/cpu/deployments/test_prefix_aware_request_router.py +++ b/python/ray/llm/tests/serve/cpu/deployments/test_prefix_aware_request_router.py @@ -48,12 +48,6 @@ async def construct_request_router(loop: asyncio.AbstractEventLoop): deployment_id=DeploymentID(name="TEST_DEPLOYMENT"), handle_source=DeploymentHandleSource.REPLICA, use_replica_queue_len_cache=False, - imbalanced_threshold=params.get("imbalanced_threshold", 10), - match_rate_threshold=params.get("match_rate_threshold", 0.1), - do_eviction=params.get("do_eviction", False), - eviction_threshold_chars=params.get("eviction_threshold_chars"), - eviction_target_chars=params.get("eviction_target_chars"), - eviction_interval_secs=params.get("eviction_interval_secs"), get_curr_time_s=TIMER.time, tree_actor=tree_actor, ) @@ -62,6 +56,14 @@ async def construct_request_router(loop: asyncio.AbstractEventLoop): request_router = asyncio.new_event_loop().run_until_complete( construct_request_router(get_or_create_event_loop()) ) + request_router.initialize_state( + imbalanced_threshold=params.get("imbalanced_threshold", 10), + match_rate_threshold=params.get("match_rate_threshold", 0.1), + do_eviction=params.get("do_eviction", False), + eviction_threshold_chars=params.get("eviction_threshold_chars"), + eviction_target_chars=params.get("eviction_target_chars"), + eviction_interval_secs=params.get("eviction_interval_secs"), + ) yield request_router assert request_router.curr_num_routing_tasks == 0 diff --git a/python/ray/serve/_private/request_router/prefix_aware_router.py b/python/ray/serve/_private/request_router/prefix_aware_router.py index cbc7b362bb34..b235d30402ce 100644 --- a/python/ray/serve/_private/request_router/prefix_aware_router.py +++ b/python/ray/serve/_private/request_router/prefix_aware_router.py @@ -55,12 +55,6 @@ class PrefixAwarePow2ReplicaRouter(LocalityMixin, MultiplexMixin, RequestRouter) def __init__( self, *args, - imbalanced_threshold=10, - match_rate_threshold=0.1, - do_eviction=False, - eviction_threshold_chars=400_000, - eviction_target_chars=360_000, - eviction_interval_secs=10, tree_actor=None, **kwargs, ): @@ -73,6 +67,15 @@ def __init__( else: self._tree_actor = tree_actor + def initialize_state( + self, + imbalanced_threshold: Optional[int] = 10, + match_rate_threshold: Optional[float] = 0.1, + do_eviction: Optional[bool] = False, + eviction_threshold_chars: Optional[int] = 400_000, + eviction_target_chars: Optional[int] = 360_000, + eviction_interval_secs: Optional[int] = 10, + ): # === Prefix-aware routing logic hyperparameters === self._imbalanced_threshold = imbalanced_threshold self._match_rate_threshold = match_rate_threshold diff --git a/python/ray/serve/_private/request_router/request_router.py b/python/ray/serve/_private/request_router/request_router.py index 4b10a37537d0..19ac7721e06e 100644 --- a/python/ray/serve/_private/request_router/request_router.py +++ b/python/ray/serve/_private/request_router/request_router.py @@ -7,6 +7,7 @@ from abc import ABC, abstractmethod from collections import defaultdict, deque from typing import ( + Any, AsyncGenerator, Callable, DefaultDict, @@ -463,6 +464,7 @@ def __init__( create_replica_wrapper_func: Optional[ Callable[[RunningReplicaInfo], RunningReplica] ] = None, + request_router_kwargs: Optional[Dict[str, Any]] = None, *args, **kwargs, ): @@ -534,6 +536,15 @@ def __init__( ) self.num_routing_tasks_in_backoff_gauge.set(self.num_routing_tasks_in_backoff) + self.initialize_state(**(request_router_kwargs or {})) + + def initialize_state(self, **kwargs): + """Initialize the state of the request router. + Called at the end of RequestRouter.__init__ with request_router_kwargs + from DeploymentConfig . + """ + pass + @property def _event_loop(self) -> asyncio.AbstractEventLoop: if self._lazily_fetched_loop is None: diff --git a/python/ray/serve/_private/router.py b/python/ray/serve/_private/router.py index b7c93b74c061..63d8b4b0096c 100644 --- a/python/ray/serve/_private/router.py +++ b/python/ray/serve/_private/router.py @@ -506,7 +506,7 @@ def request_router(self) -> Optional[RequestRouter]: prefer_local_node_routing=self._prefer_local_node_routing, prefer_local_az_routing=RAY_SERVE_PROXY_PREFER_LOCAL_AZ_ROUTING, self_availability_zone=self._availability_zone, - **self._request_router_kwargs, + request_router_kwargs=self._request_router_kwargs, ) # Populate the running replicas if they are already available. From cd0a14b4f4ba4368045f4aae04775a5a19a5c0f1 Mon Sep 17 00:00:00 2001 From: Seiji Eicher Date: Wed, 25 Jun 2025 13:42:04 -0700 Subject: [PATCH 11/36] Create RouterConfig Signed-off-by: Seiji Eicher --- python/ray/serve/_private/config.py | 116 ++++-------------- .../_private/request_router/request_router.py | 4 - python/ray/serve/_private/router.py | 11 +- python/ray/serve/api.py | 33 +---- python/ray/serve/config.py | 89 +++++++++++++- python/ray/serve/deployment.py | 6 +- python/ray/serve/schema.py | 45 +++---- src/ray/protobuf/serve.proto | 40 +++--- 8 files changed, 162 insertions(+), 182 deletions(-) diff --git a/python/ray/serve/_private/config.py b/python/ray/serve/_private/config.py index 28c8aac375c0..4580f2d7ce4e 100644 --- a/python/ray/serve/_private/config.py +++ b/python/ray/serve/_private/config.py @@ -13,10 +13,9 @@ NonNegativeInt, PositiveFloat, PositiveInt, - root_validator, validator, ) -from ray._common.utils import import_attr, resources_from_ray_options +from ray._common.utils import resources_from_ray_options from ray._private import ray_option_utils from ray._private.serialization import pickle_dumps from ray.serve._private.constants import ( @@ -25,13 +24,10 @@ DEFAULT_HEALTH_CHECK_PERIOD_S, DEFAULT_HEALTH_CHECK_TIMEOUT_S, DEFAULT_MAX_ONGOING_REQUESTS, - DEFAULT_REQUEST_ROUTER_PATH, - DEFAULT_REQUEST_ROUTING_STATS_PERIOD_S, - DEFAULT_REQUEST_ROUTING_STATS_TIMEOUT_S, MAX_REPLICAS_PER_NODE_MAX_VALUE, ) from ray.serve._private.utils import DEFAULT, DeploymentOptionUpdateType -from ray.serve.config import AutoscalingConfig +from ray.serve.config import AutoscalingConfig, RouterConfig from ray.serve.generated.serve_pb2 import ( AutoscalingConfig as AutoscalingConfigProto, DeploymentConfig as DeploymentConfigProto, @@ -39,6 +35,7 @@ EncodingType as EncodingTypeProto, LoggingConfig as LoggingConfigProto, ReplicaConfig as ReplicaConfigProto, + RouterConfig as RouterConfigProto, ) from ray.util.placement_group import validate_placement_group @@ -129,9 +126,7 @@ class DeploymentConfig(BaseModel): logging_config: Configuration for deployment logs. user_configured_option_names: The names of options manually configured by the user. - request_router_class: Custom router class to use for routing requests. - request_router_kwargs: Keyword arguments that will be passed to the - request router class __init__ method. + router_config: Configuration for deployment request router. """ num_replicas: Optional[NonNegativeInt] = Field( @@ -166,19 +161,16 @@ class DeploymentConfig(BaseModel): default=DEFAULT_HEALTH_CHECK_TIMEOUT_S, update_type=DeploymentOptionUpdateType.NeedsReconfigure, ) - request_routing_stats_period_s: PositiveFloat = Field( - default=DEFAULT_REQUEST_ROUTING_STATS_PERIOD_S, - update_type=DeploymentOptionUpdateType.NeedsReconfigure, - ) - request_routing_stats_timeout_s: PositiveFloat = Field( - default=DEFAULT_REQUEST_ROUTING_STATS_TIMEOUT_S, - update_type=DeploymentOptionUpdateType.NeedsReconfigure, - ) autoscaling_config: Optional[AutoscalingConfig] = Field( default=None, update_type=DeploymentOptionUpdateType.NeedsActorReconfigure ) + router_config: RouterConfig = Field( + default=RouterConfig(), + update_type=DeploymentOptionUpdateType.NeedsActorReconfigure, + ) + # This flag is used to let replica know they are deployed from # a different language. is_cross_language: bool = False @@ -200,17 +192,6 @@ class DeploymentConfig(BaseModel): # Contains the names of deployment options manually set by the user user_configured_option_names: Set[str] = set() - # Cloudpickled request router class. - serialized_request_router_cls: bytes = Field(default=b"") - - # Custom request router config. Defaults to the power of two request router. - request_router_class: Union[str, Callable] = Field( - default=DEFAULT_REQUEST_ROUTER_PATH - ) - - # Keyword arguments that will be passed to the request router class. - request_router_kwargs: Dict[str, Any] = Field(default_factory=dict) - class Config: validate_assignment = True arbitrary_types_allowed = True @@ -227,20 +208,6 @@ def user_config_json_serializable(cls, v): return v - @validator("request_router_kwargs", always=True) - def request_router_kwargs_json_serializable(cls, v): - if isinstance(v, bytes): - return v - if v is not None: - try: - json.dumps(v) - except TypeError as e: - raise ValueError( - f"request_router_kwargs is not JSON-serializable: {str(e)}." - ) - - return v - @validator("logging_config", always=True) def logging_config_valid(cls, v): if v is None: @@ -268,33 +235,6 @@ def validate_max_queued_requests(cls, v): return v - @root_validator - def import_and_serialize_request_router_cls(cls, values) -> Dict[str, Any]: - """Import and serialize request router class with cloudpickle. - - Import the request router if it's passed in as a string import path. - Then cloudpickle the request router and set to - `serialized_request_router_cls`. - """ - request_router_class = values.get("request_router_class") - if isinstance(request_router_class, Callable): - request_router_class = ( - f"{request_router_class.__module__}.{request_router_class.__name__}" - ) - - request_router_path = request_router_class or DEFAULT_REQUEST_ROUTER_PATH - request_router_class = import_attr(request_router_path) - - values["serialized_request_router_cls"] = cloudpickle.dumps( - request_router_class - ) - values["request_router_class"] = request_router_path - return values - - def get_request_router_class(self) -> Callable: - """Deserialize request router from cloudpickled bytes.""" - return cloudpickle.loads(self.serialized_request_router_cls) - def needs_pickle(self): return _needs_pickle(self.deployment_language, self.is_cross_language) @@ -303,21 +243,17 @@ def to_proto(self): if data.get("user_config") is not None: if self.needs_pickle(): data["user_config"] = cloudpickle.dumps(data["user_config"]) - if data.get("request_router_kwargs") is not None: - if self.needs_pickle(): - data["request_router_kwargs"] = cloudpickle.dumps( - data["request_router_kwargs"] - ) if data.get("autoscaling_config"): data["autoscaling_config"] = AutoscalingConfigProto( **data["autoscaling_config"] ) + if data.get("router_config"): + data["router_config"] = RouterConfigProto(**data["router_config"]) if data.get("logging_config"): if "encoding" in data["logging_config"]: data["logging_config"]["encoding"] = EncodingTypeProto.Value( data["logging_config"]["encoding"] ) - data["logging_config"] = LoggingConfigProto(**data["logging_config"]) data["user_configured_option_names"] = list( data["user_configured_option_names"] @@ -330,33 +266,25 @@ def to_proto_bytes(self): @classmethod def from_proto(cls, proto: DeploymentConfigProto): data = _proto_to_dict(proto) - deployment_language = ( - data["deployment_language"] - if "deployment_language" in data - else DeploymentLanguage.PYTHON - ) - is_cross_language = ( - data["is_cross_language"] if "is_cross_language" in data else False - ) - needs_pickle = _needs_pickle(deployment_language, is_cross_language) if "user_config" in data: if data["user_config"] != b"": + deployment_language = ( + data["deployment_language"] + if "deployment_language" in data + else DeploymentLanguage.PYTHON + ) + is_cross_language = ( + data["is_cross_language"] if "is_cross_language" in data else False + ) + needs_pickle = _needs_pickle(deployment_language, is_cross_language) if needs_pickle: data["user_config"] = cloudpickle.loads(proto.user_config) else: data["user_config"] = proto.user_config else: data["user_config"] = None - if "request_router_kwargs" in data: - if data["request_router_kwargs"] != b"": - if needs_pickle: - data["request_router_kwargs"] = cloudpickle.loads( - proto.request_router_kwargs - ) - else: - data["request_router_kwargs"] = proto.request_router_kwargs - else: - data["request_router_kwargs"] = {} + if "router_config" in data: + data["router_config"] = RouterConfig(**data["router_config"]) if "autoscaling_config" in data: if not data["autoscaling_config"].get("upscale_smoothing_factor"): data["autoscaling_config"]["upscale_smoothing_factor"] = None diff --git a/python/ray/serve/_private/request_router/request_router.py b/python/ray/serve/_private/request_router/request_router.py index 19ac7721e06e..85c7aa8ca9cb 100644 --- a/python/ray/serve/_private/request_router/request_router.py +++ b/python/ray/serve/_private/request_router/request_router.py @@ -7,7 +7,6 @@ from abc import ABC, abstractmethod from collections import defaultdict, deque from typing import ( - Any, AsyncGenerator, Callable, DefaultDict, @@ -464,7 +463,6 @@ def __init__( create_replica_wrapper_func: Optional[ Callable[[RunningReplicaInfo], RunningReplica] ] = None, - request_router_kwargs: Optional[Dict[str, Any]] = None, *args, **kwargs, ): @@ -536,8 +534,6 @@ def __init__( ) self.num_routing_tasks_in_backoff_gauge.set(self.num_routing_tasks_in_backoff) - self.initialize_state(**(request_router_kwargs or {})) - def initialize_state(self, **kwargs): """Initialize the state of the request router. Called at the end of RequestRouter.__init__ with request_router_kwargs diff --git a/python/ray/serve/_private/router.py b/python/ray/serve/_private/router.py index 63d8b4b0096c..b03d6313b20a 100644 --- a/python/ray/serve/_private/router.py +++ b/python/ray/serve/_private/router.py @@ -506,8 +506,9 @@ def request_router(self) -> Optional[RequestRouter]: prefer_local_node_routing=self._prefer_local_node_routing, prefer_local_az_routing=RAY_SERVE_PROXY_PREFER_LOCAL_AZ_ROUTING, self_availability_zone=self._availability_zone, - request_router_kwargs=self._request_router_kwargs, + **(self._request_router_kwargs or {}), ) + request_router.initialize_state(**(self._request_router_kwargs or {})) # Populate the running replicas if they are already available. if self._running_replicas is not None: @@ -542,8 +543,12 @@ def update_deployment_targets(self, deployment_target_info: DeploymentTargetInfo self._running_replicas_populated = True def update_deployment_config(self, deployment_config: DeploymentConfig): - self._request_router_class = deployment_config.get_request_router_class() - self._request_router_kwargs = deployment_config.request_router_kwargs + self._request_router_class = ( + deployment_config.router_config.get_request_router_class() + ) + self._request_router_kwargs = ( + deployment_config.router_config.request_router_kwargs + ) self._metrics_manager.update_deployment_config( deployment_config, curr_num_replicas=len(self.request_router.curr_replicas), diff --git a/python/ray/serve/api.py b/python/ray/serve/api.py index 4788f8304a00..9adbb40e809f 100644 --- a/python/ray/serve/api.py +++ b/python/ray/serve/api.py @@ -28,7 +28,6 @@ ) from ray.serve._private.local_testing_mode import make_local_deployment_handle from ray.serve._private.logging_utils import configure_component_logger -from ray.serve._private.request_router.request_router import RequestRouter from ray.serve._private.usage import ServeUsageTag from ray.serve._private.utils import ( DEFAULT, @@ -43,6 +42,7 @@ DeploymentMode, HTTPOptions, ProxyLocation, + RouterConfig, gRPCOptions, ) from ray.serve.context import ( @@ -334,10 +334,7 @@ def deployment( health_check_period_s: Default[float] = DEFAULT.VALUE, health_check_timeout_s: Default[float] = DEFAULT.VALUE, logging_config: Default[Union[Dict, LoggingConfig, None]] = DEFAULT.VALUE, - request_router_class: Default[Union[str, RequestRouter, None]] = DEFAULT.VALUE, - request_router_kwargs: Default[Union[Dict, None]] = DEFAULT.VALUE, - request_routing_stats_period_s: Default[float] = DEFAULT.VALUE, - request_routing_stats_timeout_s: Default[float] = DEFAULT.VALUE, + router_config: Default[Union[Dict, RouterConfig, None]] = DEFAULT.VALUE, ) -> Callable[[Callable], Deployment]: """Decorator that converts a Python class to a `Deployment`. @@ -402,22 +399,7 @@ class MyDeployment: check method to return before considering it as failed. Defaults to 30s. logging_config: Logging config options for the deployment. If provided, the config will be used to set up the Serve logger on the deployment. - request_router_class: The class of the request router used for this - deployment. This can be a string or a class. All the deployment - handle created for this deployment will use the routing policy - defined by the request router. Default to Serve's - PowerOfTwoChoicesRequestRouter. - request_router_kwargs: Keyword arguments that will be passed to the - request router class __init__ method. - request_routing_stats_period_s: Duration between record scheduling stats - calls for the replica. Defaults to 10s. The health check is by default a - no-op Actor call to the replica, but you can define your own request - scheduling stats using the "record_scheduling_stats" method in your - deployment. - request_routing_stats_timeout_s: Duration in seconds, that replicas wait for - a request scheduling stats method to return before considering it as failed. - Defaults to 30s. - + router_config: Config for the request router used for this deployment. Returns: `Deployment` """ @@ -482,17 +464,10 @@ class MyDeployment: health_check_period_s=health_check_period_s, health_check_timeout_s=health_check_timeout_s, logging_config=logging_config, - request_routing_stats_period_s=request_routing_stats_period_s, - request_routing_stats_timeout_s=request_routing_stats_timeout_s, + router_config=router_config, ) deployment_config.user_configured_option_names = set(user_configured_option_names) - if request_router_class is not DEFAULT.VALUE: - deployment_config.request_router_class = request_router_class - - if request_router_kwargs is not DEFAULT.VALUE: - deployment_config.request_router_kwargs = request_router_kwargs - def decorator(_func_or_class): replica_config = ReplicaConfig.create( _func_or_class, diff --git a/python/ray/serve/config.py b/python/ray/serve/config.py index c19b537de72a..bcc552e83431 100644 --- a/python/ray/serve/config.py +++ b/python/ray/serve/config.py @@ -1,7 +1,8 @@ +import json import logging import warnings from enum import Enum -from typing import Any, Callable, List, Optional, Union +from typing import Any, Callable, Dict, List, Optional, Union from ray import cloudpickle from ray._common.pydantic_compat import ( @@ -12,6 +13,7 @@ PositiveFloat, PositiveInt, PrivateAttr, + root_validator, validator, ) from ray._common.utils import import_attr @@ -20,6 +22,9 @@ DEFAULT_GRPC_PORT, DEFAULT_HTTP_HOST, DEFAULT_HTTP_PORT, + DEFAULT_REQUEST_ROUTER_PATH, + DEFAULT_REQUEST_ROUTING_STATS_PERIOD_S, + DEFAULT_REQUEST_ROUTING_STATS_TIMEOUT_S, DEFAULT_TARGET_ONGOING_REQUESTS, DEFAULT_UVICORN_KEEP_ALIVE_TIMEOUT_S, SERVE_LOGGER_NAME, @@ -29,6 +34,88 @@ logger = logging.getLogger(SERVE_LOGGER_NAME) +@PublicAPI(stability="stable") +class RouterConfig(BaseModel): + """Config for Serve Router. + + Args: + request_router_class: The import path of the request router if user passed a string. Will be the + concatenation of the request router module and the request router name + if user passed a callable. + """ + + # Cloudpickled request router class. + serialized_request_router_cls: bytes = Field(default=b"") + + # The class of the request router used for this + # deployment. This can be a string or a class. All the deployment + # handle created for this deployment will use the routing policy + # defined by the request router. Default to Serve's PowerOfTwoChoicesRequestRouter. + request_router_class: Union[str, Callable] = Field( + default=DEFAULT_REQUEST_ROUTER_PATH + ) + # Keyword arguments that will be passed to the + # request router class __init__ method. + request_router_kwargs: Dict[str, Any] = Field(default_factory=dict) + + # Duration between record scheduling stats + # calls for the replica. Defaults to 10s. The health check is by default a + # no-op Actor call to the replica, but you can define your own request + # scheduling stats using the "record_scheduling_stats" method in your + # deployment. + request_routing_stats_period_s: PositiveFloat = ( + DEFAULT_REQUEST_ROUTING_STATS_PERIOD_S + ) + + # Duration in seconds, that replicas wait for + # a request scheduling stats method to return before considering it as failed. + # Defaults to 30s. + request_routing_stats_timeout_s: PositiveFloat = ( + DEFAULT_REQUEST_ROUTING_STATS_TIMEOUT_S + ) + + @validator("request_router_kwargs", always=True) + def request_router_kwargs_json_serializable(cls, v): + if isinstance(v, bytes): + return v + if v is not None: + try: + json.dumps(v) + except TypeError as e: + raise ValueError( + f"request_router_kwargs is not JSON-serializable: {str(e)}." + ) + + return v + + @root_validator + def import_and_serialize_request_router_cls(cls, values) -> Dict[str, Any]: + """Import and serialize request router class with cloudpickle. + + Import the request router if it's passed in as a string import path. + Then cloudpickle the request router and set to + `serialized_request_router_cls`. + """ + request_router_class = values.get("request_router_class") + if isinstance(request_router_class, Callable): + request_router_class = ( + f"{request_router_class.__module__}.{request_router_class.__name__}" + ) + + request_router_path = request_router_class or DEFAULT_REQUEST_ROUTER_PATH + request_router_class = import_attr(request_router_path) + + values["serialized_request_router_cls"] = cloudpickle.dumps( + request_router_class + ) + values["request_router_class"] = request_router_path + return values + + def get_request_router_class(self) -> Callable: + """Deserialize request router from cloudpickled bytes.""" + return cloudpickle.loads(self.serialized_request_router_cls) + + @PublicAPI(stability="stable") class AutoscalingConfig(BaseModel): """Config for the Serve Autoscaler.""" diff --git a/python/ray/serve/deployment.py b/python/ray/serve/deployment.py index 52358ae2628c..88a7ed440230 100644 --- a/python/ray/serve/deployment.py +++ b/python/ray/serve/deployment.py @@ -457,8 +457,7 @@ def deployment_to_schema(d: Deployment) -> DeploymentSchema: "placement_group_bundles": d._replica_config.placement_group_bundles, "max_replicas_per_node": d._replica_config.max_replicas_per_node, "logging_config": d._deployment_config.logging_config, - "request_routing_stats_period_s": d._deployment_config.request_routing_stats_period_s, - "request_routing_stats_timeout_s": d._deployment_config.request_routing_stats_timeout_s, + "router_config": d._deployment_config.router_config, } # Let non-user-configured options be set to defaults. If the schema @@ -519,8 +518,7 @@ def schema_to_deployment(s: DeploymentSchema) -> Deployment: health_check_period_s=s.health_check_period_s, health_check_timeout_s=s.health_check_timeout_s, logging_config=s.logging_config, - request_routing_stats_period_s=s.request_routing_stats_period_s, - request_routing_stats_timeout_s=s.request_routing_stats_timeout_s, + router_config=s.router_config, ) deployment_config.user_configured_option_names = ( s._get_user_configured_option_names() diff --git a/python/ray/serve/schema.py b/python/ray/serve/schema.py index 040477aced73..aeceea018939 100644 --- a/python/ray/serve/schema.py +++ b/python/ray/serve/schema.py @@ -33,7 +33,7 @@ ) from ray.serve._private.deployment_info import DeploymentInfo from ray.serve._private.utils import DEFAULT -from ray.serve.config import ProxyLocation +from ray.serve.config import ProxyLocation, RouterConfig from ray.util.annotations import PublicAPI # Shared amongst multiple schemas. @@ -405,29 +405,9 @@ class DeploymentSchema(BaseModel, allow_population_by_field_name=True): default=DEFAULT.VALUE, description="Logging config for configuring serve deployment logs.", ) - request_router_class: str = Field( + router_config: Union[Dict, RouterConfig] = Field( default=DEFAULT.VALUE, - description="The path pointing to the custom request router class to use for this deployment.", - ) - request_router_kwargs: Dict[str, Any] = Field( - default=DEFAULT.VALUE, - description="Keyword arguments that will be passed to the request router class __init__ method.", - ) - request_routing_stats_period_s: float = Field( - default=DEFAULT.VALUE, - description=( - "Frequency at which the controller will record routing stats " - "replicas. Uses a default if null." - ), - gt=0, - ) - request_routing_stats_timeout_s: float = Field( - default=DEFAULT.VALUE, - description=( - "Timeout that the controller will wait for a response " - "from the replica's record routing stats. Uses a default if null." - ), - gt=0, + description="Config for the request router used for this deployment.", ) @root_validator @@ -507,9 +487,7 @@ def _deployment_info_to_schema(name: str, info: DeploymentInfo) -> DeploymentSch health_check_period_s=info.deployment_config.health_check_period_s, health_check_timeout_s=info.deployment_config.health_check_timeout_s, ray_actor_options=info.replica_config.ray_actor_options, - request_router_class=info.deployment_config.request_router_class, - request_routing_stats_period_s=info.deployment_config.request_routing_stats_period_s, - request_routing_stats_timeout_s=info.deployment_config.request_routing_stats_timeout_s, + router_config=info.deployment_config.router_config, ) if info.deployment_config.autoscaling_config is not None: @@ -1207,15 +1185,20 @@ def _get_user_facing_json_serializable_dict( """Generates json serializable dictionary with user facing data.""" values = super().dict(*args, **kwargs) - # `serialized_policy_def` and `serialized_request_router_cls` are only used + # `serialized_policy_def` and internal router config fields are only used # internally and should not be exposed to the REST api. This method iteratively - # removes them from each deployment and autoscaling config if exists. + # removes them from each deployment config if exists. for app_name, application in values["applications"].items(): for deployment_name, deployment in application["deployments"].items(): if "deployment_config" in deployment: - deployment["deployment_config"].pop( - "serialized_request_router_cls", None - ) + # Remove internal fields from router_config if it exists + if "router_config" in deployment["deployment_config"]: + if isinstance( + deployment["deployment_config"]["router_config"], dict + ): + deployment["deployment_config"]["router_config"].pop( + "serialized_request_router_cls", None + ) if "autoscaling_config" in deployment["deployment_config"]: deployment["deployment_config"]["autoscaling_config"].pop( "_serialized_policy_def", None diff --git a/src/ray/protobuf/serve.proto b/src/ray/protobuf/serve.proto index bfbd59113dbf..c601c79fed97 100644 --- a/src/ray/protobuf/serve.proto +++ b/src/ray/protobuf/serve.proto @@ -91,6 +91,28 @@ message LoggingConfig { //[End] Logging Config + +//[Begin] ROUTING CONFIG +message RouterConfig { + // Cloudpickled request router definition. + bytes serialized_request_router_cls = 1; + + // The import path of the request router if user passed a string. Will be the + // concatenation of the request router module and the request router name + // if user passed a callable. + string request_router_class = 2; + + // Frequency at which the controller records routing stats for a replica. + double request_routing_stats_period_s = 3; + + // Timeout after which a replica started a record routing stats without a response. + double request_routing_stats_timeout_s = 4; + + // kwargs which will be passed to the router class' __init__ method + bytes request_router_kwargs = 5; +} +//[End] ROUTING CONFIG + // Configuration options for a deployment, to be set by the user. message DeploymentConfig { // The number of processes to start up that will handle requests to this deployment. @@ -135,22 +157,8 @@ message DeploymentConfig { LoggingConfig logging_config = 14; - // Cloudpickled request router definition. - bytes serialized_request_router_cls = 15; - - // The import path of the request router if user passed a string. Will be the - // concatenation of the request router module and the request router name - // if user passed a callable. - string request_router_class = 16; - - // Frequency at which the controller records routing stats for a replica. - double request_routing_stats_period_s = 17; - - // Timeout after which a replica started a record routing stats without a response. - double request_routing_stats_timeout_s = 18; - - // kwargs which will be passed to the router class' __init__ method - bytes request_router_kwargs = 19; + // The deployment's routing configuration. + RouterConfig router_config = 19; } // Deployment language. From e08e3721b64e24d1f07fecbb9352e23fd87837fe Mon Sep 17 00:00:00 2001 From: Seiji Eicher Date: Wed, 25 Jun 2025 13:52:09 -0700 Subject: [PATCH 12/36] Remove excess whitespace from serve.proto Signed-off-by: Seiji Eicher --- src/ray/protobuf/serve.proto | 1 - 1 file changed, 1 deletion(-) diff --git a/src/ray/protobuf/serve.proto b/src/ray/protobuf/serve.proto index c601c79fed97..d270a4ba3d09 100644 --- a/src/ray/protobuf/serve.proto +++ b/src/ray/protobuf/serve.proto @@ -91,7 +91,6 @@ message LoggingConfig { //[End] Logging Config - //[Begin] ROUTING CONFIG message RouterConfig { // Cloudpickled request router definition. From 3b01a124cebb0a163f79402b93a90a504f657914 Mon Sep 17 00:00:00 2001 From: Seiji Eicher Date: Thu, 26 Jun 2025 15:12:09 -0700 Subject: [PATCH 13/36] Fix java files Signed-off-by: Seiji Eicher --- .../io/ray/serve/config/DeploymentConfig.java | 38 ++++++++++--------- .../io/ray/serve/config/RouterConfig.java | 38 +++++++++++++++++++ 2 files changed, 59 insertions(+), 17 deletions(-) create mode 100644 java/serve/src/main/java/io/ray/serve/config/RouterConfig.java diff --git a/java/serve/src/main/java/io/ray/serve/config/DeploymentConfig.java b/java/serve/src/main/java/io/ray/serve/config/DeploymentConfig.java index 2a86cb2d3afc..629809ef6c81 100644 --- a/java/serve/src/main/java/io/ray/serve/config/DeploymentConfig.java +++ b/java/serve/src/main/java/io/ray/serve/config/DeploymentConfig.java @@ -52,17 +52,10 @@ public class DeploymentConfig implements Serializable { */ private Double healthCheckTimeoutS = Constants.DEFAULT_HEALTH_CHECK_TIMEOUT_S; - /** Frequency at which the controller will record request routing stats. */ - private Double requestRoutingStatsPeriodS = Constants.DEFAULT_REQUEST_ROUTING_STATS_PERIOD_S; - - /** - * Timeout that the controller will wait for a response from the replica's request routing stats - * before retrying. - */ - private Double requestRoutingStatsTimeoutS = Constants.DEFAULT_REQUEST_ROUTING_STATS_TIMEOUT_S; - private AutoscalingConfig autoscalingConfig; + private RouterConfig routerConfig; + /** This flag is used to let replica know they are deplyed from a different language. */ private Boolean isCrossLanguage = false; @@ -150,23 +143,23 @@ public DeploymentConfig setHealthCheckTimeoutS(Double healthCheckTimeoutS) { } public Double getRequestRoutingStatsPeriodS() { - return requestRoutingStatsPeriodS; + return routerConfig.getRequestRoutingStatsPeriodS(); } public DeploymentConfig setRequestRoutingStatsPeriodS(Double requestRoutingStatsPeriodS) { if (requestRoutingStatsPeriodS != null) { - this.requestRoutingStatsPeriodS = requestRoutingStatsPeriodS; + routerConfig.setRequestRoutingStatsPeriodS(requestRoutingStatsPeriodS); } return this; } public Double getRequestRoutingStatsTimeoutS() { - return requestRoutingStatsTimeoutS; + return routerConfig.getRequestRoutingStatsTimeoutS(); } public DeploymentConfig setRequestRoutingStatsTimeoutS(Double requestRoutingStatsTimeoutS) { if (requestRoutingStatsTimeoutS != null) { - this.requestRoutingStatsTimeoutS = requestRoutingStatsTimeoutS; + routerConfig.setRequestRoutingStatsTimeoutS(requestRoutingStatsTimeoutS); } return this; } @@ -180,6 +173,15 @@ public DeploymentConfig setAutoscalingConfig(AutoscalingConfig autoscalingConfig return this; } + public RouterConfig getRouterConfig() { + return routerConfig; + } + + public DeploymentConfig setRouterConfig(RouterConfig routerConfig) { + this.routerConfig = routerConfig; + return this; + } + public boolean isCrossLanguage() { return isCrossLanguage; } @@ -230,8 +232,6 @@ public byte[] toProtoBytes() { .setGracefulShutdownTimeoutS(gracefulShutdownTimeoutS) .setHealthCheckPeriodS(healthCheckPeriodS) .setHealthCheckTimeoutS(healthCheckTimeoutS) - .setRequestRoutingStatsPeriodS(requestRoutingStatsPeriodS) - .setRequestRoutingStatsTimeoutS(requestRoutingStatsTimeoutS) .setIsCrossLanguage(isCrossLanguage) .setDeploymentLanguage(deploymentLanguage) .setVersion(version); @@ -241,6 +241,9 @@ public byte[] toProtoBytes() { if (null != autoscalingConfig) { builder.setAutoscalingConfig(autoscalingConfig.toProto()); } + if (null != routerConfig) { + builder.setRouterConfig(routerConfig.toProto()); + } return builder.build().toByteArray(); } @@ -253,8 +256,6 @@ public io.ray.serve.generated.DeploymentConfig toProto() { .setGracefulShutdownTimeoutS(gracefulShutdownTimeoutS) .setHealthCheckPeriodS(healthCheckPeriodS) .setHealthCheckTimeoutS(healthCheckTimeoutS) - .setRequestRoutingStatsPeriodS(requestRoutingStatsPeriodS) - .setRequestRoutingStatsTimeoutS(requestRoutingStatsTimeoutS) .setIsCrossLanguage(isCrossLanguage) .setDeploymentLanguage(deploymentLanguage); if (null != userConfig) { @@ -263,6 +264,9 @@ public io.ray.serve.generated.DeploymentConfig toProto() { if (null != autoscalingConfig) { builder.setAutoscalingConfig(autoscalingConfig.toProto()); } + if (null != routerConfig) { + builder.setRouterConfig(routerConfig.toProto()); + } return builder.build(); } diff --git a/java/serve/src/main/java/io/ray/serve/config/RouterConfig.java b/java/serve/src/main/java/io/ray/serve/config/RouterConfig.java new file mode 100644 index 000000000000..3f9b868c23e5 --- /dev/null +++ b/java/serve/src/main/java/io/ray/serve/config/RouterConfig.java @@ -0,0 +1,38 @@ +package io.ray.serve.config; + +import io.ray.serve.common.Constants; +import java.io.Serializable; + +public class RouterConfig implements Serializable { + /** Frequency at which the controller will record request routing stats. */ + private Double requestRoutingStatsPeriodS = Constants.DEFAULT_REQUEST_ROUTING_STATS_PERIOD_S; + + /** + * Timeout that the controller will wait for a response from the replica's request routing stats + * before retrying. + */ + private Double requestRoutingStatsTimeoutS = Constants.DEFAULT_REQUEST_ROUTING_STATS_TIMEOUT_S; + + public Double getRequestRoutingStatsPeriodS() { + return requestRoutingStatsPeriodS; + } + + public Double getRequestRoutingStatsTimeoutS() { + return requestRoutingStatsTimeoutS; + } + + public void setRequestRoutingStatsPeriodS(Double requestRoutingStatsPeriodS) { + this.requestRoutingStatsPeriodS = requestRoutingStatsPeriodS; + } + + public void setRequestRoutingStatsTimeoutS(Double requestRoutingStatsTimeoutS) { + this.requestRoutingStatsTimeoutS = requestRoutingStatsTimeoutS; + } + + public io.ray.serve.generated.RouterConfig toProto() { + return io.ray.serve.generated.RouterConfig.newBuilder() + .setRequestRoutingStatsPeriodS(requestRoutingStatsPeriodS) + .setRequestRoutingStatsTimeoutS(requestRoutingStatsTimeoutS) + .build(); + } +} From f5444c8720e33b7cffe03ccdee40623651322f7b Mon Sep 17 00:00:00 2001 From: Seiji Eicher Date: Thu, 26 Jun 2025 17:22:27 -0700 Subject: [PATCH 14/36] Pickle/unpickle request_router_kwargs Signed-off-by: Seiji Eicher --- python/ray/serve/_private/config.py | 37 ++++++++++++++++++++++------- 1 file changed, 28 insertions(+), 9 deletions(-) diff --git a/python/ray/serve/_private/config.py b/python/ray/serve/_private/config.py index 4580f2d7ce4e..36b08697e729 100644 --- a/python/ray/serve/_private/config.py +++ b/python/ray/serve/_private/config.py @@ -248,6 +248,11 @@ def to_proto(self): **data["autoscaling_config"] ) if data.get("router_config"): + if data.get("router_config").get("request_router_kwargs") is not None: + if self.needs_pickle(): + data["router_config"]["request_router_kwargs"] = cloudpickle.dumps( + data["router_config"]["request_router_kwargs"] + ) data["router_config"] = RouterConfigProto(**data["router_config"]) if data.get("logging_config"): if "encoding" in data["logging_config"]: @@ -266,17 +271,17 @@ def to_proto_bytes(self): @classmethod def from_proto(cls, proto: DeploymentConfigProto): data = _proto_to_dict(proto) + deployment_language = ( + data["deployment_language"] + if "deployment_language" in data + else DeploymentLanguage.PYTHON + ) + is_cross_language = ( + data["is_cross_language"] if "is_cross_language" in data else False + ) + needs_pickle = _needs_pickle(deployment_language, is_cross_language) if "user_config" in data: if data["user_config"] != b"": - deployment_language = ( - data["deployment_language"] - if "deployment_language" in data - else DeploymentLanguage.PYTHON - ) - is_cross_language = ( - data["is_cross_language"] if "is_cross_language" in data else False - ) - needs_pickle = _needs_pickle(deployment_language, is_cross_language) if needs_pickle: data["user_config"] = cloudpickle.loads(proto.user_config) else: @@ -284,6 +289,20 @@ def from_proto(cls, proto: DeploymentConfigProto): else: data["user_config"] = None if "router_config" in data: + if "request_router_kwargs" in data["router_config"]: + request_router_kwargs = data["router_config"]["request_router_kwargs"] + if request_router_kwargs != b"": + if needs_pickle: + data["router_config"][ + "request_router_kwargs" + ] = cloudpickle.loads(proto.router_config.request_router_kwargs) + else: + data["router_config"][ + "request_router_kwargs" + ] = proto.router_config.request_router_kwargs + else: + data["router_config"]["request_router_kwargs"] = {} + data["router_config"] = RouterConfig(**data["router_config"]) if "autoscaling_config" in data: if not data["autoscaling_config"].get("upscale_smoothing_factor"): From 3c403e26c4d99ce417d9c797cf14cd5958306c74 Mon Sep 17 00:00:00 2001 From: Seiji Eicher Date: Mon, 30 Jun 2025 19:05:15 +0000 Subject: [PATCH 15/36] Add to API .rst, don't serialize bytes, update tests Signed-off-by: Seiji Eicher --- doc/source/serve/api/index.md | 1 + python/ray/serve/_private/deployment_state.py | 4 ++-- python/ray/serve/_private/version.py | 5 +++++ python/ray/serve/tests/test_api.py | 3 ++- python/ray/serve/tests/test_record_routing_stats.py | 5 ++++- 5 files changed, 14 insertions(+), 4 deletions(-) diff --git a/doc/source/serve/api/index.md b/doc/source/serve/api/index.md index c63c1d95cf3b..9c3063e1ae06 100644 --- a/doc/source/serve/api/index.md +++ b/doc/source/serve/api/index.md @@ -83,6 +83,7 @@ See the [model composition guide](serve-model-composition) for how to update cod serve.config.gRPCOptions serve.config.HTTPOptions serve.config.AutoscalingConfig + serve.config.RouterConfig ``` ### Schemas diff --git a/python/ray/serve/_private/deployment_state.py b/python/ray/serve/_private/deployment_state.py index 1d5cdbe07630..2d8e0019517b 100644 --- a/python/ray/serve/_private/deployment_state.py +++ b/python/ray/serve/_private/deployment_state.py @@ -359,11 +359,11 @@ def health_check_timeout_s(self) -> float: @property def request_routing_stats_period_s(self) -> float: - return self.deployment_config.request_routing_stats_period_s + return self.deployment_config.router_config.request_routing_stats_period_s @property def request_routing_stats_timeout_s(self) -> float: - return self.deployment_config.request_routing_stats_timeout_s + return self.deployment_config.router_config.request_routing_stats_timeout_s @property def pid(self) -> Optional[int]: diff --git a/python/ray/serve/_private/version.py b/python/ray/serve/_private/version.py index 1068d5b71cb2..3b264f6467b6 100644 --- a/python/ray/serve/_private/version.py +++ b/python/ray/serve/_private/version.py @@ -186,6 +186,11 @@ def _get_serialized_options( elif isinstance(reconfigure_dict[option_name], BaseModel): reconfigure_dict[option_name] = reconfigure_dict[option_name].dict() + # Can't serialize bytes. The request router class is already + # included in the serialized config as request_router_class. + if "router_config" in reconfigure_dict: + del reconfigure_dict["router_config"]["serialized_request_router_cls"] + if ( isinstance(self.deployment_config.user_config, bytes) and "user_config" in reconfigure_dict diff --git a/python/ray/serve/tests/test_api.py b/python/ray/serve/tests/test_api.py index 35e671183f20..bd7313ae9ff5 100644 --- a/python/ray/serve/tests/test_api.py +++ b/python/ray/serve/tests/test_api.py @@ -28,6 +28,7 @@ RequestRouter, ) from ray.serve._private.test_utils import get_application_url +from ray.serve.config import RouterConfig from ray.serve.deployment import Application from ray.serve.exceptions import RayServeException from ray.serve.handle import DeploymentHandle @@ -81,7 +82,7 @@ async def choose_replicas( return [candidate_replicas] -@serve.deployment(request_router_class=FakeRequestRouter) +@serve.deployment(router_config=RouterConfig(request_router_class=FakeRequestRouter)) class AppWithCustomRequestRouter: def __call__(self) -> str: return "Hello, world!" diff --git a/python/ray/serve/tests/test_record_routing_stats.py b/python/ray/serve/tests/test_record_routing_stats.py index fae97c6f22db..3e37443c1b5f 100644 --- a/python/ray/serve/tests/test_record_routing_stats.py +++ b/python/ray/serve/tests/test_record_routing_stats.py @@ -7,12 +7,15 @@ from ray import serve from ray._common.test_utils import wait_for_condition from ray.serve._private.common import ReplicaID +from ray.serve.config import RouterConfig from ray.serve.context import _get_internal_replica_context from ray.serve.handle import DeploymentHandle @serve.deployment( - request_routing_stats_period_s=0.1, request_routing_stats_timeout_s=0.1 + router_config=RouterConfig( + request_routing_stats_period_s=0.1, request_routing_stats_timeout_s=0.1 + ) ) class Patient: def __init__(self): From 13db8dd5ebf487cc213ddee10c448c278f983c58 Mon Sep 17 00:00:00 2001 From: Seiji Eicher Date: Mon, 30 Jun 2025 19:07:40 +0000 Subject: [PATCH 16/36] Update comments Signed-off-by: Seiji Eicher --- python/ray/serve/_private/request_router/request_router.py | 6 +++--- python/ray/serve/config.py | 2 +- src/ray/protobuf/serve.proto | 2 +- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/python/ray/serve/_private/request_router/request_router.py b/python/ray/serve/_private/request_router/request_router.py index 85c7aa8ca9cb..c502aac583ab 100644 --- a/python/ray/serve/_private/request_router/request_router.py +++ b/python/ray/serve/_private/request_router/request_router.py @@ -535,9 +535,9 @@ def __init__( self.num_routing_tasks_in_backoff_gauge.set(self.num_routing_tasks_in_backoff) def initialize_state(self, **kwargs): - """Initialize the state of the request router. - Called at the end of RequestRouter.__init__ with request_router_kwargs - from DeploymentConfig . + """ + Initialize the state of the request router. Called by framework with the + contents of RequestRouter.request_router_kwargs """ pass diff --git a/python/ray/serve/config.py b/python/ray/serve/config.py index bcc552e83431..f4266bcaae93 100644 --- a/python/ray/serve/config.py +++ b/python/ray/serve/config.py @@ -55,7 +55,7 @@ class RouterConfig(BaseModel): default=DEFAULT_REQUEST_ROUTER_PATH ) # Keyword arguments that will be passed to the - # request router class __init__ method. + # request router class initialize_state method. request_router_kwargs: Dict[str, Any] = Field(default_factory=dict) # Duration between record scheduling stats diff --git a/src/ray/protobuf/serve.proto b/src/ray/protobuf/serve.proto index d270a4ba3d09..7f36f0810b25 100644 --- a/src/ray/protobuf/serve.proto +++ b/src/ray/protobuf/serve.proto @@ -107,7 +107,7 @@ message RouterConfig { // Timeout after which a replica started a record routing stats without a response. double request_routing_stats_timeout_s = 4; - // kwargs which will be passed to the router class' __init__ method + // kwargs which will be passed to the router class' initialize_state method bytes request_router_kwargs = 5; } //[End] ROUTING CONFIG From b7b3c1d286b22d936ca48cd9938f010f8047e39f Mon Sep 17 00:00:00 2001 From: Seiji Eicher Date: Mon, 30 Jun 2025 23:24:53 +0000 Subject: [PATCH 17/36] Fix tests to use RouterConfig, document attributes Signed-off-by: Seiji Eicher --- .../doc_code/custom_request_router_app.py | 5 +- python/ray/serve/config.py | 92 +++++++++++-------- python/ray/serve/schema.py | 9 +- python/ray/serve/tests/test_controller.py | 9 +- python/ray/serve/tests/test_telemetry_2.py | 5 +- python/ray/serve/tests/unit/test_config.py | 27 ++++-- 6 files changed, 92 insertions(+), 55 deletions(-) diff --git a/doc/source/serve/doc_code/custom_request_router_app.py b/doc/source/serve/doc_code/custom_request_router_app.py index e88280f5d1bb..fdb840d977d3 100644 --- a/doc/source/serve/doc_code/custom_request_router_app.py +++ b/doc/source/serve/doc_code/custom_request_router_app.py @@ -35,6 +35,7 @@ async def __call__(self): from ray import serve from ray.serve.context import _get_internal_replica_context from typing import Any, Dict +from ray.serve.config import RouterConfig def _time_ms() -> int: @@ -42,7 +43,9 @@ def _time_ms() -> int: @serve.deployment( - request_router_class="custom_request_router:ThroughputAwareRequestRouter", + router_config=RouterConfig( + request_router_class="custom_request_router:ThroughputAwareRequestRouter", + ), num_replicas=3, request_routing_stats_period_s=1, request_routing_stats_timeout_s=1, diff --git a/python/ray/serve/config.py b/python/ray/serve/config.py index f4266bcaae93..9b9985c4c56f 100644 --- a/python/ray/serve/config.py +++ b/python/ray/serve/config.py @@ -44,34 +44,43 @@ class RouterConfig(BaseModel): if user passed a callable. """ - # Cloudpickled request router class. - serialized_request_router_cls: bytes = Field(default=b"") + serialized_request_router_cls: bytes = Field( + default=b"", description="Cloudpickled request router class." + ) - # The class of the request router used for this - # deployment. This can be a string or a class. All the deployment - # handle created for this deployment will use the routing policy - # defined by the request router. Default to Serve's PowerOfTwoChoicesRequestRouter. request_router_class: Union[str, Callable] = Field( - default=DEFAULT_REQUEST_ROUTER_PATH + default=DEFAULT_REQUEST_ROUTER_PATH, + description=( + "The class of the request router used for this deployment. This can be " + "a string or a class. All the deployment handle created for this " + "deployment will use the routing policy defined by the request router. " + "Default to Serve's PowerOfTwoChoicesRequestRouter." + ), + ) + request_router_kwargs: Dict[str, Any] = Field( + default_factory=dict, + description=( + "Keyword arguments that will be passed to the request router class " + "initialize_state method." + ), ) - # Keyword arguments that will be passed to the - # request router class initialize_state method. - request_router_kwargs: Dict[str, Any] = Field(default_factory=dict) - - # Duration between record scheduling stats - # calls for the replica. Defaults to 10s. The health check is by default a - # no-op Actor call to the replica, but you can define your own request - # scheduling stats using the "record_scheduling_stats" method in your - # deployment. - request_routing_stats_period_s: PositiveFloat = ( - DEFAULT_REQUEST_ROUTING_STATS_PERIOD_S + + request_routing_stats_period_s: PositiveFloat = Field( + default=DEFAULT_REQUEST_ROUTING_STATS_PERIOD_S, + description=( + "Duration between record scheduling stats calls for the replica. " + "Defaults to 10s. The health check is by default a no-op Actor call " + "to the replica, but you can define your own request scheduling stats " + "using the 'record_scheduling_stats' method in your deployment." + ), ) - # Duration in seconds, that replicas wait for - # a request scheduling stats method to return before considering it as failed. - # Defaults to 30s. - request_routing_stats_timeout_s: PositiveFloat = ( - DEFAULT_REQUEST_ROUTING_STATS_TIMEOUT_S + request_routing_stats_timeout_s: PositiveFloat = Field( + default=DEFAULT_REQUEST_ROUTING_STATS_TIMEOUT_S, + description=( + "Duration in seconds, that replicas wait for a request scheduling " + "stats method to return before considering it as failed. Defaults to 30s." + ), ) @validator("request_router_kwargs", always=True) @@ -130,13 +139,17 @@ class AutoscalingConfig(BaseModel): target_ongoing_requests: PositiveFloat = DEFAULT_TARGET_ONGOING_REQUESTS - # How often to scrape for metrics - metrics_interval_s: PositiveFloat = 10.0 - # Time window to average over for metrics. - look_back_period_s: PositiveFloat = 30.0 + metrics_interval_s: PositiveFloat = Field( + default=10.0, description="How often to scrape for metrics." + ) + look_back_period_s: PositiveFloat = Field( + default=30.0, description="Time window to average over for metrics." + ) - # DEPRECATED - smoothing_factor: PositiveFloat = 1.0 + smoothing_factor: PositiveFloat = Field( + default=1.0, + description="[DEPRECATED] Smoothing factor for autoscaling decisions.", + ) # DEPRECATED: replaced by `downscaling_factor` upscale_smoothing_factor: Optional[PositiveFloat] = Field( default=None, description="[DEPRECATED] Please use `upscaling_factor` instead." @@ -147,16 +160,23 @@ class AutoscalingConfig(BaseModel): description="[DEPRECATED] Please use `downscaling_factor` instead.", ) - # Multiplicative "gain" factor to limit scaling decisions - upscaling_factor: Optional[PositiveFloat] = None - downscaling_factor: Optional[PositiveFloat] = None + upscaling_factor: Optional[PositiveFloat] = Field( + default=None, + description='Multiplicative "gain" factor to limit upscaling decisions.', + ) + downscaling_factor: Optional[PositiveFloat] = Field( + default=None, + description='Multiplicative "gain" factor to limit downscaling decisions.', + ) # How frequently to make autoscaling decisions # loop_period_s: float = CONTROL_LOOP_PERIOD_S - # How long to wait before scaling down replicas - downscale_delay_s: NonNegativeFloat = 600.0 - # How long to wait before scaling up replicas - upscale_delay_s: NonNegativeFloat = 30.0 + downscale_delay_s: NonNegativeFloat = Field( + default=600.0, description="How long to wait before scaling down replicas." + ) + upscale_delay_s: NonNegativeFloat = Field( + default=30.0, description="How long to wait before scaling up replicas." + ) # Cloudpickled policy definition. _serialized_policy_def: bytes = PrivateAttr(default=b"") diff --git a/python/ray/serve/schema.py b/python/ray/serve/schema.py index aeceea018939..c92666219d23 100644 --- a/python/ray/serve/schema.py +++ b/python/ray/serve/schema.py @@ -1193,12 +1193,9 @@ def _get_user_facing_json_serializable_dict( if "deployment_config" in deployment: # Remove internal fields from router_config if it exists if "router_config" in deployment["deployment_config"]: - if isinstance( - deployment["deployment_config"]["router_config"], dict - ): - deployment["deployment_config"]["router_config"].pop( - "serialized_request_router_cls", None - ) + deployment["deployment_config"]["router_config"].pop( + "serialized_request_router_cls", None + ) if "autoscaling_config" in deployment["deployment_config"]: deployment["deployment_config"]["autoscaling_config"].pop( "_serialized_policy_def", None diff --git a/python/ray/serve/tests/test_controller.py b/python/ray/serve/tests/test_controller.py index eb14d6c0da40..ee77151ceb37 100644 --- a/python/ray/serve/tests/test_controller.py +++ b/python/ray/serve/tests/test_controller.py @@ -185,9 +185,12 @@ def autoscaling_app(): "ray_actor_options": { "num_cpus": 1.0, }, - "request_router_class": "ray.serve._private.request_router:PowerOfTwoChoicesRequestRouter", - "request_routing_stats_period_s": 10.0, - "request_routing_stats_timeout_s": 30.0, + "router_config": { + "request_router_class": "ray.serve._private.request_router:PowerOfTwoChoicesRequestRouter", + "request_router_kwargs": {}, + "request_routing_stats_period_s": 10.0, + "request_routing_stats_timeout_s": 30.0, + }, }, "target_num_replicas": 1, "required_resources": {"CPU": 1}, diff --git a/python/ray/serve/tests/test_telemetry_2.py b/python/ray/serve/tests/test_telemetry_2.py index 55f4eda95847..bda1f85ec02e 100644 --- a/python/ray/serve/tests/test_telemetry_2.py +++ b/python/ray/serve/tests/test_telemetry_2.py @@ -17,6 +17,7 @@ ) from ray.serve._private.test_utils import check_apps_running, check_telemetry from ray.serve._private.usage import ServeUsageTag +from ray.serve.config import RouterConfig from ray.serve.context import _get_global_client from ray.serve.schema import ServeDeploySchema @@ -159,7 +160,9 @@ def test_custom_request_router_telemetry(manage_ray_with_telemetry): check_telemetry(ServeUsageTag.CUSTOM_REQUEST_ROUTER_USED, expected=None) @serve.deployment( - request_router_class=CustomRequestRouter, + router_config=RouterConfig( + request_router_class=CustomRequestRouter, + ), ) class CustomRequestRouterApp: async def __call__(self) -> str: diff --git a/python/ray/serve/tests/unit/test_config.py b/python/ray/serve/tests/unit/test_config.py index a1951f319455..e200ed3d8d40 100644 --- a/python/ray/serve/tests/unit/test_config.py +++ b/python/ray/serve/tests/unit/test_config.py @@ -15,6 +15,7 @@ DeploymentMode, HTTPOptions, ProxyLocation, + RouterConfig, gRPCOptions, ) from ray.serve.generated.serve_pb2 import ( @@ -144,27 +145,37 @@ def test_setting_and_getting_request_router_class(self): # Passing request_router_class as a class. deployment_config = DeploymentConfig.from_default( - request_router_class=FakeRequestRouter + router_config=RouterConfig(request_router_class=FakeRequestRouter) + ) + assert ( + deployment_config.router_config.request_router_class == request_router_path + ) + assert ( + deployment_config.router_config.get_request_router_class() + == FakeRequestRouter ) - assert deployment_config.request_router_class == request_router_path - assert deployment_config.get_request_router_class() == FakeRequestRouter # Passing request_router_class as an import path. deployment_config = DeploymentConfig.from_default( - request_router_class=request_router_path + router_config=RouterConfig(request_router_class=request_router_path) + ) + assert ( + deployment_config.router_config.request_router_class == request_router_path + ) + assert ( + deployment_config.router_config.get_request_router_class() + == FakeRequestRouter ) - assert deployment_config.request_router_class == request_router_path - assert deployment_config.get_request_router_class() == FakeRequestRouter # Not passing request_router_class should # default to `PowerOfTwoChoicesRequestRouter`. deployment_config = DeploymentConfig.from_default() assert ( - deployment_config.request_router_class + deployment_config.router_config.request_router_class == "ray.serve._private.request_router:PowerOfTwoChoicesRequestRouter" ) assert ( - deployment_config.get_request_router_class() + deployment_config.router_config.get_request_router_class() == PowerOfTwoChoicesRequestRouter ) From 07462aa2130f2e02d747d1f54d942f010068d8a5 Mon Sep 17 00:00:00 2001 From: Seiji Eicher Date: Mon, 30 Jun 2025 23:26:58 +0000 Subject: [PATCH 18/36] Fix bad rebase Signed-off-by: Seiji Eicher --- .../prefix_aware/prefix_tree.py | 60 ++++++++----------- 1 file changed, 26 insertions(+), 34 deletions(-) diff --git a/python/ray/llm/_internal/serve/request_router/prefix_aware/prefix_tree.py b/python/ray/llm/_internal/serve/request_router/prefix_aware/prefix_tree.py index ceed508f9e3f..b3765e6e6813 100644 --- a/python/ray/llm/_internal/serve/request_router/prefix_aware/prefix_tree.py +++ b/python/ray/llm/_internal/serve/request_router/prefix_aware/prefix_tree.py @@ -1,6 +1,5 @@ from __future__ import annotations -import asyncio import logging import os import threading @@ -97,7 +96,8 @@ def __init__(self) -> None: # LRU tracking - root is always the head, tail is the least recently used. self.tenant_to_lru_tail: Dict[str, Optional[Node]] = {} - self._eviction_task: Optional[asyncio.Task] = None + self._eviction_thread: Optional[threading.Thread] = None + self._eviction_stop_event: threading.Event = threading.Event() @staticmethod def _shared_prefix_count(a: str, b: str) -> int: @@ -140,9 +140,7 @@ def _insert_node_into_linked_list( """ with self.lock: if tenant not in self.tenant_to_char_count: - logger.debug( - f"[_insert_node_into_linked_list] Tenant '{tenant}' does not exist. No action taken." - ) + logger.debug(f"Tenant '{tenant}' does not exist. No action taken.") return # Skip if node is the root @@ -167,9 +165,7 @@ def _remove_node_from_linked_list(self, node: Node, tenant: str) -> None: """ with self.lock: if tenant not in self.tenant_to_char_count: - logger.debug( - f"[_remove_node_from_linked_list] Tenant '{tenant}' does not exist. No action taken." - ) + logger.debug(f"Tenant '{tenant}' does not exist. No action taken.") return # Skip if node is the root @@ -207,13 +203,11 @@ def _remove_tenant_single_node(self, tenant: str, node: Node) -> int: """ with self.lock: if tenant not in self.tenant_to_char_count: - logger.debug( - f"[_remove_tenant_single_node] Tenant '{tenant}' does not exist. No action taken." - ) + logger.debug(f"Tenant '{tenant}' does not exist. No action taken.") return 0 if tenant not in node.tenant_to_last_access_time: logger.debug( - f"[_remove_tenant_single_node] Tenant '{tenant}' does not have node '{node.text}'. No action taken." + f"Tenant '{tenant}' does not have node '{node.text}'. No action taken." ) return 0 @@ -245,9 +239,7 @@ def add_tenants(self, tenants: List[str], time_s: float) -> None: with self.lock: for tenant in tenants: if tenant in self.tenant_to_char_count: - logger.debug( - f"[_add_tenants] Tenant '{tenant}' already exists. Skipping." - ) + logger.debug(f"Tenant '{tenant}' already exists. Skipping.") continue self.tenant_to_char_count[tenant] = 0 @@ -284,7 +276,7 @@ def insert(self, text: str, tenant: str, time_s: float) -> None: with self.lock: if tenant not in self.tenant_to_char_count: logger.debug( - f"[_insert] Tenant '{tenant}' does not exist. Use add_tenants() first." + f"Tenant '{tenant}' does not exist. Use add_tenants() first." ) return @@ -469,9 +461,7 @@ def remove_tenants(self, tenants: List[str]) -> Dict[str, int]: with self.lock: for tenant in tenants: if tenant not in self.tenant_to_char_count: - logger.debug( - f"[_remove_tenants] Tenant '{tenant}' does not exist. Skipping." - ) + logger.debug(f"Tenant '{tenant}' does not exist. Skipping.") chars_removed[tenant] = 0 continue @@ -514,13 +504,13 @@ def evict_tenant_by_lru(self, tenant: str, min_remove_size: int) -> int: with self.lock: if tenant not in self.tenant_to_char_count: logger.debug( - f"[_evict_tenant_by_lru] Cannot evict tenant '{tenant}': tenant does not exist. No action taken." + f"Cannot evict tenant '{tenant}': tenant does not exist. No action taken." ) return 0 if self.tenant_to_char_count[tenant] < min_remove_size: logger.debug( - f"[_evict_tenant_by_lru] Cannot evict {min_remove_size} characters from tenant '{tenant}', which has only " + f"Cannot evict {min_remove_size} characters from tenant '{tenant}', which has only " f"{self.tenant_to_char_count[tenant]} characters. Will remove all available characters." ) min_remove_size = self.tenant_to_char_count[tenant] @@ -574,8 +564,9 @@ def get_smallest_tenants(self) -> Optional[List[str]]: def start_eviction_loop( self, eviction_threshold: int, eviction_target: int, interval_secs: float ) -> bool: - """Start a single eviction loop within the actor itself - Parameters: + """Start a single eviction loop within the actor itself. + + Args: eviction_threshold: Minimum number of characters a tenant must have to be evicted eviction_target: The maximum number of characters a tenant should have after eviction interval_secs: Number of seconds between eviction checks @@ -583,16 +574,18 @@ def start_eviction_loop( Returns: True if the loop was started, False if it was already running """ + self._eviction_stop_event.clear() with self.lock: - if self._eviction_task is None: - self._eviction_task = asyncio.create_task( - self._run_eviction_loop( - eviction_threshold, eviction_target, interval_secs - ) + if self._eviction_thread is None: + self._eviction_thread = threading.Thread( + target=self._run_eviction_loop, + args=(eviction_threshold, eviction_target, interval_secs), + daemon=True, ) + self._eviction_thread.start() return True else: - logger.debug("[_start_eviction_loop] Eviction loop already running") + logger.debug("Eviction loop already running") return False def _run_eviction_loop(self, eviction_threshold, eviction_target, interval_secs): @@ -608,11 +601,10 @@ def _run_eviction_loop(self, eviction_threshold, eviction_target, interval_secs) self.evict_tenant_by_lru(tenant, excess) def stop_eviction_loop(self): - with self.lock: - if self._eviction_task: - self._eviction_task.cancel() - # self._eviction_task.close() - self._eviction_task = None + self._eviction_stop_event.set() + if self._eviction_thread: + self._eviction_thread.join() + self._eviction_thread = None @ray.remote From e408b13ad0d391bf966938f5996bd5e924bd10a2 Mon Sep 17 00:00:00 2001 From: Seiji Eicher Date: Tue, 1 Jul 2025 20:43:04 +0000 Subject: [PATCH 19/36] Fix test to use RouterConfig Signed-off-by: Seiji Eicher --- doc/source/serve/doc_code/custom_request_router_app.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/doc/source/serve/doc_code/custom_request_router_app.py b/doc/source/serve/doc_code/custom_request_router_app.py index fdb840d977d3..181e9e8d9699 100644 --- a/doc/source/serve/doc_code/custom_request_router_app.py +++ b/doc/source/serve/doc_code/custom_request_router_app.py @@ -7,7 +7,9 @@ @serve.deployment( - request_router_class="custom_request_router:UniformRequestRouter", + router_config=RouterConfig( + request_router_class="custom_request_router:UniformRequestRouter", + ), num_replicas=10, ray_actor_options={"num_cpus": 0}, ) From f73df364bd146d0b27cff1ba5f37cda290ac9822 Mon Sep 17 00:00:00 2001 From: Seiji Eicher Date: Wed, 2 Jul 2025 00:43:15 +0000 Subject: [PATCH 20/36] Set router_kwargs to empty bytes in Java Signed-off-by: Seiji Eicher --- python/ray/serve/_private/config.py | 18 +++++++++++++++--- 1 file changed, 15 insertions(+), 3 deletions(-) diff --git a/python/ray/serve/_private/config.py b/python/ray/serve/_private/config.py index 36b08697e729..ef6d1933c2b2 100644 --- a/python/ray/serve/_private/config.py +++ b/python/ray/serve/_private/config.py @@ -248,10 +248,22 @@ def to_proto(self): **data["autoscaling_config"] ) if data.get("router_config"): - if data.get("router_config").get("request_router_kwargs") is not None: - if self.needs_pickle(): + router_kwargs = data.get("router_config").get("request_router_kwargs") + if router_kwargs is not None: + if self.deployment_language == DeploymentLanguage.JAVA: + # Java deployments don't support router kwargs + if isinstance(router_kwargs, dict) and not router_kwargs: + # Empty dict -> empty bytes for protobuf compatibility + data["router_config"]["request_router_kwargs"] = b"" + elif router_kwargs: + raise ValueError( + "request_router_kwargs is not supported for Java deployments. " + f"Got: {router_kwargs}" + ) + elif self.needs_pickle(): + # Python deployments: pickle the router kwargs data["router_config"]["request_router_kwargs"] = cloudpickle.dumps( - data["router_config"]["request_router_kwargs"] + router_kwargs ) data["router_config"] = RouterConfigProto(**data["router_config"]) if data.get("logging_config"): From b415a2f5c51d55ac299e70c178500e85d3f169fe Mon Sep 17 00:00:00 2001 From: Seiji Eicher Date: Tue, 1 Jul 2025 15:26:54 -0700 Subject: [PATCH 21/36] Fix test to use RouterConfig Signed-off-by: Seiji Eicher --- .../serve/doc_code/custom_request_router_app.py | 14 +++++--------- 1 file changed, 5 insertions(+), 9 deletions(-) diff --git a/doc/source/serve/doc_code/custom_request_router_app.py b/doc/source/serve/doc_code/custom_request_router_app.py index 181e9e8d9699..ec25b9b34e61 100644 --- a/doc/source/serve/doc_code/custom_request_router_app.py +++ b/doc/source/serve/doc_code/custom_request_router_app.py @@ -2,8 +2,12 @@ # __begin_deploy_app_with_uniform_request_router__ from ray import serve -from ray.serve.context import _get_internal_replica_context from ray.serve.request_router import ReplicaID +import time +from collections import defaultdict +from ray.serve.context import _get_internal_replica_context +from typing import Any, Dict +from ray.serve.config import RouterConfig @serve.deployment( @@ -32,14 +36,6 @@ async def __call__(self): # __begin_deploy_app_with_throughput_aware_request_router__ -import time -from collections import defaultdict -from ray import serve -from ray.serve.context import _get_internal_replica_context -from typing import Any, Dict -from ray.serve.config import RouterConfig - - def _time_ms() -> int: return int(time.time() * 1000) From 6f776247613c91c6ff19ed8011eb867b8f1a0880 Mon Sep 17 00:00:00 2001 From: Seiji Eicher Date: Wed, 2 Jul 2025 00:51:31 +0000 Subject: [PATCH 22/36] Fix ThroughputAwareRequestRouterApp Signed-off-by: Seiji Eicher --- doc/source/serve/doc_code/custom_request_router_app.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/doc/source/serve/doc_code/custom_request_router_app.py b/doc/source/serve/doc_code/custom_request_router_app.py index ec25b9b34e61..1559bd209d6d 100644 --- a/doc/source/serve/doc_code/custom_request_router_app.py +++ b/doc/source/serve/doc_code/custom_request_router_app.py @@ -43,10 +43,10 @@ def _time_ms() -> int: @serve.deployment( router_config=RouterConfig( request_router_class="custom_request_router:ThroughputAwareRequestRouter", + request_routing_stats_period_s=1, + request_routing_stats_timeout_s=1, ), num_replicas=3, - request_routing_stats_period_s=1, - request_routing_stats_timeout_s=1, ray_actor_options={"num_cpus": 0}, ) class ThroughputAwareRequestRouterApp: From 8c03deb568db12b61202fa89d9a8478b6fed2806 Mon Sep 17 00:00:00 2001 From: Seiji Eicher Date: Wed, 2 Jul 2025 21:15:23 +0000 Subject: [PATCH 23/36] Only support request_router_kwargs in Python Signed-off-by: Seiji Eicher --- python/ray/serve/_private/config.py | 24 ++++++++++++------------ 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/python/ray/serve/_private/config.py b/python/ray/serve/_private/config.py index ef6d1933c2b2..1225bbbf9380 100644 --- a/python/ray/serve/_private/config.py +++ b/python/ray/serve/_private/config.py @@ -249,22 +249,22 @@ def to_proto(self): ) if data.get("router_config"): router_kwargs = data.get("router_config").get("request_router_kwargs") - if router_kwargs is not None: - if self.deployment_language == DeploymentLanguage.JAVA: - # Java deployments don't support router kwargs - if isinstance(router_kwargs, dict) and not router_kwargs: - # Empty dict -> empty bytes for protobuf compatibility - data["router_config"]["request_router_kwargs"] = b"" - elif router_kwargs: - raise ValueError( - "request_router_kwargs is not supported for Java deployments. " - f"Got: {router_kwargs}" - ) + if router_kwargs is not None and isinstance(router_kwargs, dict): + if not router_kwargs: + data["router_config"]["request_router_kwargs"] = b"" elif self.needs_pickle(): - # Python deployments: pickle the router kwargs + # Only Python->Python deployments support non-empty router kwargs data["router_config"]["request_router_kwargs"] = cloudpickle.dumps( router_kwargs ) + else: + # All other cases: non-empty router kwargs not supported + error_msg = "Non-empty request_router_kwargs not supported for " + if self.deployment_language == DeploymentLanguage.JAVA: + error_msg += "Java deployments" + else: + error_msg += "cross-language deployments" + raise ValueError(f"{error_msg}. Got: {router_kwargs}") data["router_config"] = RouterConfigProto(**data["router_config"]) if data.get("logging_config"): if "encoding" in data["logging_config"]: From 67a8453025439e968bea11b60ba645c156cf1a3b Mon Sep 17 00:00:00 2001 From: Seiji Eicher Date: Wed, 2 Jul 2025 21:41:33 +0000 Subject: [PATCH 24/36] Lint Signed-off-by: Seiji Eicher --- python/ray/serve/_private/config.py | 17 +++++++---------- 1 file changed, 7 insertions(+), 10 deletions(-) diff --git a/python/ray/serve/_private/config.py b/python/ray/serve/_private/config.py index 1225bbbf9380..09640c31a519 100644 --- a/python/ray/serve/_private/config.py +++ b/python/ray/serve/_private/config.py @@ -248,23 +248,20 @@ def to_proto(self): **data["autoscaling_config"] ) if data.get("router_config"): - router_kwargs = data.get("router_config").get("request_router_kwargs") - if router_kwargs is not None and isinstance(router_kwargs, dict): + router_kwargs = data["router_config"].get("request_router_kwargs") + if router_kwargs is not None: if not router_kwargs: data["router_config"]["request_router_kwargs"] = b"" elif self.needs_pickle(): - # Only Python->Python deployments support non-empty router kwargs + # Protobuf requires bytes, so we need to pickle data["router_config"]["request_router_kwargs"] = cloudpickle.dumps( router_kwargs ) else: - # All other cases: non-empty router kwargs not supported - error_msg = "Non-empty request_router_kwargs not supported for " - if self.deployment_language == DeploymentLanguage.JAVA: - error_msg += "Java deployments" - else: - error_msg += "cross-language deployments" - raise ValueError(f"{error_msg}. Got: {router_kwargs}") + raise ValueError( + "Non-empty request_router_kwargs not supported" + f"for cross-language deployments. Got: {router_kwargs}" + ) data["router_config"] = RouterConfigProto(**data["router_config"]) if data.get("logging_config"): if "encoding" in data["logging_config"]: From 2776012061d08549e7b740c242895c806dabdb73 Mon Sep 17 00:00:00 2001 From: Seiji Eicher Date: Thu, 3 Jul 2025 00:00:17 +0000 Subject: [PATCH 25/36] Add test Signed-off-by: Seiji Eicher --- python/ray/serve/_private/config.py | 4 ---- python/ray/serve/deployment.py | 26 +++++--------------------- python/ray/serve/tests/test_api.py | 22 ++++++++++++++++++++++ 3 files changed, 27 insertions(+), 25 deletions(-) diff --git a/python/ray/serve/_private/config.py b/python/ray/serve/_private/config.py index 09640c31a519..21878bbea1c7 100644 --- a/python/ray/serve/_private/config.py +++ b/python/ray/serve/_private/config.py @@ -118,10 +118,6 @@ class DeploymentConfig(BaseModel): health_check_timeout_s: Timeout that the controller waits for a response from the replica's health check before marking it unhealthy. - request_routing_stats_period_s: Frequency at which the controller - record request routing stats. - request_routing_stats_timeout_s: Timeout that the controller waits - for a response from the replica's record routing stats call. autoscaling_config: Autoscaling configuration. logging_config: Configuration for deployment logs. user_configured_option_names: The names of options manually diff --git a/python/ray/serve/deployment.py b/python/ray/serve/deployment.py index 88a7ed440230..f566524a144c 100644 --- a/python/ray/serve/deployment.py +++ b/python/ray/serve/deployment.py @@ -7,10 +7,10 @@ from ray.serve._private.config import ( DeploymentConfig, ReplicaConfig, + RouterConfig, handle_num_replicas_auto, ) from ray.serve._private.constants import SERVE_LOGGER_NAME -from ray.serve._private.request_router.request_router import RequestRouter from ray.serve._private.usage import ServeUsageTag from ray.serve._private.utils import DEFAULT, Default from ray.serve.config import AutoscalingConfig @@ -237,10 +237,7 @@ def options( health_check_period_s: Default[float] = DEFAULT.VALUE, health_check_timeout_s: Default[float] = DEFAULT.VALUE, logging_config: Default[Union[Dict, LoggingConfig, None]] = DEFAULT.VALUE, - request_router_class: Default[Union[str, RequestRouter, None]] = DEFAULT.VALUE, - request_router_kwargs: Default[Union[Dict, None]] = DEFAULT.VALUE, - request_routing_stats_period_s: Default[float] = DEFAULT.VALUE, - request_routing_stats_timeout_s: Default[float] = DEFAULT.VALUE, + router_config: Default[Union[Dict, RouterConfig, None]] = DEFAULT.VALUE, _init_args: Default[Tuple[Any]] = DEFAULT.VALUE, _init_kwargs: Default[Dict[Any, Any]] = DEFAULT.VALUE, _internal: bool = False, @@ -352,6 +349,9 @@ def options( if autoscaling_config is not DEFAULT.VALUE: new_deployment_config.autoscaling_config = autoscaling_config + if router_config is not DEFAULT.VALUE: + new_deployment_config.router_config = router_config + if graceful_shutdown_wait_loop_s is not DEFAULT.VALUE: new_deployment_config.graceful_shutdown_wait_loop_s = ( graceful_shutdown_wait_loop_s @@ -373,22 +373,6 @@ def options( logging_config = logging_config.dict() new_deployment_config.logging_config = logging_config - if request_router_class is not DEFAULT.VALUE: - new_deployment_config.request_router_class = request_router_class - - if request_router_kwargs is not DEFAULT.VALUE: - new_deployment_config.request_router_kwargs = request_router_kwargs - - if request_routing_stats_period_s is not DEFAULT.VALUE: - new_deployment_config.request_routing_stats_period_s = ( - request_routing_stats_period_s - ) - - if request_routing_stats_timeout_s is not DEFAULT.VALUE: - new_deployment_config.request_routing_stats_timeout_s = ( - request_routing_stats_timeout_s - ) - new_replica_config = ReplicaConfig.create( func_or_class, init_args=_init_args, diff --git a/python/ray/serve/tests/test_api.py b/python/ray/serve/tests/test_api.py index bd7313ae9ff5..5e48971b4dae 100644 --- a/python/ray/serve/tests/test_api.py +++ b/python/ray/serve/tests/test_api.py @@ -81,6 +81,10 @@ async def choose_replicas( ) -> List[List[RunningReplica]]: return [candidate_replicas] + def initialize_state(self, test_parameter: int = 0): + print("Called initialize_state in FakeRequestRouter") + self.test_parameter = test_parameter + @serve.deployment(router_config=RouterConfig(request_router_class=FakeRequestRouter)) class AppWithCustomRequestRouter: @@ -1113,6 +1117,24 @@ def test_deploy_app_with_custom_request_router(serve_instance): assert handle.remote().result() == "Hello, world!" +@serve.deployment( + router_config=RouterConfig( + request_router_class="ray.serve.tests.test_api.FakeRequestRouter", + request_router_kwargs=dict(test_parameter=4848), + ) +) +class AppWithCustomRequestRouterAndKwargs: + def __call__(self) -> str: + return "Hello, world!" + + +def test_custom_request_router_kwargs(serve_instance): + """Check that custom kwargs can be passed to the request router.""" + + handle = serve.run(AppWithCustomRequestRouterAndKwargs.bind()) + assert handle.remote().result() == "Hello, world!" + + if __name__ == "__main__": import sys From 7e1850e060ec8af748a0c712cd60f142346d9db7 Mon Sep 17 00:00:00 2001 From: Seiji Eicher <58963096+eicherseiji@users.noreply.github.com> Date: Wed, 2 Jul 2025 21:03:00 -0700 Subject: [PATCH 26/36] Apply suggestions from code review Co-authored-by: angelinalg <122562471+angelinalg@users.noreply.github.com> Signed-off-by: Seiji Eicher <58963096+eicherseiji@users.noreply.github.com> --- .../io/ray/serve/config/RouterConfig.java | 2 +- .../_private/request_router/request_router.py | 4 ++-- python/ray/serve/config.py | 20 +++++++++---------- src/ray/protobuf/serve.proto | 4 ++-- 4 files changed, 15 insertions(+), 15 deletions(-) diff --git a/java/serve/src/main/java/io/ray/serve/config/RouterConfig.java b/java/serve/src/main/java/io/ray/serve/config/RouterConfig.java index 3f9b868c23e5..b139d8d77f82 100644 --- a/java/serve/src/main/java/io/ray/serve/config/RouterConfig.java +++ b/java/serve/src/main/java/io/ray/serve/config/RouterConfig.java @@ -8,7 +8,7 @@ public class RouterConfig implements Serializable { private Double requestRoutingStatsPeriodS = Constants.DEFAULT_REQUEST_ROUTING_STATS_PERIOD_S; /** - * Timeout that the controller will wait for a response from the replica's request routing stats + * Timeout that the controller waits for a response from the replica's request routing stats * before retrying. */ private Double requestRoutingStatsTimeoutS = Constants.DEFAULT_REQUEST_ROUTING_STATS_TIMEOUT_S; diff --git a/python/ray/serve/_private/request_router/request_router.py b/python/ray/serve/_private/request_router/request_router.py index c502aac583ab..ebfc2bc6a181 100644 --- a/python/ray/serve/_private/request_router/request_router.py +++ b/python/ray/serve/_private/request_router/request_router.py @@ -536,8 +536,8 @@ def __init__( def initialize_state(self, **kwargs): """ - Initialize the state of the request router. Called by framework with the - contents of RequestRouter.request_router_kwargs + Initialize the state of the request router. Called by the Ray Serve framework with the + contents of `RequestRouter.request_router_kwargs`. """ pass diff --git a/python/ray/serve/config.py b/python/ray/serve/config.py index 9b9985c4c56f..25987889225a 100644 --- a/python/ray/serve/config.py +++ b/python/ray/serve/config.py @@ -36,12 +36,12 @@ @PublicAPI(stability="stable") class RouterConfig(BaseModel): - """Config for Serve Router. + """Config for Serve router. Args: - request_router_class: The import path of the request router if user passed a string. Will be the - concatenation of the request router module and the request router name - if user passed a callable. + request_router_class: The import path of the request router if user passed a string. Is the + concatenation of the request router module and the request router name + if user passed a callable. """ serialized_request_router_cls: bytes = Field( @@ -51,16 +51,16 @@ class RouterConfig(BaseModel): request_router_class: Union[str, Callable] = Field( default=DEFAULT_REQUEST_ROUTER_PATH, description=( - "The class of the request router used for this deployment. This can be " - "a string or a class. All the deployment handle created for this " - "deployment will use the routing policy defined by the request router. " + "The class of the request router that Ray Serve uses for this deployment. This value can be " + "a string or a class. All the deployment handles that you create for this " + "deployment use the routing policy defined by the request router. " "Default to Serve's PowerOfTwoChoicesRequestRouter." ), ) request_router_kwargs: Dict[str, Any] = Field( default_factory=dict, description=( - "Keyword arguments that will be passed to the request router class " + "Keyword arguments that Ray Serve passes to the request router class " "initialize_state method." ), ) @@ -101,7 +101,7 @@ def request_router_kwargs_json_serializable(cls, v): def import_and_serialize_request_router_cls(cls, values) -> Dict[str, Any]: """Import and serialize request router class with cloudpickle. - Import the request router if it's passed in as a string import path. + Import the request router if you pass it in as a string import path. Then cloudpickle the request router and set to `serialized_request_router_cls`. """ @@ -121,7 +121,7 @@ def import_and_serialize_request_router_cls(cls, values) -> Dict[str, Any]: return values def get_request_router_class(self) -> Callable: - """Deserialize request router from cloudpickled bytes.""" + """Deserialize the request router from cloudpickled bytes.""" return cloudpickle.loads(self.serialized_request_router_cls) diff --git a/src/ray/protobuf/serve.proto b/src/ray/protobuf/serve.proto index 7f36f0810b25..fd294ac14211 100644 --- a/src/ray/protobuf/serve.proto +++ b/src/ray/protobuf/serve.proto @@ -96,7 +96,7 @@ message RouterConfig { // Cloudpickled request router definition. bytes serialized_request_router_cls = 1; - // The import path of the request router if user passed a string. Will be the + // The import path of the request router if user passed a string. It's the // concatenation of the request router module and the request router name // if user passed a callable. string request_router_class = 2; @@ -107,7 +107,7 @@ message RouterConfig { // Timeout after which a replica started a record routing stats without a response. double request_routing_stats_timeout_s = 4; - // kwargs which will be passed to the router class' initialize_state method + // kwargs which Ray Serve passes to the router class' initialize_state method. bytes request_router_kwargs = 5; } //[End] ROUTING CONFIG From dcf843a752ce6823e51acec62553d24ea0bbf69c Mon Sep 17 00:00:00 2001 From: Seiji Eicher Date: Wed, 2 Jul 2025 21:43:14 -0700 Subject: [PATCH 27/36] Improve RouterConfig documentation Signed-off-by: Seiji Eicher --- python/ray/serve/config.py | 38 +++++++++++++++++++++++++++++++++----- 1 file changed, 33 insertions(+), 5 deletions(-) diff --git a/python/ray/serve/config.py b/python/ray/serve/config.py index 25987889225a..420dc899ea8e 100644 --- a/python/ray/serve/config.py +++ b/python/ray/serve/config.py @@ -36,12 +36,40 @@ @PublicAPI(stability="stable") class RouterConfig(BaseModel): - """Config for Serve router. + """Config for the Serve request router. - Args: - request_router_class: The import path of the request router if user passed a string. Is the - concatenation of the request router module and the request router name - if user passed a callable. + This class configures how Ray Serve routes requests to deployment replicas. The router is + responsible for selecting which replica should handle each incoming request based on the + configured routing policy. You can customize the routing behavior by specifying a custom + request router class and providing configuration parameters. + + The router also manages periodic health checks and scheduling statistics collection from + replicas to make informed routing decisions. + + Example: + ```python + from ray.serve.config import RouterConfig, DeploymentConfig + from ray import serve + + # Use default router with custom stats collection interval + router_config = RouterConfig( + request_routing_stats_period_s=5.0, + request_routing_stats_timeout_s=15.0 + ) + + # Use custom router class + router_config = RouterConfig( + request_router_class="ray.serve._private.request_router.prefix_aware_router.PrefixAwarePow2ReplicaRouter", + request_router_kwargs={"imbalanced_threshold": 20} + ) + deployment_config = DeploymentConfig( + router_config=router_config + ) + deployment = serve.deploy( + "my_deployment", + deployment_config=deployment_config + ) + ``` """ serialized_request_router_cls: bytes = Field( From 42d08c3e8aa97bf9eb4679110318c49ffc4d3534 Mon Sep 17 00:00:00 2001 From: Seiji Eicher Date: Wed, 2 Jul 2025 22:04:07 -0700 Subject: [PATCH 28/36] Sphinx format Signed-off-by: Seiji Eicher --- python/ray/serve/config.py | 44 +++++++++++++++++++------------------- 1 file changed, 22 insertions(+), 22 deletions(-) diff --git a/python/ray/serve/config.py b/python/ray/serve/config.py index 420dc899ea8e..99d7f4537793 100644 --- a/python/ray/serve/config.py +++ b/python/ray/serve/config.py @@ -47,29 +47,29 @@ class RouterConfig(BaseModel): replicas to make informed routing decisions. Example: - ```python - from ray.serve.config import RouterConfig, DeploymentConfig - from ray import serve - - # Use default router with custom stats collection interval - router_config = RouterConfig( - request_routing_stats_period_s=5.0, - request_routing_stats_timeout_s=15.0 - ) + .. code-block:: python - # Use custom router class - router_config = RouterConfig( - request_router_class="ray.serve._private.request_router.prefix_aware_router.PrefixAwarePow2ReplicaRouter", - request_router_kwargs={"imbalanced_threshold": 20} - ) - deployment_config = DeploymentConfig( - router_config=router_config - ) - deployment = serve.deploy( - "my_deployment", - deployment_config=deployment_config - ) - ``` + from ray.serve.config import RouterConfig, DeploymentConfig + from ray import serve + + # Use default router with custom stats collection interval + router_config = RouterConfig( + request_routing_stats_period_s=5.0, + request_routing_stats_timeout_s=15.0 + ) + + # Use custom router class + router_config = RouterConfig( + request_router_class="ray.serve._private.request_router.prefix_aware_router.PrefixAwarePow2ReplicaRouter", + request_router_kwargs={"imbalanced_threshold": 20} + ) + deployment_config = DeploymentConfig( + router_config=router_config + ) + deployment = serve.deploy( + "my_deployment", + deployment_config=deployment_config + ) """ serialized_request_router_cls: bytes = Field( From 1c5b556cf0d3d745772b4eebe59d0495010d3867 Mon Sep 17 00:00:00 2001 From: Seiji Eicher Date: Thu, 3 Jul 2025 14:02:13 -0700 Subject: [PATCH 29/36] Rename RouterConfig -> RequestRouterConfig Signed-off-by: Seiji Eicher --- doc/source/serve/api/index.md | 2 +- .../serve/doc_code/custom_request_router_app.py | 6 +++--- .../java/io/ray/serve/config/DeploymentConfig.java | 10 +++++----- .../main/java/io/ray/serve/config/RouterConfig.java | 6 +++--- python/ray/serve/_private/config.py | 12 ++++++------ python/ray/serve/api.py | 4 ++-- python/ray/serve/config.py | 8 ++++---- python/ray/serve/deployment.py | 4 ++-- python/ray/serve/schema.py | 4 ++-- python/ray/serve/tests/test_api.py | 8 +++++--- python/ray/serve/tests/test_record_routing_stats.py | 4 ++-- python/ray/serve/tests/test_telemetry_2.py | 4 ++-- python/ray/serve/tests/unit/test_config.py | 6 +++--- src/ray/protobuf/serve.proto | 4 ++-- 14 files changed, 42 insertions(+), 40 deletions(-) diff --git a/doc/source/serve/api/index.md b/doc/source/serve/api/index.md index 9c3063e1ae06..52a67da52c9b 100644 --- a/doc/source/serve/api/index.md +++ b/doc/source/serve/api/index.md @@ -83,7 +83,7 @@ See the [model composition guide](serve-model-composition) for how to update cod serve.config.gRPCOptions serve.config.HTTPOptions serve.config.AutoscalingConfig - serve.config.RouterConfig + serve.config.RequestRouterConfig ``` ### Schemas diff --git a/doc/source/serve/doc_code/custom_request_router_app.py b/doc/source/serve/doc_code/custom_request_router_app.py index 1559bd209d6d..841e80c2f9c2 100644 --- a/doc/source/serve/doc_code/custom_request_router_app.py +++ b/doc/source/serve/doc_code/custom_request_router_app.py @@ -7,11 +7,11 @@ from collections import defaultdict from ray.serve.context import _get_internal_replica_context from typing import Any, Dict -from ray.serve.config import RouterConfig +from ray.serve.config import RequestRouterConfig @serve.deployment( - router_config=RouterConfig( + router_config=RequestRouterConfig( request_router_class="custom_request_router:UniformRequestRouter", ), num_replicas=10, @@ -41,7 +41,7 @@ def _time_ms() -> int: @serve.deployment( - router_config=RouterConfig( + router_config=RequestRouterConfig( request_router_class="custom_request_router:ThroughputAwareRequestRouter", request_routing_stats_period_s=1, request_routing_stats_timeout_s=1, diff --git a/java/serve/src/main/java/io/ray/serve/config/DeploymentConfig.java b/java/serve/src/main/java/io/ray/serve/config/DeploymentConfig.java index 629809ef6c81..50e5fa5297f3 100644 --- a/java/serve/src/main/java/io/ray/serve/config/DeploymentConfig.java +++ b/java/serve/src/main/java/io/ray/serve/config/DeploymentConfig.java @@ -54,7 +54,7 @@ public class DeploymentConfig implements Serializable { private AutoscalingConfig autoscalingConfig; - private RouterConfig routerConfig; + private RequestRouterConfig routerConfig; /** This flag is used to let replica know they are deplyed from a different language. */ private Boolean isCrossLanguage = false; @@ -173,11 +173,11 @@ public DeploymentConfig setAutoscalingConfig(AutoscalingConfig autoscalingConfig return this; } - public RouterConfig getRouterConfig() { + public RequestRouterConfig getRequestRouterConfig() { return routerConfig; } - public DeploymentConfig setRouterConfig(RouterConfig routerConfig) { + public DeploymentConfig setRequestRouterConfig(RequestRouterConfig routerConfig) { this.routerConfig = routerConfig; return this; } @@ -242,7 +242,7 @@ public byte[] toProtoBytes() { builder.setAutoscalingConfig(autoscalingConfig.toProto()); } if (null != routerConfig) { - builder.setRouterConfig(routerConfig.toProto()); + builder.setRequestRouterConfig(routerConfig.toProto()); } return builder.build().toByteArray(); } @@ -265,7 +265,7 @@ public io.ray.serve.generated.DeploymentConfig toProto() { builder.setAutoscalingConfig(autoscalingConfig.toProto()); } if (null != routerConfig) { - builder.setRouterConfig(routerConfig.toProto()); + builder.setRequestRouterConfig(routerConfig.toProto()); } return builder.build(); } diff --git a/java/serve/src/main/java/io/ray/serve/config/RouterConfig.java b/java/serve/src/main/java/io/ray/serve/config/RouterConfig.java index b139d8d77f82..10a61d7543b4 100644 --- a/java/serve/src/main/java/io/ray/serve/config/RouterConfig.java +++ b/java/serve/src/main/java/io/ray/serve/config/RouterConfig.java @@ -3,7 +3,7 @@ import io.ray.serve.common.Constants; import java.io.Serializable; -public class RouterConfig implements Serializable { +public class RequestRouterConfig implements Serializable { /** Frequency at which the controller will record request routing stats. */ private Double requestRoutingStatsPeriodS = Constants.DEFAULT_REQUEST_ROUTING_STATS_PERIOD_S; @@ -29,8 +29,8 @@ public void setRequestRoutingStatsTimeoutS(Double requestRoutingStatsTimeoutS) { this.requestRoutingStatsTimeoutS = requestRoutingStatsTimeoutS; } - public io.ray.serve.generated.RouterConfig toProto() { - return io.ray.serve.generated.RouterConfig.newBuilder() + public io.ray.serve.generated.RequestRouterConfig toProto() { + return io.ray.serve.generated.RequestRouterConfig.newBuilder() .setRequestRoutingStatsPeriodS(requestRoutingStatsPeriodS) .setRequestRoutingStatsTimeoutS(requestRoutingStatsTimeoutS) .build(); diff --git a/python/ray/serve/_private/config.py b/python/ray/serve/_private/config.py index 21878bbea1c7..ad82980ce764 100644 --- a/python/ray/serve/_private/config.py +++ b/python/ray/serve/_private/config.py @@ -27,7 +27,7 @@ MAX_REPLICAS_PER_NODE_MAX_VALUE, ) from ray.serve._private.utils import DEFAULT, DeploymentOptionUpdateType -from ray.serve.config import AutoscalingConfig, RouterConfig +from ray.serve.config import AutoscalingConfig, RequestRouterConfig from ray.serve.generated.serve_pb2 import ( AutoscalingConfig as AutoscalingConfigProto, DeploymentConfig as DeploymentConfigProto, @@ -35,7 +35,7 @@ EncodingType as EncodingTypeProto, LoggingConfig as LoggingConfigProto, ReplicaConfig as ReplicaConfigProto, - RouterConfig as RouterConfigProto, + RequestRouterConfig as RequestRouterConfigProto, ) from ray.util.placement_group import validate_placement_group @@ -162,8 +162,8 @@ class DeploymentConfig(BaseModel): default=None, update_type=DeploymentOptionUpdateType.NeedsActorReconfigure ) - router_config: RouterConfig = Field( - default=RouterConfig(), + router_config: RequestRouterConfig = Field( + default=RequestRouterConfig(), update_type=DeploymentOptionUpdateType.NeedsActorReconfigure, ) @@ -258,7 +258,7 @@ def to_proto(self): "Non-empty request_router_kwargs not supported" f"for cross-language deployments. Got: {router_kwargs}" ) - data["router_config"] = RouterConfigProto(**data["router_config"]) + data["router_config"] = RequestRouterConfigProto(**data["router_config"]) if data.get("logging_config"): if "encoding" in data["logging_config"]: data["logging_config"]["encoding"] = EncodingTypeProto.Value( @@ -308,7 +308,7 @@ def from_proto(cls, proto: DeploymentConfigProto): else: data["router_config"]["request_router_kwargs"] = {} - data["router_config"] = RouterConfig(**data["router_config"]) + data["router_config"] = RequestRouterConfig(**data["router_config"]) if "autoscaling_config" in data: if not data["autoscaling_config"].get("upscale_smoothing_factor"): data["autoscaling_config"]["upscale_smoothing_factor"] = None diff --git a/python/ray/serve/api.py b/python/ray/serve/api.py index 9adbb40e809f..57aec8986228 100644 --- a/python/ray/serve/api.py +++ b/python/ray/serve/api.py @@ -42,7 +42,7 @@ DeploymentMode, HTTPOptions, ProxyLocation, - RouterConfig, + RequestRouterConfig, gRPCOptions, ) from ray.serve.context import ( @@ -334,7 +334,7 @@ def deployment( health_check_period_s: Default[float] = DEFAULT.VALUE, health_check_timeout_s: Default[float] = DEFAULT.VALUE, logging_config: Default[Union[Dict, LoggingConfig, None]] = DEFAULT.VALUE, - router_config: Default[Union[Dict, RouterConfig, None]] = DEFAULT.VALUE, + router_config: Default[Union[Dict, RequestRouterConfig, None]] = DEFAULT.VALUE, ) -> Callable[[Callable], Deployment]: """Decorator that converts a Python class to a `Deployment`. diff --git a/python/ray/serve/config.py b/python/ray/serve/config.py index 99d7f4537793..4a9aca50d006 100644 --- a/python/ray/serve/config.py +++ b/python/ray/serve/config.py @@ -35,7 +35,7 @@ @PublicAPI(stability="stable") -class RouterConfig(BaseModel): +class RequestRouterConfig(BaseModel): """Config for the Serve request router. This class configures how Ray Serve routes requests to deployment replicas. The router is @@ -49,17 +49,17 @@ class RouterConfig(BaseModel): Example: .. code-block:: python - from ray.serve.config import RouterConfig, DeploymentConfig + from ray.serve.config import RequestRouterConfig, DeploymentConfig from ray import serve # Use default router with custom stats collection interval - router_config = RouterConfig( + router_config = RequestRouterConfig( request_routing_stats_period_s=5.0, request_routing_stats_timeout_s=15.0 ) # Use custom router class - router_config = RouterConfig( + router_config = RequestRouterConfig( request_router_class="ray.serve._private.request_router.prefix_aware_router.PrefixAwarePow2ReplicaRouter", request_router_kwargs={"imbalanced_threshold": 20} ) diff --git a/python/ray/serve/deployment.py b/python/ray/serve/deployment.py index f566524a144c..100b8e94592c 100644 --- a/python/ray/serve/deployment.py +++ b/python/ray/serve/deployment.py @@ -7,7 +7,7 @@ from ray.serve._private.config import ( DeploymentConfig, ReplicaConfig, - RouterConfig, + RequestRouterConfig, handle_num_replicas_auto, ) from ray.serve._private.constants import SERVE_LOGGER_NAME @@ -237,7 +237,7 @@ def options( health_check_period_s: Default[float] = DEFAULT.VALUE, health_check_timeout_s: Default[float] = DEFAULT.VALUE, logging_config: Default[Union[Dict, LoggingConfig, None]] = DEFAULT.VALUE, - router_config: Default[Union[Dict, RouterConfig, None]] = DEFAULT.VALUE, + router_config: Default[Union[Dict, RequestRouterConfig, None]] = DEFAULT.VALUE, _init_args: Default[Tuple[Any]] = DEFAULT.VALUE, _init_kwargs: Default[Dict[Any, Any]] = DEFAULT.VALUE, _internal: bool = False, diff --git a/python/ray/serve/schema.py b/python/ray/serve/schema.py index c92666219d23..737c972560db 100644 --- a/python/ray/serve/schema.py +++ b/python/ray/serve/schema.py @@ -33,7 +33,7 @@ ) from ray.serve._private.deployment_info import DeploymentInfo from ray.serve._private.utils import DEFAULT -from ray.serve.config import ProxyLocation, RouterConfig +from ray.serve.config import ProxyLocation, RequestRouterConfig from ray.util.annotations import PublicAPI # Shared amongst multiple schemas. @@ -405,7 +405,7 @@ class DeploymentSchema(BaseModel, allow_population_by_field_name=True): default=DEFAULT.VALUE, description="Logging config for configuring serve deployment logs.", ) - router_config: Union[Dict, RouterConfig] = Field( + router_config: Union[Dict, RequestRouterConfig] = Field( default=DEFAULT.VALUE, description="Config for the request router used for this deployment.", ) diff --git a/python/ray/serve/tests/test_api.py b/python/ray/serve/tests/test_api.py index 5e48971b4dae..84ac736c5866 100644 --- a/python/ray/serve/tests/test_api.py +++ b/python/ray/serve/tests/test_api.py @@ -28,7 +28,7 @@ RequestRouter, ) from ray.serve._private.test_utils import get_application_url -from ray.serve.config import RouterConfig +from ray.serve.config import RequestRouterConfig from ray.serve.deployment import Application from ray.serve.exceptions import RayServeException from ray.serve.handle import DeploymentHandle @@ -86,7 +86,9 @@ def initialize_state(self, test_parameter: int = 0): self.test_parameter = test_parameter -@serve.deployment(router_config=RouterConfig(request_router_class=FakeRequestRouter)) +@serve.deployment( + router_config=RequestRouterConfig(request_router_class=FakeRequestRouter) +) class AppWithCustomRequestRouter: def __call__(self) -> str: return "Hello, world!" @@ -1118,7 +1120,7 @@ def test_deploy_app_with_custom_request_router(serve_instance): @serve.deployment( - router_config=RouterConfig( + router_config=RequestRouterConfig( request_router_class="ray.serve.tests.test_api.FakeRequestRouter", request_router_kwargs=dict(test_parameter=4848), ) diff --git a/python/ray/serve/tests/test_record_routing_stats.py b/python/ray/serve/tests/test_record_routing_stats.py index 3e37443c1b5f..a4c5e893b56b 100644 --- a/python/ray/serve/tests/test_record_routing_stats.py +++ b/python/ray/serve/tests/test_record_routing_stats.py @@ -7,13 +7,13 @@ from ray import serve from ray._common.test_utils import wait_for_condition from ray.serve._private.common import ReplicaID -from ray.serve.config import RouterConfig +from ray.serve.config import RequestRouterConfig from ray.serve.context import _get_internal_replica_context from ray.serve.handle import DeploymentHandle @serve.deployment( - router_config=RouterConfig( + router_config=RequestRouterConfig( request_routing_stats_period_s=0.1, request_routing_stats_timeout_s=0.1 ) ) diff --git a/python/ray/serve/tests/test_telemetry_2.py b/python/ray/serve/tests/test_telemetry_2.py index bda1f85ec02e..3e4fb2d092b0 100644 --- a/python/ray/serve/tests/test_telemetry_2.py +++ b/python/ray/serve/tests/test_telemetry_2.py @@ -17,7 +17,7 @@ ) from ray.serve._private.test_utils import check_apps_running, check_telemetry from ray.serve._private.usage import ServeUsageTag -from ray.serve.config import RouterConfig +from ray.serve.config import RequestRouterConfig from ray.serve.context import _get_global_client from ray.serve.schema import ServeDeploySchema @@ -160,7 +160,7 @@ def test_custom_request_router_telemetry(manage_ray_with_telemetry): check_telemetry(ServeUsageTag.CUSTOM_REQUEST_ROUTER_USED, expected=None) @serve.deployment( - router_config=RouterConfig( + router_config=RequestRouterConfig( request_router_class=CustomRequestRouter, ), ) diff --git a/python/ray/serve/tests/unit/test_config.py b/python/ray/serve/tests/unit/test_config.py index e200ed3d8d40..884a7cff289d 100644 --- a/python/ray/serve/tests/unit/test_config.py +++ b/python/ray/serve/tests/unit/test_config.py @@ -15,7 +15,7 @@ DeploymentMode, HTTPOptions, ProxyLocation, - RouterConfig, + RequestRouterConfig, gRPCOptions, ) from ray.serve.generated.serve_pb2 import ( @@ -145,7 +145,7 @@ def test_setting_and_getting_request_router_class(self): # Passing request_router_class as a class. deployment_config = DeploymentConfig.from_default( - router_config=RouterConfig(request_router_class=FakeRequestRouter) + router_config=RequestRouterConfig(request_router_class=FakeRequestRouter) ) assert ( deployment_config.router_config.request_router_class == request_router_path @@ -157,7 +157,7 @@ def test_setting_and_getting_request_router_class(self): # Passing request_router_class as an import path. deployment_config = DeploymentConfig.from_default( - router_config=RouterConfig(request_router_class=request_router_path) + router_config=RequestRouterConfig(request_router_class=request_router_path) ) assert ( deployment_config.router_config.request_router_class == request_router_path diff --git a/src/ray/protobuf/serve.proto b/src/ray/protobuf/serve.proto index fd294ac14211..e6bf208dd8b5 100644 --- a/src/ray/protobuf/serve.proto +++ b/src/ray/protobuf/serve.proto @@ -92,7 +92,7 @@ message LoggingConfig { //[End] Logging Config //[Begin] ROUTING CONFIG -message RouterConfig { +message RequestRouterConfig { // Cloudpickled request router definition. bytes serialized_request_router_cls = 1; @@ -157,7 +157,7 @@ message DeploymentConfig { LoggingConfig logging_config = 14; // The deployment's routing configuration. - RouterConfig router_config = 19; + RequestRouterConfig router_config = 19; } // Deployment language. From f867e09de1b3dc165fcea75127e942d0e9df2bf3 Mon Sep 17 00:00:00 2001 From: Seiji Eicher Date: Thu, 3 Jul 2025 14:08:38 -0700 Subject: [PATCH 30/36] Add docstring to initialize_state and move to ray.llm Signed-off-by: Seiji Eicher --- .../prefix_aware}/prefix_aware_router.py | 18 ++++++++++++++++++ .../test_prefix_aware_request_router.py | 6 +++--- python/ray/serve/config.py | 2 +- 3 files changed, 22 insertions(+), 4 deletions(-) rename python/ray/{serve/_private/request_router => llm/_internal/serve/request_router/prefix_aware}/prefix_aware_router.py (92%) diff --git a/python/ray/serve/_private/request_router/prefix_aware_router.py b/python/ray/llm/_internal/serve/request_router/prefix_aware/prefix_aware_router.py similarity index 92% rename from python/ray/serve/_private/request_router/prefix_aware_router.py rename to python/ray/llm/_internal/serve/request_router/prefix_aware/prefix_aware_router.py index b235d30402ce..ffadef551671 100644 --- a/python/ray/serve/_private/request_router/prefix_aware_router.py +++ b/python/ray/llm/_internal/serve/request_router/prefix_aware/prefix_aware_router.py @@ -76,6 +76,24 @@ def initialize_state( eviction_target_chars: Optional[int] = 360_000, eviction_interval_secs: Optional[int] = 10, ): + """Initialize the prefix-aware routing state and configuration. + + Args: + imbalanced_threshold: Threshold for queue length difference to consider + load balanced. When the difference between replica queue lengths is + less than this value, prefix-aware routing is used. + match_rate_threshold: Minimum prefix match rate (0.0-1.0) required to + use prefix-aware routing. If match rate is below this threshold, + falls back to smallest tenant selection. + do_eviction: Whether to enable automatic eviction of old prefix tree + entries to manage memory usage. + eviction_threshold_chars: Maximum number of characters in the prefix + tree before eviction is triggered. + eviction_target_chars: Target number of characters to reduce the + prefix tree to during eviction. + eviction_interval_secs: Interval in seconds between eviction checks + when eviction is enabled. + """ # === Prefix-aware routing logic hyperparameters === self._imbalanced_threshold = imbalanced_threshold self._match_rate_threshold = match_rate_threshold diff --git a/python/ray/llm/tests/serve/cpu/deployments/test_prefix_aware_request_router.py b/python/ray/llm/tests/serve/cpu/deployments/test_prefix_aware_request_router.py index 0cf7d123686f..4158a38140bb 100644 --- a/python/ray/llm/tests/serve/cpu/deployments/test_prefix_aware_request_router.py +++ b/python/ray/llm/tests/serve/cpu/deployments/test_prefix_aware_request_router.py @@ -5,6 +5,9 @@ import ray from ray._common.utils import get_or_create_event_loop +from ray.llm._internal.serve.request_router.prefix_aware.prefix_aware_router import ( + PrefixAwarePow2ReplicaRouter, +) from ray.llm._internal.serve.request_router.prefix_aware.prefix_tree import ( PrefixTreeActor, ) @@ -14,9 +17,6 @@ RequestMetadata, ) from ray.serve._private.request_router.common import PendingRequest -from ray.serve._private.request_router.prefix_aware_router import ( - PrefixAwarePow2ReplicaRouter, -) from ray.serve._private.test_utils import MockTimer from ray.serve._private.utils import generate_request_id from ray.serve.tests.unit.test_pow_2_request_router import ( diff --git a/python/ray/serve/config.py b/python/ray/serve/config.py index 4a9aca50d006..ebbc19f3f8cf 100644 --- a/python/ray/serve/config.py +++ b/python/ray/serve/config.py @@ -60,7 +60,7 @@ class RequestRouterConfig(BaseModel): # Use custom router class router_config = RequestRouterConfig( - request_router_class="ray.serve._private.request_router.prefix_aware_router.PrefixAwarePow2ReplicaRouter", + request_router_class="ray.llm._internal.serve.request_router.prefix_aware.prefix_aware_router.PrefixAwarePow2ReplicaRouter", request_router_kwargs={"imbalanced_threshold": 20} ) deployment_config = DeploymentConfig( From 0d3938cb0a40fc1bf77a286c9ad70ba213531e73 Mon Sep 17 00:00:00 2001 From: Seiji Eicher Date: Thu, 3 Jul 2025 21:27:10 +0000 Subject: [PATCH 31/36] Rename serialized_request_router_cls -> _serialized_request_router_cls, make it private Signed-off-by: Seiji Eicher --- python/ray/serve/_private/router.py | 3 +-- python/ray/serve/_private/version.py | 2 +- python/ray/serve/config.py | 12 +++++------- python/ray/serve/schema.py | 2 +- src/ray/protobuf/serve.proto | 2 +- 5 files changed, 9 insertions(+), 12 deletions(-) diff --git a/python/ray/serve/_private/router.py b/python/ray/serve/_private/router.py index b03d6313b20a..7cca1e9f5a41 100644 --- a/python/ray/serve/_private/router.py +++ b/python/ray/serve/_private/router.py @@ -506,9 +506,8 @@ def request_router(self) -> Optional[RequestRouter]: prefer_local_node_routing=self._prefer_local_node_routing, prefer_local_az_routing=RAY_SERVE_PROXY_PREFER_LOCAL_AZ_ROUTING, self_availability_zone=self._availability_zone, - **(self._request_router_kwargs or {}), ) - request_router.initialize_state(**(self._request_router_kwargs or {})) + request_router.initialize_state(**(self._request_router_kwargs)) # Populate the running replicas if they are already available. if self._running_replicas is not None: diff --git a/python/ray/serve/_private/version.py b/python/ray/serve/_private/version.py index 3b264f6467b6..81ec55675af6 100644 --- a/python/ray/serve/_private/version.py +++ b/python/ray/serve/_private/version.py @@ -189,7 +189,7 @@ def _get_serialized_options( # Can't serialize bytes. The request router class is already # included in the serialized config as request_router_class. if "router_config" in reconfigure_dict: - del reconfigure_dict["router_config"]["serialized_request_router_cls"] + del reconfigure_dict["router_config"]["_serialized_request_router_cls"] if ( isinstance(self.deployment_config.user_config, bytes) diff --git a/python/ray/serve/config.py b/python/ray/serve/config.py index ebbc19f3f8cf..f5b1fe78a796 100644 --- a/python/ray/serve/config.py +++ b/python/ray/serve/config.py @@ -72,9 +72,7 @@ class RequestRouterConfig(BaseModel): ) """ - serialized_request_router_cls: bytes = Field( - default=b"", description="Cloudpickled request router class." - ) + _serialized_request_router_cls: bytes = PrivateAttr(default=b"") request_router_class: Union[str, Callable] = Field( default=DEFAULT_REQUEST_ROUTER_PATH, @@ -131,7 +129,7 @@ def import_and_serialize_request_router_cls(cls, values) -> Dict[str, Any]: Import the request router if you pass it in as a string import path. Then cloudpickle the request router and set to - `serialized_request_router_cls`. + `_serialized_request_router_cls`. """ request_router_class = values.get("request_router_class") if isinstance(request_router_class, Callable): @@ -142,7 +140,7 @@ def import_and_serialize_request_router_cls(cls, values) -> Dict[str, Any]: request_router_path = request_router_class or DEFAULT_REQUEST_ROUTER_PATH request_router_class = import_attr(request_router_path) - values["serialized_request_router_cls"] = cloudpickle.dumps( + values["_serialized_request_router_cls"] = cloudpickle.dumps( request_router_class ) values["request_router_class"] = request_router_path @@ -150,10 +148,10 @@ def import_and_serialize_request_router_cls(cls, values) -> Dict[str, Any]: def get_request_router_class(self) -> Callable: """Deserialize the request router from cloudpickled bytes.""" - return cloudpickle.loads(self.serialized_request_router_cls) + return cloudpickle.loads(self._serialized_request_router_cls) -@PublicAPI(stability="stable") +@PublicAPI(stability="alpha") class AutoscalingConfig(BaseModel): """Config for the Serve Autoscaler.""" diff --git a/python/ray/serve/schema.py b/python/ray/serve/schema.py index 737c972560db..c28aa48f6e45 100644 --- a/python/ray/serve/schema.py +++ b/python/ray/serve/schema.py @@ -1194,7 +1194,7 @@ def _get_user_facing_json_serializable_dict( # Remove internal fields from router_config if it exists if "router_config" in deployment["deployment_config"]: deployment["deployment_config"]["router_config"].pop( - "serialized_request_router_cls", None + "_serialized_request_router_cls", None ) if "autoscaling_config" in deployment["deployment_config"]: deployment["deployment_config"]["autoscaling_config"].pop( diff --git a/src/ray/protobuf/serve.proto b/src/ray/protobuf/serve.proto index e6bf208dd8b5..18eea888a8e5 100644 --- a/src/ray/protobuf/serve.proto +++ b/src/ray/protobuf/serve.proto @@ -94,7 +94,7 @@ message LoggingConfig { //[Begin] ROUTING CONFIG message RequestRouterConfig { // Cloudpickled request router definition. - bytes serialized_request_router_cls = 1; + bytes _serialized_request_router_cls = 1; // The import path of the request router if user passed a string. It's the // concatenation of the request router module and the request router name From ffa0ef1eb239bbe919585a622d1ba2304e4c9b67 Mon Sep 17 00:00:00 2001 From: Seiji Eicher Date: Thu, 3 Jul 2025 22:00:55 +0000 Subject: [PATCH 32/36] Rename RouterConfig.java -> RequestRouterConfig.java Signed-off-by: Seiji Eicher --- .../serve/config/{RouterConfig.java => RequestRouterConfig.java} | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename java/serve/src/main/java/io/ray/serve/config/{RouterConfig.java => RequestRouterConfig.java} (100%) diff --git a/java/serve/src/main/java/io/ray/serve/config/RouterConfig.java b/java/serve/src/main/java/io/ray/serve/config/RequestRouterConfig.java similarity index 100% rename from java/serve/src/main/java/io/ray/serve/config/RouterConfig.java rename to java/serve/src/main/java/io/ray/serve/config/RequestRouterConfig.java From e33402ed8bb90f98015a1b453a89f1574846b217 Mon Sep 17 00:00:00 2001 From: Seiji Eicher Date: Thu, 3 Jul 2025 23:36:33 +0000 Subject: [PATCH 33/36] Update Protobuf field name Signed-off-by: Seiji Eicher --- src/ray/protobuf/serve.proto | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/ray/protobuf/serve.proto b/src/ray/protobuf/serve.proto index 18eea888a8e5..ebfc36096a01 100644 --- a/src/ray/protobuf/serve.proto +++ b/src/ray/protobuf/serve.proto @@ -157,7 +157,7 @@ message DeploymentConfig { LoggingConfig logging_config = 14; // The deployment's routing configuration. - RequestRouterConfig router_config = 19; + RequestRouterConfig request_router_config = 19; } // Deployment language. From 34aab98ce93773284a4802a05d6bfd84c2138846 Mon Sep 17 00:00:00 2001 From: Seiji Eicher Date: Fri, 4 Jul 2025 03:15:30 +0000 Subject: [PATCH 34/36] Complete renaming Signed-off-by: Seiji Eicher --- .../doc_code/custom_request_router_app.py | 4 +- python/ray/serve/_private/config.py | 44 +++++++++++-------- python/ray/serve/_private/deployment_state.py | 8 +++- python/ray/serve/_private/router.py | 4 +- python/ray/serve/_private/version.py | 6 ++- python/ray/serve/api.py | 8 ++-- python/ray/serve/config.py | 32 +++++++++----- python/ray/serve/deployment.py | 12 ++--- python/ray/serve/schema.py | 10 ++--- python/ray/serve/tests/test_api.py | 4 +- python/ray/serve/tests/test_controller.py | 2 +- .../serve/tests/test_record_routing_stats.py | 2 +- python/ray/serve/tests/test_telemetry_2.py | 2 +- python/ray/serve/tests/unit/test_config.py | 22 ++++++---- 14 files changed, 96 insertions(+), 64 deletions(-) diff --git a/doc/source/serve/doc_code/custom_request_router_app.py b/doc/source/serve/doc_code/custom_request_router_app.py index 841e80c2f9c2..afabaa2d5711 100644 --- a/doc/source/serve/doc_code/custom_request_router_app.py +++ b/doc/source/serve/doc_code/custom_request_router_app.py @@ -11,7 +11,7 @@ @serve.deployment( - router_config=RequestRouterConfig( + request_router_config=RequestRouterConfig( request_router_class="custom_request_router:UniformRequestRouter", ), num_replicas=10, @@ -41,7 +41,7 @@ def _time_ms() -> int: @serve.deployment( - router_config=RequestRouterConfig( + request_router_config=RequestRouterConfig( request_router_class="custom_request_router:ThroughputAwareRequestRouter", request_routing_stats_period_s=1, request_routing_stats_timeout_s=1, diff --git a/python/ray/serve/_private/config.py b/python/ray/serve/_private/config.py index ad82980ce764..3293f5557ee4 100644 --- a/python/ray/serve/_private/config.py +++ b/python/ray/serve/_private/config.py @@ -122,7 +122,7 @@ class DeploymentConfig(BaseModel): logging_config: Configuration for deployment logs. user_configured_option_names: The names of options manually configured by the user. - router_config: Configuration for deployment request router. + request_router_config: Configuration for deployment request router. """ num_replicas: Optional[NonNegativeInt] = Field( @@ -162,7 +162,7 @@ class DeploymentConfig(BaseModel): default=None, update_type=DeploymentOptionUpdateType.NeedsActorReconfigure ) - router_config: RequestRouterConfig = Field( + request_router_config: RequestRouterConfig = Field( default=RequestRouterConfig(), update_type=DeploymentOptionUpdateType.NeedsActorReconfigure, ) @@ -243,22 +243,24 @@ def to_proto(self): data["autoscaling_config"] = AutoscalingConfigProto( **data["autoscaling_config"] ) - if data.get("router_config"): - router_kwargs = data["router_config"].get("request_router_kwargs") + if data.get("request_router_config"): + router_kwargs = data["request_router_config"].get("request_router_kwargs") if router_kwargs is not None: if not router_kwargs: - data["router_config"]["request_router_kwargs"] = b"" + data["request_router_config"]["request_router_kwargs"] = b"" elif self.needs_pickle(): # Protobuf requires bytes, so we need to pickle - data["router_config"]["request_router_kwargs"] = cloudpickle.dumps( - router_kwargs - ) + data["request_router_config"][ + "request_router_kwargs" + ] = cloudpickle.dumps(router_kwargs) else: raise ValueError( "Non-empty request_router_kwargs not supported" f"for cross-language deployments. Got: {router_kwargs}" ) - data["router_config"] = RequestRouterConfigProto(**data["router_config"]) + data["request_router_config"] = RequestRouterConfigProto( + **data["request_router_config"] + ) if data.get("logging_config"): if "encoding" in data["logging_config"]: data["logging_config"]["encoding"] = EncodingTypeProto.Value( @@ -293,22 +295,28 @@ def from_proto(cls, proto: DeploymentConfigProto): data["user_config"] = proto.user_config else: data["user_config"] = None - if "router_config" in data: - if "request_router_kwargs" in data["router_config"]: - request_router_kwargs = data["router_config"]["request_router_kwargs"] + if "request_router_config" in data: + if "request_router_kwargs" in data["request_router_config"]: + request_router_kwargs = data["request_router_config"][ + "request_router_kwargs" + ] if request_router_kwargs != b"": if needs_pickle: - data["router_config"][ + data["request_router_config"][ "request_router_kwargs" - ] = cloudpickle.loads(proto.router_config.request_router_kwargs) + ] = cloudpickle.loads( + proto.request_router_config.request_router_kwargs + ) else: - data["router_config"][ + data["request_router_config"][ "request_router_kwargs" - ] = proto.router_config.request_router_kwargs + ] = proto.request_router_config.request_router_kwargs else: - data["router_config"]["request_router_kwargs"] = {} + data["request_router_config"]["request_router_kwargs"] = {} - data["router_config"] = RequestRouterConfig(**data["router_config"]) + data["request_router_config"] = RequestRouterConfig( + **data["request_router_config"] + ) if "autoscaling_config" in data: if not data["autoscaling_config"].get("upscale_smoothing_factor"): data["autoscaling_config"]["upscale_smoothing_factor"] = None diff --git a/python/ray/serve/_private/deployment_state.py b/python/ray/serve/_private/deployment_state.py index 2d8e0019517b..d30c5ccee6fa 100644 --- a/python/ray/serve/_private/deployment_state.py +++ b/python/ray/serve/_private/deployment_state.py @@ -359,11 +359,15 @@ def health_check_timeout_s(self) -> float: @property def request_routing_stats_period_s(self) -> float: - return self.deployment_config.router_config.request_routing_stats_period_s + return ( + self.deployment_config.request_router_config.request_routing_stats_period_s + ) @property def request_routing_stats_timeout_s(self) -> float: - return self.deployment_config.router_config.request_routing_stats_timeout_s + return ( + self.deployment_config.request_router_config.request_routing_stats_timeout_s + ) @property def pid(self) -> Optional[int]: diff --git a/python/ray/serve/_private/router.py b/python/ray/serve/_private/router.py index 7cca1e9f5a41..032fa9744958 100644 --- a/python/ray/serve/_private/router.py +++ b/python/ray/serve/_private/router.py @@ -543,10 +543,10 @@ def update_deployment_targets(self, deployment_target_info: DeploymentTargetInfo def update_deployment_config(self, deployment_config: DeploymentConfig): self._request_router_class = ( - deployment_config.router_config.get_request_router_class() + deployment_config.request_router_config.get_request_router_class() ) self._request_router_kwargs = ( - deployment_config.router_config.request_router_kwargs + deployment_config.request_router_config.request_router_kwargs ) self._metrics_manager.update_deployment_config( deployment_config, diff --git a/python/ray/serve/_private/version.py b/python/ray/serve/_private/version.py index 81ec55675af6..9242dfc928e9 100644 --- a/python/ray/serve/_private/version.py +++ b/python/ray/serve/_private/version.py @@ -188,8 +188,10 @@ def _get_serialized_options( # Can't serialize bytes. The request router class is already # included in the serialized config as request_router_class. - if "router_config" in reconfigure_dict: - del reconfigure_dict["router_config"]["_serialized_request_router_cls"] + if "request_router_config" in reconfigure_dict: + reconfigure_dict["request_router_config"].pop( + "_serialized_request_router_cls", None + ) if ( isinstance(self.deployment_config.user_config, bytes) diff --git a/python/ray/serve/api.py b/python/ray/serve/api.py index 57aec8986228..865d5cedbba1 100644 --- a/python/ray/serve/api.py +++ b/python/ray/serve/api.py @@ -334,7 +334,9 @@ def deployment( health_check_period_s: Default[float] = DEFAULT.VALUE, health_check_timeout_s: Default[float] = DEFAULT.VALUE, logging_config: Default[Union[Dict, LoggingConfig, None]] = DEFAULT.VALUE, - router_config: Default[Union[Dict, RequestRouterConfig, None]] = DEFAULT.VALUE, + request_router_config: Default[ + Union[Dict, RequestRouterConfig, None] + ] = DEFAULT.VALUE, ) -> Callable[[Callable], Deployment]: """Decorator that converts a Python class to a `Deployment`. @@ -399,7 +401,7 @@ class MyDeployment: check method to return before considering it as failed. Defaults to 30s. logging_config: Logging config options for the deployment. If provided, the config will be used to set up the Serve logger on the deployment. - router_config: Config for the request router used for this deployment. + request_router_config: Config for the request router used for this deployment. Returns: `Deployment` """ @@ -464,7 +466,7 @@ class MyDeployment: health_check_period_s=health_check_period_s, health_check_timeout_s=health_check_timeout_s, logging_config=logging_config, - router_config=router_config, + request_router_config=request_router_config, ) deployment_config.user_configured_option_names = set(user_configured_option_names) diff --git a/python/ray/serve/config.py b/python/ray/serve/config.py index f5b1fe78a796..ed4427c1d82b 100644 --- a/python/ray/serve/config.py +++ b/python/ray/serve/config.py @@ -13,7 +13,6 @@ PositiveFloat, PositiveInt, PrivateAttr, - root_validator, validator, ) from ray._common.utils import import_attr @@ -53,18 +52,18 @@ class RequestRouterConfig(BaseModel): from ray import serve # Use default router with custom stats collection interval - router_config = RequestRouterConfig( + request_router_config = RequestRouterConfig( request_routing_stats_period_s=5.0, request_routing_stats_timeout_s=15.0 ) # Use custom router class - router_config = RequestRouterConfig( + request_router_config = RequestRouterConfig( request_router_class="ray.llm._internal.serve.request_router.prefix_aware.prefix_aware_router.PrefixAwarePow2ReplicaRouter", request_router_kwargs={"imbalanced_threshold": 20} ) deployment_config = DeploymentConfig( - router_config=router_config + request_router_config=request_router_config ) deployment = serve.deploy( "my_deployment", @@ -123,15 +122,26 @@ def request_router_kwargs_json_serializable(cls, v): return v - @root_validator - def import_and_serialize_request_router_cls(cls, values) -> Dict[str, Any]: + def __init__(self, **kwargs: dict[str, Any]): + """Initialize RequestRouterConfig with the given parameters. + + Needed to serialize the request router class since validators are not called + for attributes that begin with an underscore. + + Args: + **kwargs: Keyword arguments to pass to BaseModel. + """ + super().__init__(**kwargs) + self._serialize_request_router_cls() + + def _serialize_request_router_cls(self) -> None: """Import and serialize request router class with cloudpickle. Import the request router if you pass it in as a string import path. Then cloudpickle the request router and set to `_serialized_request_router_cls`. """ - request_router_class = values.get("request_router_class") + request_router_class = self.request_router_class if isinstance(request_router_class, Callable): request_router_class = ( f"{request_router_class.__module__}.{request_router_class.__name__}" @@ -140,11 +150,9 @@ def import_and_serialize_request_router_cls(cls, values) -> Dict[str, Any]: request_router_path = request_router_class or DEFAULT_REQUEST_ROUTER_PATH request_router_class = import_attr(request_router_path) - values["_serialized_request_router_cls"] = cloudpickle.dumps( - request_router_class - ) - values["request_router_class"] = request_router_path - return values + self._serialized_request_router_cls = cloudpickle.dumps(request_router_class) + # Update the request_router_class field to be the string path + self.request_router_class = request_router_path def get_request_router_class(self) -> Callable: """Deserialize the request router from cloudpickled bytes.""" diff --git a/python/ray/serve/deployment.py b/python/ray/serve/deployment.py index 100b8e94592c..5487ad4d0afc 100644 --- a/python/ray/serve/deployment.py +++ b/python/ray/serve/deployment.py @@ -237,7 +237,9 @@ def options( health_check_period_s: Default[float] = DEFAULT.VALUE, health_check_timeout_s: Default[float] = DEFAULT.VALUE, logging_config: Default[Union[Dict, LoggingConfig, None]] = DEFAULT.VALUE, - router_config: Default[Union[Dict, RequestRouterConfig, None]] = DEFAULT.VALUE, + request_router_config: Default[ + Union[Dict, RequestRouterConfig, None] + ] = DEFAULT.VALUE, _init_args: Default[Tuple[Any]] = DEFAULT.VALUE, _init_kwargs: Default[Dict[Any, Any]] = DEFAULT.VALUE, _internal: bool = False, @@ -349,8 +351,8 @@ def options( if autoscaling_config is not DEFAULT.VALUE: new_deployment_config.autoscaling_config = autoscaling_config - if router_config is not DEFAULT.VALUE: - new_deployment_config.router_config = router_config + if request_router_config is not DEFAULT.VALUE: + new_deployment_config.request_router_config = request_router_config if graceful_shutdown_wait_loop_s is not DEFAULT.VALUE: new_deployment_config.graceful_shutdown_wait_loop_s = ( @@ -441,7 +443,7 @@ def deployment_to_schema(d: Deployment) -> DeploymentSchema: "placement_group_bundles": d._replica_config.placement_group_bundles, "max_replicas_per_node": d._replica_config.max_replicas_per_node, "logging_config": d._deployment_config.logging_config, - "router_config": d._deployment_config.router_config, + "request_router_config": d._deployment_config.request_router_config, } # Let non-user-configured options be set to defaults. If the schema @@ -502,7 +504,7 @@ def schema_to_deployment(s: DeploymentSchema) -> Deployment: health_check_period_s=s.health_check_period_s, health_check_timeout_s=s.health_check_timeout_s, logging_config=s.logging_config, - router_config=s.router_config, + request_router_config=s.request_router_config, ) deployment_config.user_configured_option_names = ( s._get_user_configured_option_names() diff --git a/python/ray/serve/schema.py b/python/ray/serve/schema.py index c28aa48f6e45..607097fee8a7 100644 --- a/python/ray/serve/schema.py +++ b/python/ray/serve/schema.py @@ -405,7 +405,7 @@ class DeploymentSchema(BaseModel, allow_population_by_field_name=True): default=DEFAULT.VALUE, description="Logging config for configuring serve deployment logs.", ) - router_config: Union[Dict, RequestRouterConfig] = Field( + request_router_config: Union[Dict, RequestRouterConfig] = Field( default=DEFAULT.VALUE, description="Config for the request router used for this deployment.", ) @@ -487,7 +487,7 @@ def _deployment_info_to_schema(name: str, info: DeploymentInfo) -> DeploymentSch health_check_period_s=info.deployment_config.health_check_period_s, health_check_timeout_s=info.deployment_config.health_check_timeout_s, ray_actor_options=info.replica_config.ray_actor_options, - router_config=info.deployment_config.router_config, + request_router_config=info.deployment_config.request_router_config, ) if info.deployment_config.autoscaling_config is not None: @@ -1191,9 +1191,9 @@ def _get_user_facing_json_serializable_dict( for app_name, application in values["applications"].items(): for deployment_name, deployment in application["deployments"].items(): if "deployment_config" in deployment: - # Remove internal fields from router_config if it exists - if "router_config" in deployment["deployment_config"]: - deployment["deployment_config"]["router_config"].pop( + # Remove internal fields from request_router_config if it exists + if "request_router_config" in deployment["deployment_config"]: + deployment["deployment_config"]["request_router_config"].pop( "_serialized_request_router_cls", None ) if "autoscaling_config" in deployment["deployment_config"]: diff --git a/python/ray/serve/tests/test_api.py b/python/ray/serve/tests/test_api.py index 84ac736c5866..c74ee0467190 100644 --- a/python/ray/serve/tests/test_api.py +++ b/python/ray/serve/tests/test_api.py @@ -87,7 +87,7 @@ def initialize_state(self, test_parameter: int = 0): @serve.deployment( - router_config=RequestRouterConfig(request_router_class=FakeRequestRouter) + request_router_config=RequestRouterConfig(request_router_class=FakeRequestRouter) ) class AppWithCustomRequestRouter: def __call__(self) -> str: @@ -1120,7 +1120,7 @@ def test_deploy_app_with_custom_request_router(serve_instance): @serve.deployment( - router_config=RequestRouterConfig( + request_router_config=RequestRouterConfig( request_router_class="ray.serve.tests.test_api.FakeRequestRouter", request_router_kwargs=dict(test_parameter=4848), ) diff --git a/python/ray/serve/tests/test_controller.py b/python/ray/serve/tests/test_controller.py index ee77151ceb37..d3696ca521fb 100644 --- a/python/ray/serve/tests/test_controller.py +++ b/python/ray/serve/tests/test_controller.py @@ -185,7 +185,7 @@ def autoscaling_app(): "ray_actor_options": { "num_cpus": 1.0, }, - "router_config": { + "request_router_config": { "request_router_class": "ray.serve._private.request_router:PowerOfTwoChoicesRequestRouter", "request_router_kwargs": {}, "request_routing_stats_period_s": 10.0, diff --git a/python/ray/serve/tests/test_record_routing_stats.py b/python/ray/serve/tests/test_record_routing_stats.py index a4c5e893b56b..100e57859a32 100644 --- a/python/ray/serve/tests/test_record_routing_stats.py +++ b/python/ray/serve/tests/test_record_routing_stats.py @@ -13,7 +13,7 @@ @serve.deployment( - router_config=RequestRouterConfig( + request_router_config=RequestRouterConfig( request_routing_stats_period_s=0.1, request_routing_stats_timeout_s=0.1 ) ) diff --git a/python/ray/serve/tests/test_telemetry_2.py b/python/ray/serve/tests/test_telemetry_2.py index 3e4fb2d092b0..0413400d5bf2 100644 --- a/python/ray/serve/tests/test_telemetry_2.py +++ b/python/ray/serve/tests/test_telemetry_2.py @@ -160,7 +160,7 @@ def test_custom_request_router_telemetry(manage_ray_with_telemetry): check_telemetry(ServeUsageTag.CUSTOM_REQUEST_ROUTER_USED, expected=None) @serve.deployment( - router_config=RequestRouterConfig( + request_router_config=RequestRouterConfig( request_router_class=CustomRequestRouter, ), ) diff --git a/python/ray/serve/tests/unit/test_config.py b/python/ray/serve/tests/unit/test_config.py index 884a7cff289d..41ef5010eed5 100644 --- a/python/ray/serve/tests/unit/test_config.py +++ b/python/ray/serve/tests/unit/test_config.py @@ -145,25 +145,31 @@ def test_setting_and_getting_request_router_class(self): # Passing request_router_class as a class. deployment_config = DeploymentConfig.from_default( - router_config=RequestRouterConfig(request_router_class=FakeRequestRouter) + request_router_config=RequestRouterConfig( + request_router_class=FakeRequestRouter + ) ) assert ( - deployment_config.router_config.request_router_class == request_router_path + deployment_config.request_router_config.request_router_class + == request_router_path ) assert ( - deployment_config.router_config.get_request_router_class() + deployment_config.request_router_config.get_request_router_class() == FakeRequestRouter ) # Passing request_router_class as an import path. deployment_config = DeploymentConfig.from_default( - router_config=RequestRouterConfig(request_router_class=request_router_path) + request_router_config=RequestRouterConfig( + request_router_class=request_router_path + ) ) assert ( - deployment_config.router_config.request_router_class == request_router_path + deployment_config.request_router_config.request_router_class + == request_router_path ) assert ( - deployment_config.router_config.get_request_router_class() + deployment_config.request_router_config.get_request_router_class() == FakeRequestRouter ) @@ -171,11 +177,11 @@ def test_setting_and_getting_request_router_class(self): # default to `PowerOfTwoChoicesRequestRouter`. deployment_config = DeploymentConfig.from_default() assert ( - deployment_config.router_config.request_router_class + deployment_config.request_router_config.request_router_class == "ray.serve._private.request_router:PowerOfTwoChoicesRequestRouter" ) assert ( - deployment_config.router_config.get_request_router_class() + deployment_config.request_router_config.get_request_router_class() == PowerOfTwoChoicesRequestRouter ) From b4d42e052541d0eedb0e685af7b864ffc32fefd2 Mon Sep 17 00:00:00 2001 From: Seiji Eicher Date: Tue, 8 Jul 2025 11:55:33 -0700 Subject: [PATCH 35/36] Correct API stability Signed-off-by: Seiji Eicher --- 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 ed4427c1d82b..1386a99b48f4 100644 --- a/python/ray/serve/config.py +++ b/python/ray/serve/config.py @@ -33,7 +33,7 @@ logger = logging.getLogger(SERVE_LOGGER_NAME) -@PublicAPI(stability="stable") +@PublicAPI(stability="alpha") class RequestRouterConfig(BaseModel): """Config for the Serve request router. @@ -159,7 +159,7 @@ def get_request_router_class(self) -> Callable: return cloudpickle.loads(self._serialized_request_router_cls) -@PublicAPI(stability="alpha") +@PublicAPI(stability="stable") class AutoscalingConfig(BaseModel): """Config for the Serve Autoscaler.""" From c3b0d444fe8cef9bd48e3b4eb06daa9759319687 Mon Sep 17 00:00:00 2001 From: Seiji Eicher Date: Tue, 8 Jul 2025 11:56:04 -0700 Subject: [PATCH 36/36] Remove PrefixAwarePow2ReplicaRouter.__init__ Signed-off-by: Seiji Eicher --- .../prefix_aware/prefix_aware_router.py | 27 +++++++++---------- .../test_prefix_aware_request_router.py | 2 +- 2 files changed, 13 insertions(+), 16 deletions(-) diff --git a/python/ray/llm/_internal/serve/request_router/prefix_aware/prefix_aware_router.py b/python/ray/llm/_internal/serve/request_router/prefix_aware/prefix_aware_router.py index ffadef551671..5c6e8b28d504 100644 --- a/python/ray/llm/_internal/serve/request_router/prefix_aware/prefix_aware_router.py +++ b/python/ray/llm/_internal/serve/request_router/prefix_aware/prefix_aware_router.py @@ -7,6 +7,7 @@ ) import ray +from ray.actor import ActorHandle from ray.llm._internal.serve.request_router.prefix_aware.prefix_tree import ( PrefixTreeActor, ) @@ -52,21 +53,6 @@ class PrefixAwarePow2ReplicaRouter(LocalityMixin, MultiplexMixin, RequestRouter) increasing cache locality and reducing overhead for language model inference. """ - def __init__( - self, - *args, - tree_actor=None, - **kwargs, - ): - super().__init__(*args, **kwargs) - if tree_actor is None: - # Use a detached actor to avoid issues with actor lifetime since this is shared between routers - self._tree_actor = PrefixTreeActor.options( - name="LlmPrefixTreeActor", get_if_exists=True, lifetime="detached" - ).remote() - else: - self._tree_actor = tree_actor - def initialize_state( self, imbalanced_threshold: Optional[int] = 10, @@ -75,6 +61,7 @@ def initialize_state( eviction_threshold_chars: Optional[int] = 400_000, eviction_target_chars: Optional[int] = 360_000, eviction_interval_secs: Optional[int] = 10, + tree_actor: Optional[ActorHandle] = None, ): """Initialize the prefix-aware routing state and configuration. @@ -93,6 +80,8 @@ def initialize_state( prefix tree to during eviction. eviction_interval_secs: Interval in seconds between eviction checks when eviction is enabled. + tree_actor: The actor to use for the prefix tree in a test environment. + If None, a detached actor will be created/retrieved. """ # === Prefix-aware routing logic hyperparameters === self._imbalanced_threshold = imbalanced_threshold @@ -110,6 +99,14 @@ def initialize_state( ) self._eviction_interval_secs = eviction_interval_secs + if tree_actor is None: + # Use a detached actor to avoid issues with actor lifetime since this is shared between routers + self._tree_actor = PrefixTreeActor.options( + name="LlmPrefixTreeActor", get_if_exists=True, lifetime="detached" + ).remote() + else: + self._tree_actor = tree_actor + def _extract_text_from_request(self, pending_request: PendingRequest) -> str: """Extracts the text content from a pending request for prefix matching. diff --git a/python/ray/llm/tests/serve/cpu/deployments/test_prefix_aware_request_router.py b/python/ray/llm/tests/serve/cpu/deployments/test_prefix_aware_request_router.py index 4158a38140bb..e65efe879cc9 100644 --- a/python/ray/llm/tests/serve/cpu/deployments/test_prefix_aware_request_router.py +++ b/python/ray/llm/tests/serve/cpu/deployments/test_prefix_aware_request_router.py @@ -49,7 +49,6 @@ async def construct_request_router(loop: asyncio.AbstractEventLoop): handle_source=DeploymentHandleSource.REPLICA, use_replica_queue_len_cache=False, get_curr_time_s=TIMER.time, - tree_actor=tree_actor, ) return request_router @@ -63,6 +62,7 @@ async def construct_request_router(loop: asyncio.AbstractEventLoop): eviction_threshold_chars=params.get("eviction_threshold_chars"), eviction_target_chars=params.get("eviction_target_chars"), eviction_interval_secs=params.get("eviction_interval_secs"), + tree_actor=tree_actor, ) yield request_router