-
Notifications
You must be signed in to change notification settings - Fork 5.5k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[Serve] Decrement ray_serve_deployment_queued_queries
when client disconnects
#37965
Changes from 3 commits
73960d5
1f0c58f
f6032f5
35f04c8
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -974,35 +974,48 @@ async def assign_request( | |
) -> Union[ray.ObjectRef, "ray._raylet.StreamingObjectRefGenerator"]: | ||
"""Assign a query to a replica and return the resulting object_ref.""" | ||
|
||
self.num_router_requests.inc( | ||
tags={"route": request_meta.route, "application": request_meta.app_name} | ||
) | ||
self.num_queued_queries += 1 | ||
self.num_queued_queries_gauge.set( | ||
self.num_queued_queries, | ||
tags={ | ||
"application": request_meta.app_name, | ||
}, | ||
) | ||
incremented_queue_metric = False | ||
try: | ||
self.num_router_requests.inc( | ||
tags={"route": request_meta.route, "application": request_meta.app_name} | ||
) | ||
self.num_queued_queries += 1 | ||
self.num_queued_queries_gauge.set( | ||
self.num_queued_queries, | ||
tags={ | ||
"application": request_meta.app_name, | ||
}, | ||
) | ||
incremented_queue_metric += True | ||
|
||
query = Query( | ||
args=list(request_args), | ||
kwargs=request_kwargs, | ||
metadata=request_meta, | ||
) | ||
await query.resolve_async_tasks() | ||
await query.buffer_starlette_requests_and_warn() | ||
result = await self._replica_scheduler.assign_replica(query) | ||
|
||
self.num_queued_queries -= 1 | ||
self.num_queued_queries_gauge.set( | ||
self.num_queued_queries, | ||
tags={ | ||
"application": request_meta.app_name, | ||
}, | ||
) | ||
query = Query( | ||
args=list(request_args), | ||
kwargs=request_kwargs, | ||
metadata=request_meta, | ||
) | ||
await query.resolve_async_tasks() | ||
await query.buffer_starlette_requests_and_warn() | ||
result = await self._replica_scheduler.assign_replica(query) | ||
|
||
self.num_queued_queries -= 1 | ||
self.num_queued_queries_gauge.set( | ||
self.num_queued_queries, | ||
tags={ | ||
"application": request_meta.app_name, | ||
}, | ||
) | ||
|
||
return result | ||
return result | ||
except asyncio.CancelledError: | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. would be an issue if different exceptions happens? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yes, it would. I changed the code to a |
||
if incremented_queue_metric: | ||
self.num_queued_queries -= 1 | ||
self.num_queued_queries_gauge.set( | ||
self.num_queued_queries, | ||
tags={ | ||
"application": request_meta.app_name, | ||
}, | ||
) | ||
raise | ||
|
||
def shutdown(self): | ||
"""Shutdown router gracefully. | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
add
try
here, so that we don't need to haveincremented_queue_metric
.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Good idea, that makes the code simpler. I made the change.