berkaysynnada commented on code in PR #12302:
URL: https://github.com/apache/datafusion/pull/12302#discussion_r1741916817
##########
datafusion/physical-plan/src/sorts/merge.rs:
##########
@@ -156,12 +164,22 @@ impl<C: CursorValues> SortPreservingMergeStream<C> {
}
// try to initialize the loser tree
if self.loser_tree.is_empty() {
- // Ensure all non-exhausted streams have a cursor from which
- // rows can be pulled
- for i in 0..self.streams.partitions() {
- if let Err(e) = ready!(self.maybe_poll_stream(cx, i)) {
- self.aborted = true;
- return Poll::Ready(Some(Err(e)));
+ // Ensure all non-exhausted streams have a cursor from which rows
can be pulled
+ let remaining_partitions = self.uninitiated_partitions.clone();
+ for i in remaining_partitions {
+ match self.maybe_poll_stream(cx, i) {
+ Poll::Ready(Err(e)) => {
+ self.aborted = true;
+ return Poll::Ready(Some(Err(e)));
+ }
+ Poll::Pending => {
+ self.uninitiated_partitions.rotate_left(1);
+ cx.waker().wake_by_ref();
Review Comment:
I am not sure if this usage has some side-effects or decrease performance,
but I cannot wake the SPM poll again once it receives a pending from its first
partition
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]