Skip to content

Commit e466c6f

Browse files
authored
use correct route value in proxy metrics tags instead of route_prefix (#58180)
## Expose Route Patterns in Proxy Metrics fixes #52212 ### Problem Proxy metrics (`ray_serve_num_http_requests_total`, `ray_serve_http_request_latency_ms`) only expose `route_prefix` (e.g., `/api`) instead of actual route patterns (e.g., `/api/users/{user_id}`). This prevents granular monitoring of individual endpoints without causing high cardinality from unique request paths. ### Design **Route Pattern Extraction & Propagation:** - Replicas extract route patterns from ASGI apps (FastAPI/Starlette) at initialization using `extract_route_patterns()` - Patterns propagate: Replica → `ReplicaMetadata` → `DeploymentState` → `EndpointInfo` → Proxy - Works with both normal patterns (routes in class) and factory patterns (callable returns app) **Proxy Route Matching:** - `ProxyRouter.match_route_pattern()` matches incoming requests to specific patterns using cached mock Starlette apps - Metrics tag requests with parameterized routes (e.g., `/api/users/{user_id}`) instead of prefixes - Fallback to `route_prefix` if patterns unavailable or matching fails **Performance:** Metric | Before | After -- | -- | -- Requests per second (RPS) | 403.39 | 397.82 Mean latency (ms) | 247.9 | 251.37 p50 (ms) | 224 | 223 p90 (ms) | 415 | 428 p99 (ms) | 526 | 544 ### Testing - Unit tests for `extract_route_patterns()` - Integration test verifying metrics use patterns and avoid high cardinality - Parametrized for both normal and factory patterns --------- Signed-off-by: abrar <abrar@anyscale.com>
1 parent 6dd3776 commit e466c6f

File tree

13 files changed

+821
-14
lines changed

13 files changed

+821
-14
lines changed

python/ray/serve/_private/application_state.py

Lines changed: 8 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -538,6 +538,13 @@ def apply_deployment_info(
538538

539539
if deployment_info.route_prefix is not None:
540540
config = deployment_info.deployment_config
541+
# Try to get route_patterns from deployment state first (most up-to-date),
542+
# otherwise fall back to existing endpoint patterns
543+
route_patterns = (
544+
self._deployment_state_manager.get_deployment_route_patterns(
545+
deployment_id
546+
)
547+
)
541548
self._endpoint_state.update_endpoint(
542549
deployment_id,
543550
# The current meaning of the "is_cross_language" field is ambiguous.
@@ -550,6 +557,7 @@ def apply_deployment_info(
550557
route=deployment_info.route_prefix,
551558
app_is_cross_language=config.deployment_language
552559
!= DeploymentLanguage.PYTHON,
560+
route_patterns=route_patterns,
553561
),
554562
)
555563
else:

python/ray/serve/_private/common.py

Lines changed: 32 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -101,8 +101,40 @@ def __str__(self) -> str:
101101

102102
@dataclass
103103
class EndpointInfo:
104+
"""Metadata about a deployment's HTTP/gRPC endpoint.
105+
106+
This represents the public routing interface for a deployment. It's created when
107+
a deployment is registered with a route prefix and broadcast to all proxies via
108+
the long poll mechanism (ROUTE_TABLE namespace).
109+
110+
Flow:
111+
1. Created in ApplicationState when deployment is applied
112+
2. Stored in EndpointState (controller's source of truth)
113+
3. Broadcast to all ProxyActors via long poll (ROUTE_TABLE)
114+
4. Cached in ProxyRouter for request routing
115+
5. Used to route incoming HTTP/gRPC requests to correct deployments
116+
6. Used to determine route patterns for accurate metrics tagging
117+
118+
Key Difference from DeploymentInfo:
119+
- EndpointInfo: Just HTTP/gRPC routing metadata (shared with proxies)
120+
- DeploymentInfo: Complete deployment config (replicas, resources, etc.)
121+
122+
Attributes:
123+
route: The route prefix for this deployment (e.g., "/api").
124+
app_is_cross_language: Whether the deployment uses a different language
125+
than the proxy (e.g., Java deployment with Python proxy). This affects
126+
how the proxy serializes/deserializes requests.
127+
route_patterns: List of all ASGI route patterns for this deployment
128+
(e.g., ["/", "/users/{user_id}", "/items/{item_id}/details"]).
129+
Used by proxies to match incoming requests to specific route patterns
130+
for accurate metrics tagging. This avoids high cardinality by using
131+
parameterized patterns instead of individual request paths.
132+
Only populated for deployments with ASGI apps (FastAPI/Starlette).
133+
"""
134+
104135
route: str
105136
app_is_cross_language: bool = False
137+
route_patterns: Optional[List[str]] = None
106138

107139

108140
# Keep in sync with ServeReplicaState in dashboard/client/src/type/serve.ts

python/ray/serve/_private/deployment_state.py

Lines changed: 25 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -251,6 +251,7 @@ def __init__(
251251
self._initialization_latency_s: Optional[float] = None
252252
self._internal_grpc_port: Optional[int] = None
253253
self._docs_path: Optional[str] = None
254+
self._route_patterns: Optional[List[str]] = None
254255
# Rank assigned to the replica.
255256
self._rank: Optional[int] = None
256257
# Populated in `on_scheduled` or `recover`.
@@ -348,6 +349,10 @@ def deployment_config(self) -> DeploymentConfig:
348349
def docs_path(self) -> Optional[str]:
349350
return self._docs_path
350351

352+
@property
353+
def route_patterns(self) -> Optional[List[str]]:
354+
return self._route_patterns
355+
351356
@property
352357
def max_ongoing_requests(self) -> int:
353358
return self.deployment_config.max_ongoing_requests
@@ -768,6 +773,7 @@ def check_ready(self) -> Tuple[ReplicaStartupStatus, Optional[str]]:
768773
self._http_port,
769774
self._grpc_port,
770775
self._rank,
776+
self._route_patterns,
771777
) = ray.get(self._ready_obj_ref)
772778
except RayTaskError as e:
773779
logger.exception(
@@ -1126,6 +1132,10 @@ def version(self):
11261132
def docs_path(self) -> Optional[str]:
11271133
return self._actor.docs_path
11281134

1135+
@property
1136+
def route_patterns(self) -> Optional[List[str]]:
1137+
return self._actor.route_patterns
1138+
11291139
@property
11301140
def actor_id(self) -> str:
11311141
return self._actor.actor_id
@@ -1773,6 +1783,7 @@ def __init__(
17731783
self._last_broadcasted_deployment_config = None
17741784

17751785
self._docs_path: Optional[str] = None
1786+
self._route_patterns: Optional[List[str]] = None
17761787

17771788
def should_autoscale(self) -> bool:
17781789
"""
@@ -1865,6 +1876,10 @@ def app_name(self) -> str:
18651876
def docs_path(self) -> Optional[str]:
18661877
return self._docs_path
18671878

1879+
@property
1880+
def route_patterns(self) -> Optional[List[str]]:
1881+
return self._route_patterns
1882+
18681883
@property
18691884
def _failed_to_start_threshold(self) -> int:
18701885
# Use global override if set, otherwise use deployment config
@@ -2538,9 +2553,10 @@ def _check_startup_replicas(
25382553
)
25392554

25402555
# if replica version is the same as the target version,
2541-
# we update the docs path
2556+
# we update the docs path and route patterns
25422557
if replica.version == self._target_state.version:
25432558
self._docs_path = replica.docs_path
2559+
self._route_patterns = replica.route_patterns
25442560

25452561
# Log the startup latency.
25462562
e2e_replica_start_latency = time.time() - replica._start_time
@@ -3191,6 +3207,14 @@ def get_deployment_docs_path(self, deployment_id: DeploymentID) -> Optional[str]
31913207
if deployment_id in self._deployment_states:
31923208
return self._deployment_states[deployment_id].docs_path
31933209

3210+
def get_deployment_route_patterns(
3211+
self, deployment_id: DeploymentID
3212+
) -> Optional[List[str]]:
3213+
"""Get route patterns for a deployment if available."""
3214+
if deployment_id in self._deployment_states:
3215+
return self._deployment_states[deployment_id].route_patterns
3216+
return None
3217+
31943218
def get_deployment_target_num_replicas(
31953219
self, deployment_id: DeploymentID
31963220
) -> Optional[int]:

python/ray/serve/_private/proxy.py

Lines changed: 11 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -375,14 +375,17 @@ def _get_response_handler_info(
375375
if version.parse(starlette.__version__) < version.parse("0.33.0"):
376376
proxy_request.set_path(route_path.replace(route_prefix, "", 1))
377377

378-
# NOTE(edoakes): we use the route_prefix instead of the full HTTP path
379-
# for logs & metrics to avoid high cardinality.
380-
# See: https://github.com/ray-project/ray/issues/47999
381-
logs_and_metrics_route = (
382-
route_prefix
383-
if self.protocol == RequestProtocol.HTTP
384-
else handle.deployment_id.app_name
385-
)
378+
# NOTE(abrar): we try to match to a specific route pattern (e.g., /api/{user_id})
379+
# for logs & metrics when available. If no pattern matches, we fall back to the
380+
# route_prefix to avoid high cardinality.
381+
# See: https://github.com/ray-project/ray/issues/47999 and
382+
# https://github.com/ray-project/ray/issues/52212
383+
if self.protocol == RequestProtocol.HTTP:
384+
logs_and_metrics_route = self.proxy_router.match_route_pattern(
385+
route_prefix, proxy_request.scope
386+
)
387+
else:
388+
logs_and_metrics_route = handle.deployment_id.app_name
386389
internal_request_id = generate_request_id()
387390
handle, request_id = self.setup_request_context_and_handle(
388391
app_name=handle.deployment_id.app_name,

python/ray/serve/_private/proxy_router.py

Lines changed: 81 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -1,8 +1,14 @@
11
import logging
2-
from typing import Callable, Dict, List, Optional, Tuple
2+
from typing import Any, Callable, Dict, List, Optional, Tuple
3+
4+
from starlette.applications import Starlette
5+
from starlette.requests import Request
6+
from starlette.routing import Route
7+
from starlette.types import Scope
38

49
from ray.serve._private.common import ApplicationName, DeploymentID, EndpointInfo
510
from ray.serve._private.constants import SERVE_LOGGER_NAME
11+
from ray.serve._private.thirdparty.get_asgi_route_name import get_asgi_route_name
612
from ray.serve.handle import DeploymentHandle
713

814
logger = logging.getLogger(SERVE_LOGGER_NAME)
@@ -38,6 +44,13 @@ def __init__(
3844
# Endpoints info associated with endpoints.
3945
self.endpoints: Dict[DeploymentID, EndpointInfo] = dict()
4046

47+
# Map of route prefix to list of route patterns for that endpoint
48+
# Used to match incoming requests to ASGI route patterns for metrics
49+
self.route_patterns: Dict[str, List[str]] = dict()
50+
# Cache of mock Starlette apps for route pattern matching
51+
# Key: route prefix, Value: pre-built Starlette app with routes
52+
self._route_pattern_apps: Dict[str, Any] = dict()
53+
4154
def ready_for_traffic(self, is_head: bool) -> Tuple[bool, str]:
4255
"""Whether the proxy router is ready to serve traffic.
4356
@@ -80,10 +93,13 @@ def update_routes(self, endpoints: Dict[DeploymentID, EndpointInfo]):
8093
routes = []
8194
route_info = {}
8295
app_to_is_cross_language = {}
96+
route_patterns = {}
8397
for endpoint, info in endpoints.items():
8498
routes.append(info.route)
8599
route_info[info.route] = endpoint
86100
app_to_is_cross_language[endpoint.app_name] = info.app_is_cross_language
101+
if info.route_patterns:
102+
route_patterns[info.route] = info.route_patterns
87103
if endpoint in self.handles:
88104
existing_handles.remove(endpoint)
89105
else:
@@ -103,6 +119,9 @@ def update_routes(self, endpoints: Dict[DeploymentID, EndpointInfo]):
103119
self.sorted_routes = sorted(routes, key=lambda x: len(x), reverse=True)
104120
self.route_info = route_info
105121
self.app_to_is_cross_language = app_to_is_cross_language
122+
self.route_patterns = route_patterns
123+
# Invalidate cached mock apps when route patterns change
124+
self._route_pattern_apps.clear()
106125

107126
def match_route(
108127
self, target_route: str
@@ -163,3 +182,64 @@ def get_handle_for_endpoint(
163182
)
164183

165184
return None
185+
186+
def match_route_pattern(self, route_prefix: str, asgi_scope: Scope) -> str:
187+
"""Match an incoming request to a specific route pattern.
188+
189+
This attempts to match the request path to a route pattern (e.g., /api/{user_id})
190+
rather than just the route prefix. This provides more granular metrics.
191+
192+
The mock Starlette app is cached per route_prefix for performance, avoiding
193+
the overhead of recreating the app and routes on every request.
194+
195+
Args:
196+
route_prefix: The matched route prefix from match_route()
197+
asgi_scope: The ASGI scope containing the request path and method
198+
199+
Returns:
200+
The matched route pattern if available, otherwise the route_prefix
201+
"""
202+
# If we don't have route patterns for this prefix, return the prefix
203+
if route_prefix not in self.route_patterns:
204+
return route_prefix
205+
206+
patterns = self.route_patterns[route_prefix]
207+
if not patterns:
208+
return route_prefix
209+
210+
# Get or create the cached mock app for this route_prefix
211+
mock_app = self._route_pattern_apps.get(route_prefix)
212+
if mock_app is None:
213+
try:
214+
# Create routes from patterns
215+
# We use a dummy endpoint since we only need pattern matching
216+
async def dummy_endpoint(request: Request):
217+
pass
218+
219+
routes = [Route(pattern, dummy_endpoint) for pattern in patterns]
220+
mock_app = Starlette(routes=routes)
221+
222+
# Cache the mock app for future requests
223+
self._route_pattern_apps[route_prefix] = mock_app
224+
except Exception:
225+
# If app creation fails, fall back to route prefix
226+
logger.debug(
227+
f"Failed to create mock app for route pattern matching: {route_prefix}",
228+
exc_info=True,
229+
)
230+
return route_prefix
231+
232+
# Use the cached mock app to match the route pattern
233+
try:
234+
matched = get_asgi_route_name(mock_app, asgi_scope)
235+
if matched:
236+
return matched
237+
except Exception:
238+
# If matching fails for any reason, fall back to route prefix
239+
logger.debug(
240+
f"Failed to match route pattern for {route_prefix}",
241+
exc_info=True,
242+
)
243+
244+
# Fall back to route prefix if no pattern matched
245+
return route_prefix

python/ray/serve/_private/replica.py

Lines changed: 17 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -20,6 +20,7 @@
2020
Callable,
2121
Dict,
2222
Generator,
23+
List,
2324
Optional,
2425
Tuple,
2526
Union,
@@ -93,7 +94,10 @@
9394
)
9495
from ray.serve._private.metrics_utils import InMemoryMetricsStore, MetricsPusher
9596
from ray.serve._private.task_consumer import TaskConsumerWrapper
96-
from ray.serve._private.thirdparty.get_asgi_route_name import get_asgi_route_name
97+
from ray.serve._private.thirdparty.get_asgi_route_name import (
98+
extract_route_patterns,
99+
get_asgi_route_name,
100+
)
97101
from ray.serve._private.usage import ServeUsageTag
98102
from ray.serve._private.utils import (
99103
Semaphore,
@@ -121,6 +125,9 @@
121125
Optional[int],
122126
Optional[str],
123127
int,
128+
int,
129+
int, # rank
130+
Optional[List[str]], # route_patterns
124131
]
125132

126133

@@ -573,6 +580,14 @@ def get_num_ongoing_requests(self) -> int:
573580

574581
def get_metadata(self) -> ReplicaMetadata:
575582
current_rank = ray.serve.context._get_internal_replica_context().rank
583+
# Extract route patterns from ASGI app if available
584+
route_patterns = None
585+
if self._user_callable_asgi_app is not None:
586+
# _user_callable_asgi_app is the actual ASGI app (FastAPI/Starlette)
587+
# It's set when initialize_callable() returns an ASGI app
588+
if hasattr(self._user_callable_asgi_app, "routes"):
589+
route_patterns = extract_route_patterns(self._user_callable_asgi_app)
590+
576591
return (
577592
self._version.deployment_config,
578593
self._version,
@@ -582,6 +597,7 @@ def get_metadata(self) -> ReplicaMetadata:
582597
self._http_port,
583598
self._grpc_port,
584599
current_rank,
600+
route_patterns,
585601
)
586602

587603
def _set_internal_replica_context(

0 commit comments

Comments
 (0)