Skip to content
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] recursive cancellation #47873

Merged
merged 19 commits into from
Oct 29, 2024
Merged

Conversation

zcin
Copy link
Contributor

@zcin zcin commented Sep 30, 2024

Why are these changes needed?

Implement full recursive cancellation at Serve layer.

Currently, at the time that a parent request is cancelled, if:

  • The child request has not been assigned to a replica, meaning the replica scheduler in the router is still processing that request
  • AND the parent request is not directly blocking and waiting on the result of the child request

Then the child request will not be correctly cancelled. Note that if the second bullet point were actually flipped, i.e. the parent request was waiting on the result at the time of cancellation, then an asyncio.CancelledError will be sent directly to the replica scheduler and the child request will get successfully cancelled. However obviously this cannot be guaranteed to be true.

This PR adds a cache for asyncio tasks that are launched in the router to assign a request to a replica. When a parent request is cancelled, if there are any "pending assignment requests" in the cache, they are cancelled.

Related issue number

Checks

  • I've signed off every commit(by using the -s flag, i.e., git commit -s) in this PR.
  • I've run scripts/format.sh to lint the changes in this PR.
  • I've included any doc changes needed for https://docs.ray.io/en/master/.
    • I've added any new APIs to the API Reference. For example, if I added a
      method in Tune, I've added it in doc/source/tune/api/ under the
      corresponding .rst file.
  • I've made sure the tests are passing. Note that there might be a few flaky tests, see the recent failures at https://flakey-tests.ray.io/
  • Testing Strategy
    • Unit tests
    • Release tests
    • This PR is not tested :(

@zcin zcin marked this pull request as ready for review October 25, 2024 19:05
@zcin zcin requested review from GeneDer and edoakes October 25, 2024 19:06
Signed-off-by: Cindy Zhang <[email protected]>
@@ -365,5 +365,104 @@ async def get_out_of_band_response(self):
assert h.get_out_of_band_response.remote().result() == "ok"


def test_recursive_cancellation_during_execution(serve_instance):
Copy link
Contributor

Choose a reason for hiding this comment

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

in order for these tests to be meaningful, don't we need to turn off the core cancellation support?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

So at first I took the approach of implementing full recursive cancellation at the Serve layer, but that leads to more of a performance drop. I think it's unnecessary to cover the part that core already covers (recursive task cancellation). We just need to take care of canceling the asyncio request assignment task before a request has been assigned to a replica.

So in this test file test_recursive_cancellation_during_execution already passes, and test_recursive_cancellation_during_assignment is the one that doesn't pass without this PR.

python/ray/serve/_private/replica.py Outdated Show resolved Hide resolved
@@ -594,6 +599,17 @@ async def assign_request(
) -> ReplicaResult:
"""Assign a request to a replica and return the resulting object_ref."""

response_id = uuid.uuid4()
Copy link
Contributor

Choose a reason for hiding this comment

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

Non-blocker: There is this generate_request_id() maybe we can rename and reuse here?

python/ray/serve/context.py Outdated Show resolved Hide resolved
python/ray/serve/context.py Show resolved Hide resolved
response_id = uuid.uuid4()
assign_request_task = asyncio.current_task()
ray.serve.context._add_request_pending_assignment(
request_meta.request_id, response_id, assign_request_task
Copy link
Contributor

Choose a reason for hiding this comment

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

We should probably use internal_request_id for this. request_id can be set by the client from the header and can be repeated/ other unexpected behavior.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

nice thanks for the suggestion, changed to use internal request id!

python/ray/serve/_private/router.py Outdated Show resolved Hide resolved
python/ray/serve/context.py Outdated Show resolved Hide resolved
python/ray/serve/context.py Show resolved Hide resolved
Copy link
Contributor

@GeneDer GeneDer left a comment

Choose a reason for hiding this comment

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

:shipit:

Signed-off-by: Cindy Zhang <[email protected]>
@zcin zcin added the go add ONLY when ready to merge, run all tests label Oct 28, 2024
Copy link
Contributor

@GeneDer GeneDer left a comment

Choose a reason for hiding this comment

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

Thanks Cindy!

Signed-off-by: Cindy Zhang <[email protected]>
@zcin zcin merged commit cd4c34a into ray-project:master Oct 29, 2024
5 checks passed
Jay-ju pushed a commit to Jay-ju/ray that referenced this pull request Nov 5, 2024
## Why are these changes needed?

Implement full recursive cancellation at Serve layer.

Currently, at the time that a parent request is cancelled, if:
- The child request has not been assigned to a replica, _meaning_ the
replica scheduler in the router is still processing that request
- AND the parent request is not directly blocking and waiting on the
result of the child request

Then the child request will not be correctly cancelled. Note that if the
second bullet point were actually flipped, i.e. the parent request _was_
waiting on the result at the time of cancellation, then an
`asyncio.CancelledError` will be sent directly to the replica scheduler
and the child request will get successfully cancelled. However obviously
this cannot be guaranteed to be true.

This PR adds a cache for asyncio tasks that are launched in the router
to assign a request to a replica. When a parent request is cancelled, if
there are any "pending assignment requests" in the cache, they are
cancelled.



Signed-off-by: Cindy Zhang <[email protected]>
JP-sDEV pushed a commit to JP-sDEV/ray that referenced this pull request Nov 14, 2024
## Why are these changes needed?

Implement full recursive cancellation at Serve layer.

Currently, at the time that a parent request is cancelled, if:
- The child request has not been assigned to a replica, _meaning_ the
replica scheduler in the router is still processing that request
- AND the parent request is not directly blocking and waiting on the
result of the child request

Then the child request will not be correctly cancelled. Note that if the
second bullet point were actually flipped, i.e. the parent request _was_
waiting on the result at the time of cancellation, then an
`asyncio.CancelledError` will be sent directly to the replica scheduler
and the child request will get successfully cancelled. However obviously
this cannot be guaranteed to be true.

This PR adds a cache for asyncio tasks that are launched in the router
to assign a request to a replica. When a parent request is cancelled, if
there are any "pending assignment requests" in the cache, they are
cancelled.



Signed-off-by: Cindy Zhang <[email protected]>
mohitjain2504 pushed a commit to mohitjain2504/ray that referenced this pull request Nov 15, 2024
## Why are these changes needed?

Implement full recursive cancellation at Serve layer.

Currently, at the time that a parent request is cancelled, if:
- The child request has not been assigned to a replica, _meaning_ the
replica scheduler in the router is still processing that request
- AND the parent request is not directly blocking and waiting on the
result of the child request

Then the child request will not be correctly cancelled. Note that if the
second bullet point were actually flipped, i.e. the parent request _was_
waiting on the result at the time of cancellation, then an
`asyncio.CancelledError` will be sent directly to the replica scheduler
and the child request will get successfully cancelled. However obviously
this cannot be guaranteed to be true.

This PR adds a cache for asyncio tasks that are launched in the router
to assign a request to a replica. When a parent request is cancelled, if
there are any "pending assignment requests" in the cache, they are
cancelled.

Signed-off-by: Cindy Zhang <[email protected]>
Signed-off-by: mohitjain2504 <[email protected]>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
go add ONLY when ready to merge, run all tests
Projects
None yet
Development

Successfully merging this pull request may close these issues.

3 participants