-
Notifications
You must be signed in to change notification settings - Fork 1.9k
Use fetch limit in get_sorted_iter
#3545
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
Changes from all commits
c42bb7b
1b4a7d9
b5b6812
6157d32
f3bb742
1a1b8e5
6de4a21
c6bb97c
b30dd8b
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
This file was deleted.
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -38,7 +38,7 @@ use crate::physical_plan::{ | |
| RecordBatchStream, SendableRecordBatchStream, Statistics, | ||
| }; | ||
| use crate::prelude::SessionConfig; | ||
| use arrow::array::{make_array, Array, ArrayRef, MutableArrayData, UInt32Array}; | ||
| use arrow::array::{make_array, Array, ArrayRef, MutableArrayData}; | ||
| pub use arrow::compute::SortOptions; | ||
| use arrow::compute::{concat, lexsort_to_indices, take, SortColumn, TakeOptions}; | ||
| use arrow::datatypes::SchemaRef; | ||
|
|
@@ -152,6 +152,7 @@ impl ExternalSorter { | |
| &self.expr, | ||
| batch_size, | ||
| tracking_metrics, | ||
| self.fetch, | ||
| )?; | ||
| let prev_used = self.free_all_memory(); | ||
| streams.push(SortedStream::new(in_mem_stream, prev_used)); | ||
|
|
@@ -183,6 +184,7 @@ impl ExternalSorter { | |
| &self.expr, | ||
| batch_size, | ||
| tracking_metrics, | ||
| self.fetch, | ||
| ); | ||
| // Report to the memory manager we are no longer using memory | ||
| self.free_all_memory(); | ||
|
|
@@ -273,6 +275,7 @@ impl MemoryConsumer for ExternalSorter { | |
| &self.expr, | ||
| self.session_config.batch_size(), | ||
| tracking_metrics, | ||
| self.fetch, | ||
|
Contributor
Author
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. Nice thing is that it also reduces disk spilling, as sort + limit is done before writing.
Contributor
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. Although to be honest, I would hope that if there is a LIMIT on the query we could probably avoid the spilling entirely
Contributor
Author
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. Yeah maybe the spilling could see the remaining batch is so small it could add the sorted data to memory again - avoiding the spill 🤔
Contributor
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. As a future PR / optimization perhaps |
||
| ); | ||
|
|
||
| spill_partial_sorted_stream(&mut stream?, spillfile.path(), self.schema.clone()) | ||
|
|
@@ -289,13 +292,14 @@ impl MemoryConsumer for ExternalSorter { | |
| } | ||
| } | ||
|
|
||
| /// consume the non-empty `sorted_bathes` and do in_mem_sort | ||
| /// consume the non-empty `sorted_batches` and do in_mem_sort | ||
| fn in_mem_partial_sort( | ||
| buffered_batches: &mut Vec<BatchWithSortArray>, | ||
| schema: SchemaRef, | ||
| expressions: &[PhysicalSortExpr], | ||
| batch_size: usize, | ||
| tracking_metrics: MemTrackingMetrics, | ||
| fetch: Option<usize>, | ||
| ) -> Result<SendableRecordBatchStream> { | ||
| assert_ne!(buffered_batches.len(), 0); | ||
| if buffered_batches.len() == 1 { | ||
|
|
@@ -323,7 +327,7 @@ fn in_mem_partial_sort( | |
| // NB timer records time taken on drop, so there are no | ||
| // calls to `timer.done()` below. | ||
| let _timer = tracking_metrics.elapsed_compute().timer(); | ||
| get_sorted_iter(&sorted_arrays, expressions, batch_size)? | ||
| get_sorted_iter(&sorted_arrays, expressions, batch_size, fetch)? | ||
| }; | ||
| Ok(Box::pin(SortedSizedRecordBatchStream::new( | ||
| schema, | ||
|
|
@@ -345,6 +349,7 @@ fn get_sorted_iter( | |
| sort_arrays: &[Vec<ArrayRef>], | ||
| expr: &[PhysicalSortExpr], | ||
| batch_size: usize, | ||
| fetch: Option<usize>, | ||
| ) -> Result<SortedIterator> { | ||
| let row_indices = sort_arrays | ||
| .iter() | ||
|
|
@@ -374,44 +379,38 @@ fn get_sorted_iter( | |
| }) | ||
| }) | ||
| .collect::<Result<Vec<_>>>()?; | ||
| let indices = lexsort_to_indices(&sort_columns, None)?; | ||
| let indices = lexsort_to_indices(&sort_columns, fetch)?; | ||
|
Contributor
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. ✨ 👍 |
||
|
|
||
| Ok(SortedIterator::new(indices, row_indices, batch_size)) | ||
| // Calculate composite index based on sorted indices | ||
| let row_indices = indices | ||
| .values() | ||
| .iter() | ||
| .map(|i| row_indices[*i as usize]) | ||
| .collect(); | ||
|
|
||
| Ok(SortedIterator::new(row_indices, batch_size)) | ||
|
Contributor
Author
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. Some cleanup - we can do this immediately instead of keeping it in |
||
| } | ||
|
|
||
| struct SortedIterator { | ||
| /// Current logical position in the iterator | ||
| pos: usize, | ||
| /// Indexes into the input representing the correctly sorted total output | ||
| indices: UInt32Array, | ||
| /// Map each each logical input index to where it can be found in the sorted input batches | ||
| /// Sorted composite index of where to find the rows in buffered batches | ||
|
Contributor
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. 👍 |
||
| composite: Vec<CompositeIndex>, | ||
| /// Maximum batch size to produce | ||
| batch_size: usize, | ||
| /// total length of the iterator | ||
| length: usize, | ||
| } | ||
|
|
||
| impl SortedIterator { | ||
| fn new( | ||
| indices: UInt32Array, | ||
| composite: Vec<CompositeIndex>, | ||
| batch_size: usize, | ||
| ) -> Self { | ||
| let length = composite.len(); | ||
| fn new(composite: Vec<CompositeIndex>, batch_size: usize) -> Self { | ||
| Self { | ||
| pos: 0, | ||
| indices, | ||
| composite, | ||
| batch_size, | ||
| length, | ||
| } | ||
| } | ||
|
|
||
| fn memory_size(&self) -> usize { | ||
| std::mem::size_of_val(self) | ||
| + self.indices.get_array_memory_size() | ||
| + std::mem::size_of_val(&self.composite[..]) | ||
| std::mem::size_of_val(self) + std::mem::size_of_val(&self.composite[..]) | ||
| } | ||
| } | ||
|
|
||
|
|
@@ -420,33 +419,25 @@ impl Iterator for SortedIterator { | |
|
|
||
| /// Emit a max of `batch_size` positions each time | ||
| fn next(&mut self) -> Option<Self::Item> { | ||
| if self.pos >= self.length { | ||
| let length = self.composite.len(); | ||
| if self.pos >= length { | ||
| return None; | ||
| } | ||
|
|
||
| let current_size = min(self.batch_size, self.length - self.pos); | ||
| let current_size = min(self.batch_size, length - self.pos); | ||
|
|
||
| // Combine adjacent indexes from the same batch to make a slice, | ||
| // for more efficient `extend` later. | ||
| let mut last_batch_idx = 0; | ||
| let mut indices_in_batch = vec![]; | ||
| let mut last_batch_idx = self.composite[self.pos].batch_idx; | ||
| let mut indices_in_batch = Vec::with_capacity(current_size); | ||
|
|
||
| let mut slices = vec![]; | ||
| for i in 0..current_size { | ||
| let p = self.pos + i; | ||
| let c_index = self.indices.value(p) as usize; | ||
| let ci = self.composite[c_index]; | ||
|
|
||
| if indices_in_batch.is_empty() { | ||
| last_batch_idx = ci.batch_idx; | ||
| indices_in_batch.push(ci.row_idx); | ||
| } else if ci.batch_idx == last_batch_idx { | ||
| indices_in_batch.push(ci.row_idx); | ||
| } else { | ||
| for ci in &self.composite[self.pos..self.pos + current_size] { | ||
| if ci.batch_idx != last_batch_idx { | ||
| group_indices(last_batch_idx, &mut indices_in_batch, &mut slices); | ||
| last_batch_idx = ci.batch_idx; | ||
| indices_in_batch.push(ci.row_idx); | ||
| } | ||
| indices_in_batch.push(ci.row_idx); | ||
| } | ||
|
|
||
| assert!( | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.