-
Notifications
You must be signed in to change notification settings - Fork 1.5k
Document guidelines for physical operator yielding #15030
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
Conversation
To start a policy of the behavior physical operator streams should have and drive improvements in this area to allow for timely cancellation. Connects to apache#14036 and related to pull requests such as apache#14028.
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.
Thank you so much @carols10cents -- I think this is super valuable, and in my opinion is not changing the DataFusion semantics, but simply documenting what is implicitly already done in the code.
cc @berkaysynnada / @ozankabak in case you have other thoughts in this area.
/// batches. | ||
/// | ||
/// The goal is for `datafusion`-provided operator implementation to | ||
/// strive for [the guideline of not spending a long time without reaching |
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.
I don't know if there is any implementation such that it manually yields because of spending long time without a yield point (for CPU bound works), and I am also not very sure we could need that.
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.
I'm not sure if I'm understanding what you're saying correctly-- but I believe datafusion does need this as demonstrated by the cancellation benchmarks I recently added that show there's at least one case where it takes 32ms (on @alamb's machine) to cancel/drop the runtime because there are operations that aren't yielding often enough (the guidelines I've seen suggest aiming for 1ms of work between yield points).
There are also issues such as #14036 where queries aren't able to be cancelled, and the root cause also appears to be not yielding often enough.
Could you elaborate on why you don't think datafusion needs manual yields, based on the behavior of the benchmarks and uncancellable query issues?
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.
I think what he means is that in most cases a manual yield shouldn't be necessary for CPU-bound operators in most cases (not all). It is possible for this to be necessary in certain situations (very large batch sizes, operator does superlinear-complexity work w.r.t. batch size etc.), but it shouldn't be a common situation. I think the documentation should probably state this, and give some concrete examples when manual yielding may be necessary.
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.
I think what he means is that in most cases a manual yield shouldn't be necessary for CPU-bound operators in most cases (not all).
I think we can state that if you work packages are record batches and your compute complexity is linear (like filter and map/project operation), then you probably don't need this. But if you do any form of aggregation or super-linear behavior (e.g. unnest, data decompression), then you must think about that issue.
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.
Could someone make a concrete suggestion here on how the text should be changed for discussion?
Thanks for improving the docs, left my suggestions inline |
I just pushed some more commits addressing some comments; there is one TODO commit in there that I will update once #15054 has been merged in so that I can link to the relevant part of the benchmarks readme that I added in that PR. I will squash all these commits when we're done revising. |
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.
Left a suggestion per your request
#15054 has been merged. @carols10cents are you willing to update this PR again? If not I can do so too |
Co-authored-by: Mehmet Ozan Kabak <[email protected]>
Whoops, just updated! I also took @ozankabak's suggestion since there were no objections; thank you! How is this looking now? |
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.
LGTM - thank you!
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.
Thank you @carols10cents and @ozankabak @berkaysynnada and @crepererum
🚀 📖 |
To start a policy of the behavior physical operator streams should have and drive improvements in this area to allow for timely cancellation.
I'm very open to any wording changes and looking forward to discussion on whether I've accurately captured the intent of datafusion!
Which issue does this PR close?
Connects to #14036 and related to pull requests such as #14028.
Rationale for this change
Datafusion should be explicit about its aspirations for what it provides as well as provide guidelines for trait implementers about yielding to enable timely cancellation.
What changes are included in this PR?
A policy statement in the documentation with some reference links.
Are these changes tested?
No, this is only documentation.
Are there any user-facing changes?
This is the documentation :)