- 
                Notifications
    
You must be signed in to change notification settings  - Fork 1.7k
 
bug: remove busy-wait while sort is ongoing #16322
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
          
     Merged
      
      
    
  
     Merged
                    Changes from all commits
      Commits
    
    
            Show all changes
          
          
            3 commits
          
        
        Select commit
          Hold shift + click to select a range
      
      
    File filter
Filter by extension
Conversations
          Failed to load comments.   
        
        
          
      Loading
        
  Jump to
        
          Jump to file
        
      
      
          Failed to load files.   
        
        
          
      Loading
        
  Diff view
Diff view
There are no files selected for viewing
  
    
      This file contains hidden or 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
    
  
  
    
              | Original file line number | Diff line number | Diff line change | 
|---|---|---|
| 
          
            
          
           | 
    @@ -18,7 +18,6 @@ | |
| //! Merge that deals with an arbitrary size of streaming inputs. | ||
| //! This is an order-preserving merge. | ||
| 
     | 
||
| use std::collections::VecDeque; | ||
| use std::pin::Pin; | ||
| use std::sync::Arc; | ||
| use std::task::{ready, Context, Poll}; | ||
| 
          
            
          
           | 
    @@ -143,11 +142,8 @@ pub(crate) struct SortPreservingMergeStream<C: CursorValues> { | |
| /// number of rows produced | ||
| produced: usize, | ||
| 
     | 
||
| /// This queue contains partition indices in order. When a partition is polled and returns `Poll::Ready`, | ||
| /// it is removed from the vector. If a partition returns `Poll::Pending`, it is moved to the end of the | ||
| /// vector to ensure the next iteration starts with a different partition, preventing the same partition | ||
| /// from being continuously polled. | ||
| uninitiated_partitions: VecDeque<usize>, | ||
| /// This vector contains the indices of the partitions that have not started emitting yet. | ||
| uninitiated_partitions: Vec<usize>, | ||
| } | ||
| 
     | 
||
| impl<C: CursorValues> SortPreservingMergeStream<C> { | ||
| 
          
            
          
           | 
    @@ -216,36 +212,50 @@ impl<C: CursorValues> SortPreservingMergeStream<C> { | |
| // Once all partitions have set their corresponding cursors for the loser tree, | ||
| // we skip the following block. Until then, this function may be called multiple | ||
| // times and can return Poll::Pending if any partition returns Poll::Pending. | ||
| 
     | 
||
| if self.loser_tree.is_empty() { | ||
| while let Some(&partition_idx) = self.uninitiated_partitions.front() { | ||
| // Manual indexing since we're iterating over the vector and shrinking it in the loop | ||
| let mut idx = 0; | ||
| while idx < self.uninitiated_partitions.len() { | ||
| let partition_idx = self.uninitiated_partitions[idx]; | ||
| match self.maybe_poll_stream(cx, partition_idx) { | ||
| Poll::Ready(Err(e)) => { | ||
| self.aborted = true; | ||
| return Poll::Ready(Some(Err(e))); | ||
| } | ||
| Poll::Pending => { | ||
| // If a partition returns Poll::Pending, to avoid continuously polling it | ||
| // and potentially increasing upstream buffer sizes, we move it to the | ||
| // back of the polling queue. | ||
| self.uninitiated_partitions.rotate_left(1); | ||
| 
     | 
||
| // This function could remain in a pending state, so we manually wake it here. | ||
| // However, this approach can be investigated further to find a more natural way | ||
| // to avoid disrupting the runtime scheduler. | ||
| cx.waker().wake_by_ref(); | ||
| return Poll::Pending; | ||
| // The polled stream is pending which means we're already set up to | ||
| // be woken when necessary | ||
| // Try the next stream | ||
| idx += 1; | ||
| } | ||
| _ => { | ||
| // If the polling result is Poll::Ready(Some(batch)) or Poll::Ready(None), | ||
| // we remove this partition from the queue so it is not polled again. | ||
| self.uninitiated_partitions.pop_front(); | ||
| // The polled stream is ready | ||
| 
         There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. thank you for these comments 👍  | 
||
| // Remove it from uninitiated_partitions | ||
| // Don't bump idx here, since a new element will have taken its | ||
| // place which we'll try in the next loop iteration | ||
| // swap_remove will change the partition poll order, but that shouldn't | ||
| // make a difference since we're waiting for all streams to be ready. | ||
| self.uninitiated_partitions.swap_remove(idx); | ||
| } | ||
| } | ||
| } | ||
| 
     | 
||
| // Claim the memory for the uninitiated partitions | ||
| self.uninitiated_partitions.shrink_to_fit(); | ||
| self.init_loser_tree(); | ||
| if self.uninitiated_partitions.is_empty() { | ||
| // If there are no more uninitiated partitions, set up the loser tree and continue | ||
| // to the next phase. | ||
| 
     | 
||
| // Claim the memory for the uninitiated partitions | ||
| self.uninitiated_partitions.shrink_to_fit(); | ||
| self.init_loser_tree(); | ||
| } else { | ||
| // There are still uninitiated partitions so return pending. | ||
| // We only get here if we've polled all uninitiated streams and at least one of them | ||
| // returned pending itself. That means we will be woken as soon as one of the | ||
| // streams would like to be polled again. | ||
| // There is no need to reschedule ourselves eagerly. | ||
| return Poll::Pending; | ||
| } | ||
| } | ||
| 
     | 
||
| // NB timer records time taken on drop, so there are no | ||
| 
          
            
          
           | 
    ||
  
    
      This file contains hidden or 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
    
  
  
    
              
  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.
  
    
  
    
Uh oh!
There was an error while loading. Please reload this page.