-
Notifications
You must be signed in to change notification settings - Fork 5.8k
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
[core] Streaming generator executor waits for item report to complete before continuing #44257
Merged
stephanie-wang
merged 3 commits into
ray-project:master
from
stephanie-wang:revert-42260
Mar 26, 2024
Merged
[core] Streaming generator executor waits for item report to complete before continuing #44257
stephanie-wang
merged 3 commits into
ray-project:master
from
stephanie-wang:revert-42260
Mar 26, 2024
Conversation
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Signed-off-by: Stephanie Wang <[email protected]>
8 tasks
alexeykudinkin
approved these changes
Mar 25, 2024
python/ray/_raylet.pyx
Outdated
Comment on lines
1446
to
1447
# we do not wait for the reply here (unless we are under | ||
# backpressure). |
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.
Please clarify that back-pressure doesn't apply to RPC Actor calls (only Ray Data)
Signed-off-by: Stephanie Wang <[email protected]>
stephanie-wang
added a commit
that referenced
this pull request
Mar 26, 2024
…44197) When a streaming generator task is cancelled, we should mark the end of the stream at the caller's current pointer into the stream. Otherwise, if we receive out-of-order item reports, we may end up hanging, because the cancelled task will never report the intermediate items. We may end up dropping some values that were already reported, but this is OK since the user already cancelled the task. Closes #43852. Note: #44257 is also enough to make the relevant flaky test stable, probably because it makes it less likely to produce out-of-order item reports. Meanwhile, this PR addresses the root cause of the flaky test, i.e. out-of-order item reports during task cancellation. --------- Signed-off-by: Stephanie Wang <[email protected]>
stephanie-wang
added a commit
to stephanie-wang/ray
that referenced
this pull request
Mar 27, 2024
…ay-project#44197) When a streaming generator task is cancelled, we should mark the end of the stream at the caller's current pointer into the stream. Otherwise, if we receive out-of-order item reports, we may end up hanging, because the cancelled task will never report the intermediate items. We may end up dropping some values that were already reported, but this is OK since the user already cancelled the task. Closes ray-project#43852. Note: ray-project#44257 is also enough to make the relevant flaky test stable, probably because it makes it less likely to produce out-of-order item reports. Meanwhile, this PR addresses the root cause of the flaky test, i.e. out-of-order item reports during task cancellation. --------- Signed-off-by: Stephanie Wang <[email protected]>
stephanie-wang
added a commit
to stephanie-wang/ray
that referenced
this pull request
Mar 27, 2024
… before continuing (ray-project#44257) ray-project#42260 updated streaming generator tasks to asynchronously report generator returns, instead of synchronously reporting each generator return before yielding the next return. However this has a couple problems: If the task still has a reference to the yielded value, it may modify the value. The serialized and reported return will then have a different value than expected. As per [core] Streaming generator task waits for all object report acks before finishing the task ray-project#44079, we need to track the number of in-flight RPCs to report generator returns, so that we can wait for them all to reply before we return from the end of the task. If we increment the count of in-flight RPCs asynchronously, we can end up returning from the task while there are still in-flight RPCs. So this PR reverts some of the logic in ray-project#42260 to wait for the generator return to be serialized into the protobuf sent back to the caller. Note that we do not wait for the reply (unless under backpressure). We can later re-introduce asynchronous generator reports, but we will need to evaluate the performance benefit of a new implementation that also addresses both of the above points. --------- Signed-off-by: Stephanie Wang <[email protected]>
Sign up for free
to join this conversation on GitHub.
Already have an account?
Sign in to comment
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
Why are these changes needed?
#42260 updated streaming generator tasks to asynchronously report generator returns, instead of synchronously reporting each generator return before yielding the next return. However this has a couple problems:
So this PR reverts some of the logic in #42260 to wait for the generator return to be serialized into the protobuf sent back to the caller. Note that we do not wait for the reply (unless under backpressure).
We can later re-introduce asynchronous generator reports, but we will need to evaluate the performance benefit of a new implementation that also addresses both of the above points.
Checks
git commit -s
) in this PR.scripts/format.sh
to lint the changes in this PR.method in Tune, I've added it in
doc/source/tune/api/
under thecorresponding
.rst
file.