-
Notifications
You must be signed in to change notification settings - Fork 5.7k
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][adag] Support asyncio.gather on multiple CompiledDAGFutures (#47684) #47860
[core][adag] Support asyncio.gather on multiple CompiledDAGFutures (#47684) #47860
Conversation
…multiple CompiledDAGFutures Signed-off-by: jeffreyjeffreywang <jeffjeffreywang@gmail.com>
Hey @rkooo567 @stephanie-wang, could you please take a look once you have time? This addresses the issue of not able to invoke |
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.
Thanks for the fix, @jeffreyjeffreywang!
Can you measure throughput before/after? A little worried about the lock overhead and dynamic lock allocation.
Also, can you explain what data/code the lock protects? Is there a lock-free design that can work?
Thanks for the review, @stephanie-wang! The locks protects Imagine that there are two output channels, and I can add some logic to avoid using lock when there is a single output channel so that we don't influence the performance when |
Ah I see, thanks for the explanation! Seems like it could also be fixed by checking again if the result is still not cached after
Hmm unfortunately I don't think we have an asyncio microbenchmark. But you can take a look at the tests that are run on nightly for compiled graph. Actually adding an asyncio version of this microbenchmark would be useful (I'll file an issue for this if you're interested). |
Yup, we could simply check whether the result has been cached in |
Signed-off-by: jeffreyjeffreywang <jeffjeffreywang@gmail.com>
Signed-off-by: jeffreyjeffreywang <jeffjeffreywang@gmail.com>
Signed-off-by: jeffreyjeffreywang <jeffjeffreywang@gmail.com>
Hey @stephanie-wang, this is now ready for another review. |
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.
Oops sorry for the delay in review. This looks great, thanks!
there are test failures! |
btw is the description still relevant? I am not seeing the lock from code |
@stephanie-wang no worries, I was busy dealing with work as well. Thanks for the review! |
…-ref Signed-off-by: jeffreyjeffreywang <jeffjeffreywang@gmail.com>
Hey @jeffreyjeffreywang FYI I re-merged master, not sure why there was a core test failure in your earlier merge. |
…ay-project#47684) (ray-project#47860) Currently, awaiting individual `CompiledDAGFuture` works but performing `asyncio.gather` on a list of `CompiledDAGFuture` won't because there is a race condition between `CompiledDAGFuture`s that originate from the same execution in a aDAG. One solution is to relax the assertion in `_cache_execution_results`, allowing multiple `CompiledDAGFuture` originating from the same execution to attempt to cache the result for that execution. The other solution is to protect the critical section with `asyncio.Lock`. The first solution could result in awaiting on the same future multiple times, but since this overhead is lower than that of the second solution (i.e. locks introduce more overhead), we go for the first solution. Resolves ray-project#47684. --------- Signed-off-by: jeffreyjeffreywang <jeffjeffreywang@gmail.com> Co-authored-by: jeffreyjeffreywang <jeffjeffreywang@gmail.com> Co-authored-by: Stephanie Wang <swang@cs.berkeley.edu>
Why are these changes needed?
Currently, awaiting individual
CompiledDAGFuture
works but performingasyncio.gather
on a list ofCompiledDAGFuture
won't because there is a race condition betweenCompiledDAGFuture
s that originate from the same execution in a aDAG. One solution is to relax the assertion in_cache_execution_results
, allowing multipleCompiledDAGFuture
originating from the same execution to attempt to cache the result for that execution. The other solution is to protect the critical section withasyncio.Lock
. The first solution could result in awaiting on the same future multiple times, but since this overhead is lower than that of the second solution (i.e. locks introduce more overhead), we go for the first solution.Related issue number
Resolves #47684.
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.