-
Notifications
You must be signed in to change notification settings - Fork 1.8k
Top-K eager batch sorting #7180
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 9 commits
39d6d16
dcba28f
c4a40f2
ca0786f
5e110d5
7b1a4d2
24d8359
4ce8184
a22cc71
fff49ab
daa9337
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 |
|---|---|---|
|
|
@@ -35,6 +35,7 @@ use arrow::compute::{concat_batches, lexsort_to_indices, take}; | |
| use arrow::datatypes::SchemaRef; | ||
| use arrow::ipc::reader::FileReader; | ||
| use arrow::record_batch::RecordBatch; | ||
| use arrow_schema::ArrowError; | ||
| use datafusion_common::{plan_err, DataFusionError, Result}; | ||
| use datafusion_execution::memory_pool::{ | ||
| human_readable_size, MemoryConsumer, MemoryReservation, | ||
|
|
@@ -45,14 +46,14 @@ use datafusion_physical_expr::EquivalenceProperties; | |
| use futures::{StreamExt, TryStreamExt}; | ||
| use log::{debug, error, trace}; | ||
| use std::any::Any; | ||
| use std::ffi::OsString; | ||
| use std::fmt; | ||
| use std::fmt::{Debug, Formatter}; | ||
| use std::fs::File; | ||
| use std::io::BufReader; | ||
| use std::path::{Path, PathBuf}; | ||
| use std::sync::Arc; | ||
| use tempfile::NamedTempFile; | ||
| use tokio::sync::mpsc::Sender; | ||
| use tokio::task; | ||
|
|
||
| struct ExternalSorterMetrics { | ||
|
|
@@ -85,6 +86,10 @@ impl ExternalSorterMetrics { | |
| /// | ||
| /// 1. get a non-empty new batch from input | ||
| /// | ||
| /// 1.2. if a `fetch` parameter has been provided, and the batch size | ||
| /// is larger than `fetch`, sort the incoming batch in order to | ||
| /// reduce its size and thus use less memory. | ||
| /// | ||
| /// 2. check with the memory manager there is sufficient space to | ||
| /// buffer the batch in memory 2.1 if memory sufficient, buffer | ||
| /// batch in memory, go to 1. | ||
|
|
@@ -196,10 +201,9 @@ impl ExternalSorterMetrics { | |
| struct ExternalSorter { | ||
| /// schema of the output (and the input) | ||
| schema: SchemaRef, | ||
| /// Potentially unsorted in memory buffer | ||
| in_mem_batches: Vec<RecordBatch>, | ||
| /// if `Self::in_mem_batches` are sorted | ||
| in_mem_batches_sorted: bool, | ||
| /// A vector of tuples, with each tuple consisting of a flag | ||
| /// denoting whether the batch is sorted, and the batch itself | ||
| in_mem_batches: Vec<(bool, RecordBatch)>, | ||
| /// If data has previously been spilled, the locations of the | ||
| /// spill files (in Arrow IPC format) | ||
| spills: Vec<NamedTempFile>, | ||
|
|
@@ -238,7 +242,6 @@ impl ExternalSorter { | |
| Self { | ||
| schema, | ||
| in_mem_batches: vec![], | ||
| in_mem_batches_sorted: true, | ||
| spills: vec![], | ||
| expr: expr.into(), | ||
| metrics, | ||
|
|
@@ -253,11 +256,19 @@ impl ExternalSorter { | |
| /// Appends an unsorted [`RecordBatch`] to `in_mem_batches` | ||
| /// | ||
| /// Updates memory usage metrics, and possibly triggers spilling to disk | ||
| async fn insert_batch(&mut self, input: RecordBatch) -> Result<()> { | ||
| async fn insert_batch(&mut self, mut input: RecordBatch) -> Result<()> { | ||
| if input.num_rows() == 0 { | ||
| return Ok(()); | ||
| } | ||
|
|
||
| let mut batch_sorted = false; | ||
| if self.fetch.map_or(false, |f| f < input.num_rows()) { | ||
| // Eagerly sort the batch to potentially reduce the number of rows | ||
| // after applying the fetch parameter. | ||
| input = sort_batch(&input, &self.expr, self.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. I wonder whether you could recover part of the perf difference by concat + sorting only once every
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, that's worth trying to benchmark as well. It could also be something like every
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. Yes, definitely.
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. Fwiw, I did try out a variety of approaches along these lines, but still failed to eliminate the perf difference. There's probably a combination of parameters that is just right and could yield better results, but with the native Top-K operator in the works that likely won't be necessary. |
||
| batch_sorted = true; | ||
| } | ||
|
|
||
| let size = batch_byte_size(&input); | ||
| if self.reservation.try_grow(size).is_err() { | ||
| let before = self.reservation.size(); | ||
|
|
@@ -279,8 +290,7 @@ impl ExternalSorter { | |
| } | ||
| } | ||
|
|
||
| self.in_mem_batches.push(input); | ||
| self.in_mem_batches_sorted = false; | ||
| self.in_mem_batches.push((batch_sorted, input)); | ||
| Ok(()) | ||
| } | ||
|
|
||
|
|
@@ -307,8 +317,9 @@ impl ExternalSorter { | |
| } | ||
|
|
||
| for spill in self.spills.drain(..) { | ||
| let stream = read_spill_as_stream(spill, self.schema.clone())?; | ||
| streams.push(stream); | ||
| let spill_streams = | ||
| read_spill_as_streams(spill.path(), self.schema.clone())?; | ||
| streams.extend(spill_streams); | ||
| } | ||
|
|
||
| streaming_merge( | ||
|
|
@@ -345,7 +356,7 @@ impl ExternalSorter { | |
| } | ||
|
|
||
| /// Writes any `in_memory_batches` to a spill file and clears | ||
| /// the batches. The contents of the spil file are sorted. | ||
| /// the batches. The contents of the spill file are sorted. | ||
| /// | ||
| /// Returns the amount of memory freed. | ||
| async fn spill(&mut self) -> Result<usize> { | ||
|
|
@@ -359,7 +370,11 @@ impl ExternalSorter { | |
| self.in_mem_sort().await?; | ||
|
|
||
| let spillfile = self.runtime.disk_manager.create_tmp_file("Sorting")?; | ||
| let batches = std::mem::take(&mut self.in_mem_batches); | ||
|
|
||
| let (sorted, batches): (Vec<bool>, Vec<RecordBatch>) = | ||
| std::mem::take(&mut self.in_mem_batches).into_iter().unzip(); | ||
| assert!(sorted.iter().all(|&s| s)); | ||
|
|
||
| spill_sorted_batches(batches, spillfile.path(), self.schema.clone()).await?; | ||
| let used = self.reservation.free(); | ||
| self.metrics.spill_count.add(1); | ||
|
|
@@ -370,23 +385,27 @@ impl ExternalSorter { | |
|
|
||
| /// Sorts the in_mem_batches in place | ||
| async fn in_mem_sort(&mut self) -> Result<()> { | ||
| if self.in_mem_batches_sorted { | ||
| let batch_count = self.in_mem_batches.len(); | ||
| let all_batches_sorted = self.in_mem_batches.iter().all(|(sorted, _)| *sorted); | ||
| if batch_count == 0 || all_batches_sorted { | ||
| return Ok(()); | ||
| } | ||
|
|
||
| self.in_mem_batches = self | ||
| .in_mem_sort_stream(self.metrics.baseline.intermediate())? | ||
| .try_collect() | ||
| .await?; | ||
| .try_collect::<Vec<_>>() | ||
| .await? | ||
| .into_iter() | ||
| .map(|batch| (true, batch)) | ||
| .collect(); | ||
|
|
||
| let size: usize = self | ||
| .in_mem_batches | ||
| .iter() | ||
| .map(|x| x.get_array_memory_size()) | ||
| .map(|(_, x)| x.get_array_memory_size()) | ||
| .sum(); | ||
|
|
||
| self.reservation.resize(size); | ||
| self.in_mem_batches_sorted = true; | ||
| Ok(()) | ||
| } | ||
|
|
||
|
|
@@ -454,8 +473,8 @@ impl ExternalSorter { | |
| ) -> Result<SendableRecordBatchStream> { | ||
| 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)?; | ||
| let (sorted, batch) = self.in_mem_batches.remove(0); | ||
| let stream = self.sort_batch_stream(batch, sorted, metrics)?; | ||
| self.in_mem_batches.clear(); | ||
| return Ok(stream); | ||
| } | ||
|
|
@@ -465,16 +484,23 @@ impl ExternalSorter { | |
| // This is a very rough heuristic and likely could be refined further | ||
| if self.reservation.size() < 1048576 { | ||
| // Concatenate memory batches together and sort | ||
| let batch = concat_batches(&self.schema, &self.in_mem_batches)?; | ||
| let (_, batches): (Vec<bool>, Vec<RecordBatch>) = | ||
| std::mem::take(&mut self.in_mem_batches).into_iter().unzip(); | ||
| let batch = concat_batches(&self.schema, &batches)?; | ||
| self.in_mem_batches.clear(); | ||
| return self.sort_batch_stream(batch, metrics); | ||
| // Even if all individual batches were themselves sorted the resulting concatenated one | ||
| // isn't guaranteed to be sorted, so we must perform sorting on the stream. | ||
| return self.sort_batch_stream(batch, false, metrics); | ||
|
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. Another approach might be be to not use the
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. Good point; will try to benchmark that change too.
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. So I did try to test this approach as well, and then saw some improvements that seemed too good to be true. I went and re-ran the benchmarks again and the improvements held, until they didn't at some point 🤷🏻♂️ (fwiw I'm running the benchmarks on a cloud VM, not dedicated hardware). In hindsight, the sorting benchmarks actually do not use a memory limit and so there were no spills and this code path wasn't exercised. I did try running the benchmarks with memory limits on, but then I hit Either way, I'll add this check now even without doing benchmarking on it because it seems it can only help. |
||
| } | ||
|
|
||
| let streams = std::mem::take(&mut self.in_mem_batches) | ||
| .into_iter() | ||
| .map(|batch| { | ||
| .map(|(sorted, batch)| { | ||
| let metrics = self.metrics.baseline.intermediate(); | ||
| Ok(spawn_buffered(self.sort_batch_stream(batch, metrics)?, 1)) | ||
| Ok(spawn_buffered( | ||
| self.sort_batch_stream(batch, sorted, metrics)?, | ||
| 1, | ||
| )) | ||
| }) | ||
| .collect::<Result<_>>()?; | ||
|
|
||
|
|
@@ -492,27 +518,34 @@ impl ExternalSorter { | |
| fn sort_batch_stream( | ||
| &self, | ||
| batch: RecordBatch, | ||
| sorted: bool, | ||
| metrics: BaselineMetrics, | ||
| ) -> 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(); | ||
| Ok(sorted) | ||
| })); | ||
| Ok(Box::pin(RecordBatchStreamAdapter::new(schema, stream))) | ||
| if !sorted { | ||
| // Reserve some memory for sorting the batch | ||
| 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 output = sort_batch(&batch, &expressions, fetch)?; | ||
| metrics.record_output(output.num_rows()); | ||
| drop(batch); | ||
| reservation.free(); | ||
| Ok(output) | ||
| })); | ||
| Ok(Box::pin(RecordBatchStreamAdapter::new(schema, stream))) | ||
| } else { | ||
| let stream = futures::stream::once(futures::future::lazy(move |_| Ok(batch))); | ||
| Ok(Box::pin(RecordBatchStreamAdapter::new(schema, stream))) | ||
| } | ||
| } | ||
| } | ||
|
|
||
|
|
@@ -562,22 +595,6 @@ async fn spill_sorted_batches( | |
| } | ||
| } | ||
|
|
||
| fn read_spill_as_stream( | ||
| path: NamedTempFile, | ||
| schema: SchemaRef, | ||
| ) -> Result<SendableRecordBatchStream> { | ||
| let mut builder = RecordBatchReceiverStream::builder(schema, 2); | ||
| let sender = builder.tx(); | ||
|
|
||
| builder.spawn_blocking(move || { | ||
| if let Err(e) = read_spill(sender, path.path()) { | ||
| error!("Failure while reading spill file: {:?}. Error: {}", path, e); | ||
| } | ||
| }); | ||
|
|
||
| Ok(builder.build()) | ||
| } | ||
|
|
||
| fn write_sorted( | ||
| batches: Vec<RecordBatch>, | ||
| path: PathBuf, | ||
|
|
@@ -597,15 +614,47 @@ fn write_sorted( | |
| Ok(()) | ||
| } | ||
|
|
||
| fn read_spill(sender: Sender<Result<RecordBatch>>, path: &Path) -> Result<()> { | ||
| /// Stream batches from spill files. | ||
| /// | ||
| /// Each spill file has one or more batches. Intra-batch order is guaranteed (each one is sorted), | ||
| /// but the inter-batch ordering is not guaranteed, hence why we need to convert each batch from the | ||
| /// spill to a separate input stream for the merge-sort procedure. | ||
|
||
| fn read_spill_as_streams( | ||
| path: &Path, | ||
| schema: SchemaRef, | ||
| ) -> Result<Vec<SendableRecordBatchStream>> { | ||
| let file = BufReader::new(File::open(path)?); | ||
| let reader = FileReader::try_new(file, None)?; | ||
|
|
||
| let mut streams = vec![]; | ||
| let file_path = path.as_os_str().to_os_string(); | ||
| for batch in reader { | ||
| sender | ||
| .blocking_send(batch.map_err(Into::into)) | ||
| .map_err(|e| DataFusionError::Execution(format!("{e}")))?; | ||
| let stream = build_receiver_stream(batch, schema.clone(), file_path.clone()); | ||
| streams.push(stream); | ||
| } | ||
| Ok(()) | ||
| Ok(streams) | ||
| } | ||
|
|
||
| fn build_receiver_stream( | ||
| maybe_batch: Result<RecordBatch, ArrowError>, | ||
| schema: SchemaRef, | ||
| file_path: OsString, | ||
| ) -> SendableRecordBatchStream { | ||
| let mut builder = RecordBatchReceiverStream::builder(schema.clone(), 2); | ||
| let sender = builder.tx(); | ||
|
|
||
| builder.spawn_blocking(move || { | ||
| if let Err(e) = sender | ||
| .blocking_send(maybe_batch.map_err(Into::into)) | ||
| .map_err(|e| DataFusionError::Execution(format!("{e}"))) | ||
| { | ||
| error!( | ||
| "Failure while reading spill file: {:?}. Error: {}", | ||
| file_path, e | ||
| ); | ||
| } | ||
| }); | ||
| builder.build() | ||
| } | ||
|
|
||
| /// Sort execution plan. | ||
|
|
||
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 thinking can we make the heuristic
f < input.num_rows() / 10or something magic numbers to only do eager sort for small `K's?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 was also thinking something similar but along the lines of
f < input.num_rows() && f <= 100, so that we effectively have a hard cur-off for eager sorting after 100 rows.