-
Notifications
You must be signed in to change notification settings - Fork 1.8k
Account for memory usage in SortPreservingMerge (#5885) #6382
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
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change | ||||||||||
|---|---|---|---|---|---|---|---|---|---|---|---|---|
|
|
@@ -55,6 +55,9 @@ use tempfile::NamedTempFile; | |||||||||||
| use tokio::sync::mpsc::{Receiver, Sender}; | ||||||||||||
| use tokio::task; | ||||||||||||
|
|
||||||||||||
| /// How much memory to reserve for performing in-memory sorts | ||||||||||||
|
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.
Suggested change
|
||||||||||||
| const EXTERNAL_SORTER_MERGE_RESERVATION: usize = 10 * 1024 * 1024; | ||||||||||||
|
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. I'm not a massive fan of this, but this somewhat patches around the issue that once we initiate a merge we can't then 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. The problem with this approach is that even 10MB may not be enough to correctly merge the batches prior to spilling. So some queries that today would succeed (though exceed their memory limits) might fail. It seems to me better approaches (as follow on PRs) would be:
However, given the behavior on master today is to simply ignore the reservation and exceed the memory limit this behavior seems better than before. I suggest we merge this PR as is and file a follow on ticket for the improved behavior |
||||||||||||
|
|
||||||||||||
| struct ExternalSorterMetrics { | ||||||||||||
| /// metrics | ||||||||||||
| baseline: BaselineMetrics, | ||||||||||||
|
|
@@ -94,8 +97,10 @@ struct ExternalSorter { | |||||||||||
| expr: Arc<[PhysicalSortExpr]>, | ||||||||||||
| metrics: ExternalSorterMetrics, | ||||||||||||
| fetch: Option<usize>, | ||||||||||||
| /// Reservation for in_mem_batches | ||||||||||||
| reservation: MemoryReservation, | ||||||||||||
| partition_id: usize, | ||||||||||||
| /// Reservation for in memory sorting of 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.
Suggested change
|
||||||||||||
| merge_reservation: MemoryReservation, | ||||||||||||
| runtime: Arc<RuntimeEnv>, | ||||||||||||
| batch_size: usize, | ||||||||||||
| } | ||||||||||||
|
|
@@ -115,6 +120,12 @@ impl ExternalSorter { | |||||||||||
| .with_can_spill(true) | ||||||||||||
| .register(&runtime.memory_pool); | ||||||||||||
|
|
||||||||||||
| let mut merge_reservation = | ||||||||||||
| MemoryConsumer::new(format!("ExternalSorterMerge[{partition_id}]")) | ||||||||||||
| .register(&runtime.memory_pool); | ||||||||||||
|
|
||||||||||||
| merge_reservation.resize(EXTERNAL_SORTER_MERGE_RESERVATION); | ||||||||||||
|
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. I take it as a positive sign that this was required to make the spill tests pass, without this the merge would exceed the memory limit and fail |
||||||||||||
|
|
||||||||||||
| Self { | ||||||||||||
| schema, | ||||||||||||
| in_mem_batches: vec![], | ||||||||||||
|
|
@@ -124,7 +135,7 @@ impl ExternalSorter { | |||||||||||
| metrics, | ||||||||||||
| fetch, | ||||||||||||
| reservation, | ||||||||||||
| partition_id, | ||||||||||||
| merge_reservation, | ||||||||||||
| runtime, | ||||||||||||
| batch_size, | ||||||||||||
| } | ||||||||||||
|
|
@@ -189,12 +200,10 @@ impl ExternalSorter { | |||||||||||
| &self.expr, | ||||||||||||
| self.metrics.baseline.clone(), | ||||||||||||
| self.batch_size, | ||||||||||||
| self.reservation.split_empty(), | ||||||||||||
| ) | ||||||||||||
| } else if !self.in_mem_batches.is_empty() { | ||||||||||||
| let result = self.in_mem_sort_stream(self.metrics.baseline.clone()); | ||||||||||||
| // Report to the memory manager we are no longer using memory | ||||||||||||
| self.reservation.free(); | ||||||||||||
| result | ||||||||||||
| self.in_mem_sort_stream(self.metrics.baseline.clone()) | ||||||||||||
| } else { | ||||||||||||
| Ok(Box::pin(EmptyRecordBatchStream::new(self.schema.clone()))) | ||||||||||||
| } | ||||||||||||
|
|
@@ -238,6 +247,9 @@ impl ExternalSorter { | |||||||||||
| return Ok(()); | ||||||||||||
| } | ||||||||||||
|
|
||||||||||||
| // Release the memory reserved for merge | ||||||||||||
| self.merge_reservation.free(); | ||||||||||||
|
|
||||||||||||
| self.in_mem_batches = self | ||||||||||||
| .in_mem_sort_stream(self.metrics.baseline.intermediate())? | ||||||||||||
|
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. I double checked that |
||||||||||||
| .try_collect() | ||||||||||||
|
|
@@ -249,7 +261,11 @@ impl ExternalSorter { | |||||||||||
| .map(|x| x.get_array_memory_size()) | ||||||||||||
| .sum(); | ||||||||||||
|
|
||||||||||||
| self.reservation.resize(size); | ||||||||||||
| // Reserve headroom for next sort | ||||||||||||
| self.merge_reservation | ||||||||||||
| .resize(EXTERNAL_SORTER_MERGE_RESERVATION); | ||||||||||||
|
|
||||||||||||
| self.reservation.try_resize(size)?; | ||||||||||||
| self.in_mem_batches_sorted = true; | ||||||||||||
| Ok(()) | ||||||||||||
| } | ||||||||||||
|
|
@@ -262,9 +278,8 @@ impl ExternalSorter { | |||||||||||
| assert_ne!(self.in_mem_batches.len(), 0); | ||||||||||||
| if self.in_mem_batches.len() == 1 { | ||||||||||||
| let batch = self.in_mem_batches.remove(0); | ||||||||||||
| let stream = self.sort_batch_stream(batch, metrics)?; | ||||||||||||
| self.in_mem_batches.clear(); | ||||||||||||
| return Ok(stream); | ||||||||||||
| let reservation = self.reservation.take(); | ||||||||||||
| return self.sort_batch_stream(batch, metrics, reservation); | ||||||||||||
| } | ||||||||||||
|
|
||||||||||||
| // If less than 1MB of in-memory data, concatenate and sort in place | ||||||||||||
|
|
@@ -274,14 +289,19 @@ impl ExternalSorter { | |||||||||||
| // Concatenate memory batches together and sort | ||||||||||||
| let batch = concat_batches(&self.schema, &self.in_mem_batches)?; | ||||||||||||
| self.in_mem_batches.clear(); | ||||||||||||
| return self.sort_batch_stream(batch, metrics); | ||||||||||||
| self.reservation.try_resize(batch.get_array_memory_size())?; | ||||||||||||
| let reservation = self.reservation.take(); | ||||||||||||
| return self.sort_batch_stream(batch, metrics, reservation); | ||||||||||||
| } | ||||||||||||
|
|
||||||||||||
| let streams = std::mem::take(&mut self.in_mem_batches) | ||||||||||||
| .into_iter() | ||||||||||||
| .map(|batch| { | ||||||||||||
| let metrics = self.metrics.baseline.intermediate(); | ||||||||||||
| Ok(spawn_buffered(self.sort_batch_stream(batch, metrics)?, 1)) | ||||||||||||
| let reservation = | ||||||||||||
| self.reservation.split(batch.get_array_memory_size())?; | ||||||||||||
| let input = self.sort_batch_stream(batch, metrics, reservation)?; | ||||||||||||
| Ok(spawn_buffered(input, 1)) | ||||||||||||
| }) | ||||||||||||
| .collect::<Result<_>>()?; | ||||||||||||
|
|
||||||||||||
|
|
@@ -293,30 +313,25 @@ impl ExternalSorter { | |||||||||||
| &self.expr, | ||||||||||||
| metrics, | ||||||||||||
| self.batch_size, | ||||||||||||
| self.merge_reservation.split_empty(), | ||||||||||||
| ) | ||||||||||||
| } | ||||||||||||
|
|
||||||||||||
| fn sort_batch_stream( | ||||||||||||
| &self, | ||||||||||||
| batch: RecordBatch, | ||||||||||||
| metrics: BaselineMetrics, | ||||||||||||
| reservation: MemoryReservation, | ||||||||||||
| ) -> Result<SendableRecordBatchStream> { | ||||||||||||
| let schema = batch.schema(); | ||||||||||||
|
|
||||||||||||
| let mut reservation = | ||||||||||||
| MemoryConsumer::new(format!("sort_batch_stream{}", self.partition_id)) | ||||||||||||
| .register(&self.runtime.memory_pool); | ||||||||||||
|
|
||||||||||||
| // TODO: This should probably be try_grow (#5885) | ||||||||||||
| reservation.resize(batch.get_array_memory_size()); | ||||||||||||
|
|
||||||||||||
| let fetch = self.fetch; | ||||||||||||
| let expressions = self.expr.clone(); | ||||||||||||
| let stream = futures::stream::once(futures::future::lazy(move |_| { | ||||||||||||
| let sorted = sort_batch(&batch, &expressions, fetch)?; | ||||||||||||
| metrics.record_output(sorted.num_rows()); | ||||||||||||
| drop(batch); | ||||||||||||
| reservation.free(); | ||||||||||||
| drop(reservation); | ||||||||||||
| Ok(sorted) | ||||||||||||
| })); | ||||||||||||
| Ok(Box::pin(RecordBatchStreamAdapter::new(schema, stream))) | ||||||||||||
|
|
@@ -723,7 +738,8 @@ mod tests { | |||||||||||
| #[tokio::test] | ||||||||||||
| async fn test_sort_spill() -> Result<()> { | ||||||||||||
| // trigger spill there will be 4 batches with 5.5KB for each | ||||||||||||
| let config = RuntimeConfig::new().with_memory_limit(12288, 1.0); | ||||||||||||
| let config = RuntimeConfig::new() | ||||||||||||
| .with_memory_limit(EXTERNAL_SORTER_MERGE_RESERVATION + 12288, 1.0); | ||||||||||||
| let runtime = Arc::new(RuntimeEnv::new(config)?); | ||||||||||||
| let session_ctx = SessionContext::with_config_rt(SessionConfig::new(), runtime); | ||||||||||||
|
|
||||||||||||
|
|
@@ -805,8 +821,10 @@ mod tests { | |||||||||||
| ]; | ||||||||||||
|
|
||||||||||||
| for (fetch, expect_spillage) in test_options { | ||||||||||||
| let config = RuntimeConfig::new() | ||||||||||||
| .with_memory_limit(avg_batch_size * (partitions - 1), 1.0); | ||||||||||||
| let config = RuntimeConfig::new().with_memory_limit( | ||||||||||||
| EXTERNAL_SORTER_MERGE_RESERVATION + avg_batch_size * (partitions - 1), | ||||||||||||
| 1.0, | ||||||||||||
| ); | ||||||||||||
| let runtime = Arc::new(RuntimeEnv::new(config)?); | ||||||||||||
| let session_ctx = | ||||||||||||
| SessionContext::with_config_rt(SessionConfig::new(), runtime); | ||||||||||||
|
|
||||||||||||
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 it would help to note here in comments why the code needs to keep around a field that is never read (
dead_code). I think it is to keep the reservation around long enough?