From 5eaaeeca0a0b1bb99a2e08562676ca7457c3876b Mon Sep 17 00:00:00 2001 From: wiedld Date: Tue, 15 Aug 2023 09:02:53 -0700 Subject: [PATCH 01/24] refactor(7181): move the management of input data, and output data, to the same structure. During the later stages of Cascade merge, we will no longer be sorting based on each streaming batch (one cursor at a time). Instead will be referencing a previous sort_order per [batch_idx][row_idx] when merging previous steps in the cascade. Therefore, in order to keep the same set of Cursors we are moving the input and output structures more closely together. Later optimizations may be able to decouple these again. --- .../core/src/physical_plan/sorts/builder.rs | 105 ++++++++++++++++-- .../core/src/physical_plan/sorts/merge.rs | 56 +++------- 2 files changed, 114 insertions(+), 47 deletions(-) diff --git a/datafusion/core/src/physical_plan/sorts/builder.rs b/datafusion/core/src/physical_plan/sorts/builder.rs index 3527d5738223..f79e9c1f0150 100644 --- a/datafusion/core/src/physical_plan/sorts/builder.rs +++ b/datafusion/core/src/physical_plan/sorts/builder.rs @@ -21,35 +21,43 @@ use arrow::record_batch::RecordBatch; use datafusion_common::Result; use datafusion_execution::memory_pool::MemoryReservation; -#[derive(Debug, Copy, Clone, Default)] -struct BatchCursor { +use super::cursor::Cursor; + +#[derive(Debug, Default)] +struct BatchCursor { /// The index into BatchBuilder::batches batch_idx: usize, /// The row index within the given batch row_idx: usize, + /// The cursor for the given batch. If None, the batch is finished. + cursor: Option, } /// Provides an API to incrementally build a [`RecordBatch`] from partitioned [`RecordBatch`] #[derive(Debug)] -pub struct BatchBuilder { +pub struct BatchBuilder { /// The schema of the RecordBatches yielded by this stream schema: SchemaRef, /// Maintain a list of [`RecordBatch`] and their corresponding stream batches: Vec<(usize, RecordBatch)>, + /// Maintain a list of cursors for each finished (sorted) batch + /// The number of total batches can be larger than the number of total streams + batch_cursors: Vec>, + /// Accounts for memory used by buffered batches reservation: MemoryReservation, /// The current [`BatchCursor`] for each stream - cursors: Vec, + cursors: Vec>, /// The accumulated stream indexes from which to pull rows /// Consists of a tuple of `(batch_idx, row_idx)` indices: Vec<(usize, usize)>, } -impl BatchBuilder { +impl BatchBuilder { /// Create a new [`BatchBuilder`] with the provided `stream_count` and `batch_size` pub fn new( schema: SchemaRef, @@ -60,20 +68,34 @@ impl BatchBuilder { Self { schema, batches: Vec::with_capacity(stream_count * 2), - cursors: vec![BatchCursor::default(); stream_count], + batch_cursors: Vec::with_capacity(stream_count * 2), + cursors: (0..stream_count) + .map(|_| BatchCursor { + batch_idx: 0, + row_idx: 0, + cursor: None, + }) + .collect(), indices: Vec::with_capacity(batch_size), reservation, } } /// Append a new batch in `stream_idx` - pub fn push_batch(&mut self, stream_idx: usize, batch: RecordBatch) -> Result<()> { + pub fn push_batch( + &mut self, + stream_idx: usize, + batch: RecordBatch, + cursor: C, + ) -> Result<()> { self.reservation.try_grow(batch.get_array_memory_size())?; let batch_idx = self.batches.len(); self.batches.push((stream_idx, batch)); + self.batch_cursors.push(None); // placehold until cursor is finished self.cursors[stream_idx] = BatchCursor { batch_idx, row_idx: 0, + cursor: Some(cursor), }; Ok(()) } @@ -101,6 +123,75 @@ impl BatchBuilder { &self.schema } + /// For a finished cursor, remove from BatchCursor (per stream_idx), and track in batch_cursors (per batch_idx) + fn cursor_finished(&mut self, stream_idx: usize) { + let batch_idx = self.cursors[stream_idx].batch_idx; + let row_idx = self.cursors[stream_idx].row_idx; + match std::mem::replace( + &mut self.cursors[stream_idx], + BatchCursor { + batch_idx, + row_idx, + cursor: None, + }, + ) + .cursor + { + Some(prev_batch_cursor) => { + self.batch_cursors[batch_idx] = Some(prev_batch_cursor) + } + None => unreachable!("previous cursor should not be None"), + } + } + + /// Advance the cursor for `stream_idx` + /// Returns `true` if the cursor was advanced + pub fn advance(&mut self, stream_idx: usize) -> bool { + match &mut self.cursors[stream_idx].cursor { + Some(c) => { + c.advance(); + if c.is_finished() { + self.cursor_finished(stream_idx); + } + true + } + None => false, + } + } + + /// Returns true if there is an in-progress cursor for a given stream + pub fn cursor_in_progress(&self, stream_idx: usize) -> bool { + self.cursors[stream_idx] + .cursor + .as_ref() + .map(|cursor| !cursor.is_finished()) + .unwrap_or(false) + } + + /// Returns `true` if the cursor at index `a` is greater than at index `b` + #[inline] + pub fn is_gt(&self, stream_idx_a: usize, stream_idx_b: usize) -> bool { + match ( + self.cursor_in_progress(stream_idx_a), + self.cursor_in_progress(stream_idx_b), + ) { + (false, _) => true, + (_, false) => false, + _ => { + match ( + &self.cursors[stream_idx_a].cursor, + &self.cursors[stream_idx_b].cursor, + ) { + (Some(a), Some(b)) => a + .cmp(&b) + .then_with(|| stream_idx_a.cmp(&stream_idx_b)) + .is_gt(), + _ => unreachable!(), + } + } + } + } + /// Drains the in_progress row indexes, and builds a new RecordBatch from them /// /// Will then drop any batches for which all rows have been yielded to the output diff --git a/datafusion/core/src/physical_plan/sorts/merge.rs b/datafusion/core/src/physical_plan/sorts/merge.rs index f8a1457dd62a..711aa07ef4e9 100644 --- a/datafusion/core/src/physical_plan/sorts/merge.rs +++ b/datafusion/core/src/physical_plan/sorts/merge.rs @@ -102,8 +102,8 @@ pub fn streaming_merge( type CursorStream = Box>>; #[derive(Debug)] -struct SortPreservingMergeStream { - in_progress: BatchBuilder, +struct SortPreservingMergeStream { + in_progress: BatchBuilder, /// The sorted input streams to merge together streams: CursorStream, @@ -155,9 +155,6 @@ struct SortPreservingMergeStream { /// target batch size batch_size: usize, - /// Vector that holds cursors for each non-exhausted input partition - cursors: Vec>, - /// Optional number of rows to fetch fetch: Option, @@ -181,7 +178,6 @@ impl SortPreservingMergeStream { streams, metrics, aborted: false, - cursors: (0..stream_count).map(|_| None).collect(), loser_tree: vec![], loser_tree_adjusted: false, batch_size, @@ -198,7 +194,7 @@ impl SortPreservingMergeStream { cx: &mut Context<'_>, idx: usize, ) -> Poll> { - if self.cursors[idx].is_some() { + if self.in_progress.cursor_in_progress(idx) { // Cursor is not finished - don't need a new RecordBatch yet return Poll::Ready(Ok(())); } @@ -207,8 +203,7 @@ impl SortPreservingMergeStream { None => Poll::Ready(Ok(())), Some(Err(e)) => Poll::Ready(Err(e)), Some(Ok((cursor, batch))) => { - self.cursors[idx] = Some(cursor); - Poll::Ready(self.in_progress.push_batch(idx, batch)) + Poll::Ready(self.in_progress.push_batch(idx, batch, cursor)) } } } @@ -246,11 +241,12 @@ impl SortPreservingMergeStream { self.aborted = true; return Poll::Ready(Some(Err(e))); } + self.update_loser_tree(); } let stream_idx = self.loser_tree[0]; - if self.advance(stream_idx) { + if self.in_progress.advance(stream_idx) { self.loser_tree_adjusted = false; self.in_progress.push_row(stream_idx); @@ -263,7 +259,6 @@ impl SortPreservingMergeStream { } self.produced += self.in_progress.len(); - return Poll::Ready(self.in_progress.build_record_batch().transpose()); } } @@ -274,30 +269,6 @@ impl SortPreservingMergeStream { .unwrap_or(false) } - fn advance(&mut self, stream_idx: usize) -> bool { - let slot = &mut self.cursors[stream_idx]; - match slot.as_mut() { - Some(c) => { - c.advance(); - if c.is_finished() { - *slot = None; - } - true - } - None => false, - } - } - - /// Returns `true` if the cursor at index `a` is greater than at index `b` - #[inline] - fn is_gt(&self, a: usize, b: usize) -> bool { - match (&self.cursors[a], &self.cursors[b]) { - (None, _) => true, - (_, None) => false, - (Some(ac), Some(bc)) => ac.cmp(bc).then_with(|| a.cmp(&b)).is_gt(), - } - } - /// Find the leaf node index in the loser tree for the given cursor index /// /// Note that this is not necessarily a leaf node in the tree, but it can @@ -328,7 +299,7 @@ impl SortPreservingMergeStream { /// #[inline] fn lt_leaf_node_index(&self, cursor_index: usize) -> usize { - (self.cursors.len() + cursor_index) / 2 + (self.num_leaf_nodes() + cursor_index) / 2 } /// Find the parent node index for the given node index @@ -337,17 +308,22 @@ impl SortPreservingMergeStream { node_idx / 2 } + #[inline] + fn num_leaf_nodes(&self) -> usize { + self.streams.partitions() + } + /// Attempts to initialize the loser tree with one value from each /// non exhausted input, if possible fn init_loser_tree(&mut self) { // Init loser tree - self.loser_tree = vec![usize::MAX; self.cursors.len()]; - for i in 0..self.cursors.len() { + self.loser_tree = vec![usize::MAX; self.num_leaf_nodes()]; + for i in 0..self.num_leaf_nodes() { let mut winner = i; let mut cmp_node = self.lt_leaf_node_index(i); while cmp_node != 0 && self.loser_tree[cmp_node] != usize::MAX { let challenger = self.loser_tree[cmp_node]; - if self.is_gt(winner, challenger) { + if self.in_progress.is_gt(winner, challenger) { self.loser_tree[cmp_node] = winner; winner = challenger; } @@ -366,7 +342,7 @@ impl SortPreservingMergeStream { let mut cmp_node = self.lt_leaf_node_index(winner); while cmp_node != 0 { let challenger = self.loser_tree[cmp_node]; - if self.is_gt(winner, challenger) { + if self.in_progress.is_gt(winner, challenger) { self.loser_tree[cmp_node] = winner; winner = challenger; } From a3870d09a2cbc4e1f44796d28ab5a99de89107bf Mon Sep 17 00:00:00 2001 From: wiedld Date: Tue, 15 Aug 2023 12:27:31 -0700 Subject: [PATCH 02/24] feat(7181): add cursor.seek() --- .../core/src/physical_plan/sorts/cursor.rs | 49 +++++++++++++++++++ 1 file changed, 49 insertions(+) diff --git a/datafusion/core/src/physical_plan/sorts/cursor.rs b/datafusion/core/src/physical_plan/sorts/cursor.rs index c0c791288644..726101d58013 100644 --- a/datafusion/core/src/physical_plan/sorts/cursor.rs +++ b/datafusion/core/src/physical_plan/sorts/cursor.rs @@ -99,6 +99,9 @@ pub trait Cursor: Ord { /// Advance the cursor, returning the previous row index fn advance(&mut self) -> usize; + + /// Go to reference row, returning the previous row index + fn seek(&mut self, row: usize) -> usize; } impl Cursor for RowCursor { @@ -113,6 +116,13 @@ impl Cursor for RowCursor { self.cur_row += 1; t } + + #[inline] + fn seek(&mut self, goto: usize) -> usize { + let previous = self.cur_row; + self.cur_row = goto; + previous + } } /// An [`Array`] that can be converted into [`FieldValues`] @@ -275,6 +285,12 @@ impl Cursor for FieldCursor { self.offset += 1; t } + + fn seek(&mut self, goto: usize) -> usize { + let previous = self.offset; + self.offset = goto; + previous + } } #[cfg(test)] @@ -432,4 +448,37 @@ mod tests { b.advance(); assert_eq!(a.cmp(&b), Ordering::Less); } + + #[test] + fn test_primitive_goto() { + let options = SortOptions { + descending: false, + nulls_first: true, + }; + + let buffer = ScalarBuffer::from(vec![1]); + let cmp = new_primitive(options, buffer, 0); + + let buffer = ScalarBuffer::from(vec![0, 1, 2]); + let mut cursor = new_primitive(options, buffer, 0); + + // comparisons as advance + assert_eq!(cursor.cmp(&cmp), Ordering::Less); + cursor.advance(); + assert_eq!(cursor.cmp(&cmp), Ordering::Equal); + cursor.advance(); + assert_eq!(cursor.cmp(&cmp), Ordering::Greater); + + // goto + cursor.seek(1); + assert_eq!(cursor.cmp(&cmp), Ordering::Equal); + cursor.seek(0); + assert_eq!(cursor.cmp(&cmp), Ordering::Less); + cursor.seek(2); + assert_eq!(cursor.cmp(&cmp), Ordering::Greater); + + // goto returns previous row_idx + let prev = cursor.seek(0); + assert_eq!(prev, 2); + } } From e54e92cece20c0438ceeba942a262186b6fdcd94 Mon Sep 17 00:00:00 2001 From: wiedld Date: Tue, 15 Aug 2023 12:43:48 -0700 Subject: [PATCH 03/24] refactor(7181): move streaming_merge() into separate mod from the merge mod. Merge mod has the SortPreservingMergeStream, containing the loser tree. This SortPreservingMergeStream struct will be used repeatedly as part of the cascading merge; in turn, the cascading merge will be implemented for the streaming_merge() method. --- .../core/src/physical_plan/sorts/merge.rs | 80 +--------------- .../core/src/physical_plan/sorts/mod.rs | 5 +- .../core/src/physical_plan/sorts/sort.rs | 2 +- .../physical_plan/sorts/streaming_merge.rs | 92 +++++++++++++++++++ 4 files changed, 101 insertions(+), 78 deletions(-) create mode 100644 datafusion/core/src/physical_plan/sorts/streaming_merge.rs diff --git a/datafusion/core/src/physical_plan/sorts/merge.rs b/datafusion/core/src/physical_plan/sorts/merge.rs index 711aa07ef4e9..ab501052a20f 100644 --- a/datafusion/core/src/physical_plan/sorts/merge.rs +++ b/datafusion/core/src/physical_plan/sorts/merge.rs @@ -15,94 +15,24 @@ // specific language governing permissions and limitations // under the License. -//! Merge that deals with an arbitrary size of streaming inputs. -//! This is an order-preserving merge. - use crate::physical_plan::metrics::BaselineMetrics; use crate::physical_plan::sorts::builder::BatchBuilder; use crate::physical_plan::sorts::cursor::Cursor; -use crate::physical_plan::sorts::stream::{ - FieldCursorStream, PartitionedStream, RowCursorStream, -}; -use crate::physical_plan::{ - PhysicalSortExpr, RecordBatchStream, SendableRecordBatchStream, -}; -use arrow::datatypes::{DataType, SchemaRef}; +use crate::physical_plan::sorts::stream::PartitionedStream; +use crate::physical_plan::RecordBatchStream; +use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; -use arrow_array::*; use datafusion_common::Result; use datafusion_execution::memory_pool::MemoryReservation; use futures::Stream; use std::pin::Pin; use std::task::{ready, Context, Poll}; -macro_rules! primitive_merge_helper { - ($t:ty, $($v:ident),+) => { - merge_helper!(PrimitiveArray<$t>, $($v),+) - }; -} - -macro_rules! merge_helper { - ($t:ty, $sort:ident, $streams:ident, $schema:ident, $tracking_metrics:ident, $batch_size:ident, $fetch:ident, $reservation:ident) => {{ - let streams = FieldCursorStream::<$t>::new($sort, $streams); - return Ok(Box::pin(SortPreservingMergeStream::new( - Box::new(streams), - $schema, - $tracking_metrics, - $batch_size, - $fetch, - $reservation, - ))); - }}; -} - -/// Perform a streaming merge of [`SendableRecordBatchStream`] based on provided sort expressions -/// while preserving order. -pub fn streaming_merge( - streams: Vec, - schema: SchemaRef, - expressions: &[PhysicalSortExpr], - metrics: BaselineMetrics, - batch_size: usize, - fetch: Option, - reservation: MemoryReservation, -) -> Result { - // Special case single column comparisons with optimized cursor implementations - if expressions.len() == 1 { - let sort = expressions[0].clone(); - let data_type = sort.expr.data_type(schema.as_ref())?; - downcast_primitive! { - data_type => (primitive_merge_helper, sort, streams, schema, metrics, batch_size, fetch, reservation), - DataType::Utf8 => merge_helper!(StringArray, sort, streams, schema, metrics, batch_size, fetch, reservation) - DataType::LargeUtf8 => merge_helper!(LargeStringArray, sort, streams, schema, metrics, batch_size, fetch, reservation) - DataType::Binary => merge_helper!(BinaryArray, sort, streams, schema, metrics, batch_size, fetch, reservation) - DataType::LargeBinary => merge_helper!(LargeBinaryArray, sort, streams, schema, metrics, batch_size, fetch, reservation) - _ => {} - } - } - - let streams = RowCursorStream::try_new( - schema.as_ref(), - expressions, - streams, - reservation.new_empty(), - )?; - - Ok(Box::pin(SortPreservingMergeStream::new( - Box::new(streams), - schema, - metrics, - batch_size, - fetch, - reservation, - ))) -} - /// A fallible [`PartitionedStream`] of [`Cursor`] and [`RecordBatch`] type CursorStream = Box>>; #[derive(Debug)] -struct SortPreservingMergeStream { +pub(crate) struct SortPreservingMergeStream { in_progress: BatchBuilder, /// The sorted input streams to merge together @@ -163,7 +93,7 @@ struct SortPreservingMergeStream { } impl SortPreservingMergeStream { - fn new( + pub(crate) fn new( streams: CursorStream, schema: SchemaRef, metrics: BaselineMetrics, diff --git a/datafusion/core/src/physical_plan/sorts/mod.rs b/datafusion/core/src/physical_plan/sorts/mod.rs index dff39db423f0..8a1184d3c2b5 100644 --- a/datafusion/core/src/physical_plan/sorts/mod.rs +++ b/datafusion/core/src/physical_plan/sorts/mod.rs @@ -20,10 +20,11 @@ mod builder; mod cursor; mod index; -pub mod merge; +mod merge; pub mod sort; pub mod sort_preserving_merge; mod stream; +pub mod streaming_merge; pub use index::RowIndex; -pub(crate) use merge::streaming_merge; +pub(crate) use streaming_merge::streaming_merge; diff --git a/datafusion/core/src/physical_plan/sorts/sort.rs b/datafusion/core/src/physical_plan/sorts/sort.rs index 5d23e72fe7cf..d3cd6c36c606 100644 --- a/datafusion/core/src/physical_plan/sorts/sort.rs +++ b/datafusion/core/src/physical_plan/sorts/sort.rs @@ -24,7 +24,7 @@ use crate::physical_plan::expressions::PhysicalSortExpr; use crate::physical_plan::metrics::{ BaselineMetrics, Count, ExecutionPlanMetricsSet, MetricBuilder, MetricsSet, }; -use crate::physical_plan::sorts::merge::streaming_merge; +use crate::physical_plan::sorts::streaming_merge::streaming_merge; use crate::physical_plan::stream::{RecordBatchReceiverStream, RecordBatchStreamAdapter}; use crate::physical_plan::{ DisplayAs, DisplayFormatType, Distribution, EmptyRecordBatchStream, ExecutionPlan, diff --git a/datafusion/core/src/physical_plan/sorts/streaming_merge.rs b/datafusion/core/src/physical_plan/sorts/streaming_merge.rs new file mode 100644 index 000000000000..625699a3174b --- /dev/null +++ b/datafusion/core/src/physical_plan/sorts/streaming_merge.rs @@ -0,0 +1,92 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +//! Merge that deals with an arbitrary size of streaming inputs. +//! This is an order-preserving merge. + +use crate::physical_plan::metrics::BaselineMetrics; +use crate::physical_plan::sorts::{ + merge::SortPreservingMergeStream, + stream::{FieldCursorStream, RowCursorStream}, +}; +use crate::physical_plan::{PhysicalSortExpr, SendableRecordBatchStream}; +use arrow::datatypes::{DataType, SchemaRef}; +use arrow_array::*; +use datafusion_common::Result; +use datafusion_execution::memory_pool::MemoryReservation; + +macro_rules! primitive_merge_helper { + ($t:ty, $($v:ident),+) => { + merge_helper!(PrimitiveArray<$t>, $($v),+) + }; +} + +macro_rules! merge_helper { + ($t:ty, $sort:ident, $streams:ident, $schema:ident, $tracking_metrics:ident, $batch_size:ident, $fetch:ident, $reservation:ident) => {{ + let streams = FieldCursorStream::<$t>::new($sort, $streams); + return Ok(Box::pin(SortPreservingMergeStream::new( + Box::new(streams), + $schema, + $tracking_metrics, + $batch_size, + $fetch, + $reservation, + ))); + }}; +} + +/// Perform a streaming merge of [`SendableRecordBatchStream`] based on provided sort expressions +/// while preserving order. +pub fn streaming_merge( + streams: Vec, + schema: SchemaRef, + expressions: &[PhysicalSortExpr], + metrics: BaselineMetrics, + batch_size: usize, + fetch: Option, + reservation: MemoryReservation, +) -> Result { + // Special case single column comparisons with optimized cursor implementations + if expressions.len() == 1 { + let sort = expressions[0].clone(); + let data_type = sort.expr.data_type(schema.as_ref())?; + downcast_primitive! { + data_type => (primitive_merge_helper, sort, streams, schema, metrics, batch_size, fetch, reservation), + DataType::Utf8 => merge_helper!(StringArray, sort, streams, schema, metrics, batch_size, fetch, reservation) + DataType::LargeUtf8 => merge_helper!(LargeStringArray, sort, streams, schema, metrics, batch_size, fetch, reservation) + DataType::Binary => merge_helper!(BinaryArray, sort, streams, schema, metrics, batch_size, fetch, reservation) + DataType::LargeBinary => merge_helper!(LargeBinaryArray, sort, streams, schema, metrics, batch_size, fetch, reservation) + _ => {} + } + } + + let streams = RowCursorStream::try_new( + schema.as_ref(), + expressions, + streams, + reservation.new_empty(), + )?; + + Ok(Box::pin(SortPreservingMergeStream::new( + Box::new(streams), + schema, + metrics, + batch_size, + fetch, + reservation, + ))) +} From 3d43e97b50d10d7d5a080f25684e5239ccb1b856 Mon Sep 17 00:00:00 2001 From: wiedld Date: Tue, 15 Aug 2023 14:12:52 -0700 Subject: [PATCH 04/24] feat(7181): streaming_merge() consumes SortPreservingCascadeStream SortPreservingCascadeStream currrently has a single root node of SortPreservingMergeStream. TODO: build out tree of SortPreservingMergeStream. --- .../core/src/physical_plan/sorts/builder.rs | 5 - .../core/src/physical_plan/sorts/cascade.rs | 97 +++++++++++++++++++ .../core/src/physical_plan/sorts/merge.rs | 13 +-- .../core/src/physical_plan/sorts/mod.rs | 1 + .../physical_plan/sorts/streaming_merge.rs | 6 +- 5 files changed, 104 insertions(+), 18 deletions(-) create mode 100644 datafusion/core/src/physical_plan/sorts/cascade.rs diff --git a/datafusion/core/src/physical_plan/sorts/builder.rs b/datafusion/core/src/physical_plan/sorts/builder.rs index f79e9c1f0150..243bd897947c 100644 --- a/datafusion/core/src/physical_plan/sorts/builder.rs +++ b/datafusion/core/src/physical_plan/sorts/builder.rs @@ -118,11 +118,6 @@ impl BatchBuilder { self.indices.is_empty() } - /// Returns the schema of this [`BatchBuilder`] - pub fn schema(&self) -> &SchemaRef { - &self.schema - } - /// For a finished cursor, remove from BatchCursor (per stream_idx), and track in batch_cursors (per batch_idx) fn cursor_finished(&mut self, stream_idx: usize) { let batch_idx = self.cursors[stream_idx].batch_idx; diff --git a/datafusion/core/src/physical_plan/sorts/cascade.rs b/datafusion/core/src/physical_plan/sorts/cascade.rs new file mode 100644 index 000000000000..a07659281e15 --- /dev/null +++ b/datafusion/core/src/physical_plan/sorts/cascade.rs @@ -0,0 +1,97 @@ +use crate::physical_plan::metrics::BaselineMetrics; +use crate::physical_plan::sorts::cursor::Cursor; +use crate::physical_plan::sorts::merge::{CursorStream, SortPreservingMergeStream}; +use crate::physical_plan::RecordBatchStream; + +use arrow::datatypes::SchemaRef; +use arrow::record_batch::RecordBatch; +use datafusion_common::Result; +use datafusion_execution::memory_pool::MemoryReservation; +use futures::Stream; +use std::pin::Pin; +use std::sync::Arc; +use std::task::{Context, Poll}; + +// TODO: +// this will be changed to dyn PartitionedStream, Vec, Vec)>> +// and the final interleave will be done here, after root node returns +type MergeStream = Pin>>>; + +pub(crate) struct SortPreservingCascadeStream { + /// If the stream has encountered an error + aborted: bool, + + /// used to record execution metrics + metrics: BaselineMetrics, + + /// The cascading stream + cascade: MergeStream, + + /// The schema of the RecordBatches yielded by this stream + schema: SchemaRef, +} + +impl SortPreservingCascadeStream { + pub(crate) fn new( + streams: CursorStream, + schema: SchemaRef, + metrics: BaselineMetrics, + batch_size: usize, + fetch: Option, + reservation: MemoryReservation, + ) -> Self { + let root: MergeStream = Box::pin(SortPreservingMergeStream::new( + streams, + Arc::clone(&schema), + metrics.clone(), + batch_size, + fetch, + reservation, + )); + + // TODO (followup commit): build the cascade tree here + let cascade = root; + Self { + aborted: false, + cascade, + schema, + metrics, + } + } + + fn poll_next_inner( + &mut self, + cx: &mut Context<'_>, + ) -> Poll>> { + if self.aborted { + return Poll::Ready(None); + } + + match futures::ready!(self.cascade.as_mut().poll_next(cx)) { + None => Poll::Ready(None), + Some(Err(e)) => { + self.aborted = true; + Poll::Ready(Some(Err(e))) + } + Some(Ok(inner_result)) => Poll::Ready(Some(Ok(inner_result))), + } + } +} + +impl Stream for SortPreservingCascadeStream { + type Item = Result; + + fn poll_next( + mut self: Pin<&mut Self>, + cx: &mut Context<'_>, + ) -> Poll> { + let poll = self.poll_next_inner(cx); + self.metrics.record_poll(poll) + } +} + +impl RecordBatchStream for SortPreservingCascadeStream { + fn schema(&self) -> SchemaRef { + self.schema.clone() + } +} diff --git a/datafusion/core/src/physical_plan/sorts/merge.rs b/datafusion/core/src/physical_plan/sorts/merge.rs index ab501052a20f..e8a75a8ed37e 100644 --- a/datafusion/core/src/physical_plan/sorts/merge.rs +++ b/datafusion/core/src/physical_plan/sorts/merge.rs @@ -19,7 +19,6 @@ use crate::physical_plan::metrics::BaselineMetrics; use crate::physical_plan::sorts::builder::BatchBuilder; use crate::physical_plan::sorts::cursor::Cursor; use crate::physical_plan::sorts::stream::PartitionedStream; -use crate::physical_plan::RecordBatchStream; use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; use datafusion_common::Result; @@ -29,7 +28,8 @@ use std::pin::Pin; use std::task::{ready, Context, Poll}; /// A fallible [`PartitionedStream`] of [`Cursor`] and [`RecordBatch`] -type CursorStream = Box>>; +pub(crate) type CursorStream = + Box>>; #[derive(Debug)] pub(crate) struct SortPreservingMergeStream { @@ -290,13 +290,6 @@ impl Stream for SortPreservingMergeStream { mut self: Pin<&mut Self>, cx: &mut Context<'_>, ) -> Poll> { - let poll = self.poll_next_inner(cx); - self.metrics.record_poll(poll) - } -} - -impl RecordBatchStream for SortPreservingMergeStream { - fn schema(&self) -> SchemaRef { - self.in_progress.schema().clone() + self.poll_next_inner(cx) } } diff --git a/datafusion/core/src/physical_plan/sorts/mod.rs b/datafusion/core/src/physical_plan/sorts/mod.rs index 8a1184d3c2b5..a6b18b3c673f 100644 --- a/datafusion/core/src/physical_plan/sorts/mod.rs +++ b/datafusion/core/src/physical_plan/sorts/mod.rs @@ -18,6 +18,7 @@ //! Sort functionalities mod builder; +mod cascade; mod cursor; mod index; mod merge; diff --git a/datafusion/core/src/physical_plan/sorts/streaming_merge.rs b/datafusion/core/src/physical_plan/sorts/streaming_merge.rs index 625699a3174b..e059c4191de2 100644 --- a/datafusion/core/src/physical_plan/sorts/streaming_merge.rs +++ b/datafusion/core/src/physical_plan/sorts/streaming_merge.rs @@ -20,7 +20,7 @@ use crate::physical_plan::metrics::BaselineMetrics; use crate::physical_plan::sorts::{ - merge::SortPreservingMergeStream, + cascade::SortPreservingCascadeStream, stream::{FieldCursorStream, RowCursorStream}, }; use crate::physical_plan::{PhysicalSortExpr, SendableRecordBatchStream}; @@ -38,7 +38,7 @@ macro_rules! primitive_merge_helper { macro_rules! merge_helper { ($t:ty, $sort:ident, $streams:ident, $schema:ident, $tracking_metrics:ident, $batch_size:ident, $fetch:ident, $reservation:ident) => {{ let streams = FieldCursorStream::<$t>::new($sort, $streams); - return Ok(Box::pin(SortPreservingMergeStream::new( + return Ok(Box::pin(SortPreservingCascadeStream::new( Box::new(streams), $schema, $tracking_metrics, @@ -81,7 +81,7 @@ pub fn streaming_merge( reservation.new_empty(), )?; - Ok(Box::pin(SortPreservingMergeStream::new( + Ok(Box::pin(SortPreservingCascadeStream::new( Box::new(streams), schema, metrics, From 1a6a364d5d9efb322473096b04b6838acdedfacf Mon Sep 17 00:00:00 2001 From: wiedld Date: Tue, 15 Aug 2023 16:28:02 -0700 Subject: [PATCH 05/24] feat(7181): change BatchBuilder to be a SortOrder Builder, with the SortPreservingCascadeStream doing the final interleave(). This commit knowingly fails for tests which are utilizing multiple polls to return all record batches. Specifically: * dataframe::tests::with_column_renamed_join * physical_plan::sorts::sort_preserving_merge::tests::test_partition_sort_streaming_input_output TODO: splicing the RecordBatch and Cursor per merge yield. --- .../core/src/physical_plan/sorts/builder.rs | 87 ++++++++----------- .../core/src/physical_plan/sorts/cascade.rs | 55 +++++++++--- .../core/src/physical_plan/sorts/merge.rs | 14 ++- 3 files changed, 81 insertions(+), 75 deletions(-) diff --git a/datafusion/core/src/physical_plan/sorts/builder.rs b/datafusion/core/src/physical_plan/sorts/builder.rs index 243bd897947c..7c015fffff77 100644 --- a/datafusion/core/src/physical_plan/sorts/builder.rs +++ b/datafusion/core/src/physical_plan/sorts/builder.rs @@ -15,17 +15,17 @@ // specific language governing permissions and limitations // under the License. -use arrow::compute::interleave; -use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; use datafusion_common::Result; use datafusion_execution::memory_pool::MemoryReservation; use super::cursor::Cursor; +pub type SortOrder = (usize, usize); // batch_idx, row_idx + #[derive(Debug, Default)] struct BatchCursor { - /// The index into BatchBuilder::batches + /// The index into SortOrderBuilder::batches batch_idx: usize, /// The row index within the given batch row_idx: usize, @@ -35,10 +35,7 @@ struct BatchCursor { /// Provides an API to incrementally build a [`RecordBatch`] from partitioned [`RecordBatch`] #[derive(Debug)] -pub struct BatchBuilder { - /// The schema of the RecordBatches yielded by this stream - schema: SchemaRef, - +pub struct SortOrderBuilder { /// Maintain a list of [`RecordBatch`] and their corresponding stream batches: Vec<(usize, RecordBatch)>, @@ -53,20 +50,17 @@ pub struct BatchBuilder { cursors: Vec>, /// The accumulated stream indexes from which to pull rows - /// Consists of a tuple of `(batch_idx, row_idx)` - indices: Vec<(usize, usize)>, + indices: Vec, } -impl BatchBuilder { - /// Create a new [`BatchBuilder`] with the provided `stream_count` and `batch_size` +impl SortOrderBuilder { + /// Create a new [`SortOrderBuilder`] with the provided `stream_count` and `batch_size` pub fn new( - schema: SchemaRef, stream_count: usize, batch_size: usize, reservation: MemoryReservation, ) -> Self { Self { - schema, batches: Vec::with_capacity(stream_count * 2), batch_cursors: Vec::with_capacity(stream_count * 2), cursors: (0..stream_count) @@ -108,12 +102,12 @@ impl BatchBuilder { self.indices.push((cursor.batch_idx, row_idx)); } - /// Returns the number of in-progress rows in this [`BatchBuilder`] + /// Returns the number of in-progress rows in this [`SortOrderBuilder`] pub fn len(&self) -> usize { self.indices.len() } - /// Returns `true` if this [`BatchBuilder`] contains no in-progress rows + /// Returns `true` if this [`SortOrderBuilder`] contains no in-progress rows pub fn is_empty(&self) -> bool { self.indices.is_empty() } @@ -187,50 +181,37 @@ impl BatchBuilder { } } - /// Drains the in_progress row indexes, and builds a new RecordBatch from them + /// Takes the batches which already are sorted, and returns them with the corresponding cursors and sort order /// - /// Will then drop any batches for which all rows have been yielded to the output - /// - /// Returns `None` if no pending rows - pub fn build_record_batch(&mut self) -> Result> { + /// This will drain the internal state of the builder, and return `None` if there are no pendin + pub fn build_sort_order( + &mut self, + ) -> Result, Vec, Vec)>> { if self.is_empty() { return Ok(None); } - let columns = (0..self.schema.fields.len()) - .map(|column_idx| { - let arrays: Vec<_> = self - .batches - .iter() - .map(|(_, batch)| batch.column(column_idx).as_ref()) - .collect(); - Ok(interleave(&arrays, &self.indices)?) + let batches = std::mem::take(&mut self.batches); + let batch_cursors = std::mem::take(&mut self.batch_cursors); + let sort_order = std::mem::take(&mut self.indices); + + // reset the cursors per stream_idx (but keep the same capacity) + // these should already be empty when the caller calls this method, but just in case + self.cursors = (0..self.cursors.len()) + .map(|_| BatchCursor { + batch_idx: 0, + row_idx: 0, + cursor: None, }) - .collect::>>()?; - - self.indices.clear(); - - // New cursors are only created once the previous cursor for the stream - // is finished. This means all remaining rows from all but the last batch - // for each stream have been yielded to the newly created record batch - // - // We can therefore drop all but the last batch for each stream - let mut batch_idx = 0; - let mut retained = 0; - self.batches.retain(|(stream_idx, batch)| { - let stream_cursor = &mut self.cursors[*stream_idx]; - let retain = stream_cursor.batch_idx == batch_idx; - batch_idx += 1; - - if retain { - stream_cursor.batch_idx = retained; - retained += 1; - } else { - self.reservation.shrink(batch.get_array_memory_size()); - } - retain - }); + .collect(); - Ok(Some(RecordBatch::try_new(self.schema.clone(), columns)?)) + Ok(Some(( + batches.into_iter().map(|(_, batch)| batch).collect(), + batch_cursors + .into_iter() + .map(|cursor| cursor.expect("batch cursor should be Some")) + .collect(), + sort_order, + ))) } } diff --git a/datafusion/core/src/physical_plan/sorts/cascade.rs b/datafusion/core/src/physical_plan/sorts/cascade.rs index a07659281e15..61a010a22bdc 100644 --- a/datafusion/core/src/physical_plan/sorts/cascade.rs +++ b/datafusion/core/src/physical_plan/sorts/cascade.rs @@ -1,23 +1,23 @@ use crate::physical_plan::metrics::BaselineMetrics; +use crate::physical_plan::sorts::builder::SortOrder; use crate::physical_plan::sorts::cursor::Cursor; use crate::physical_plan::sorts::merge::{CursorStream, SortPreservingMergeStream}; use crate::physical_plan::RecordBatchStream; +use arrow::compute::interleave; use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; use datafusion_common::Result; use datafusion_execution::memory_pool::MemoryReservation; use futures::Stream; use std::pin::Pin; -use std::sync::Arc; use std::task::{Context, Poll}; -// TODO: -// this will be changed to dyn PartitionedStream, Vec, Vec)>> -// and the final interleave will be done here, after root node returns -type MergeStream = Pin>>>; +type MergeStream = Pin< + Box, Vec, Vec)>>>, +>; -pub(crate) struct SortPreservingCascadeStream { +pub(crate) struct SortPreservingCascadeStream { /// If the stream has encountered an error aborted: bool, @@ -25,14 +25,14 @@ pub(crate) struct SortPreservingCascadeStream { metrics: BaselineMetrics, /// The cascading stream - cascade: MergeStream, + cascade: MergeStream, /// The schema of the RecordBatches yielded by this stream schema: SchemaRef, } -impl SortPreservingCascadeStream { - pub(crate) fn new( +impl SortPreservingCascadeStream { + pub(crate) fn new( streams: CursorStream, schema: SchemaRef, metrics: BaselineMetrics, @@ -40,9 +40,8 @@ impl SortPreservingCascadeStream { fetch: Option, reservation: MemoryReservation, ) -> Self { - let root: MergeStream = Box::pin(SortPreservingMergeStream::new( + let root: MergeStream = Box::pin(SortPreservingMergeStream::new( streams, - Arc::clone(&schema), metrics.clone(), batch_size, fetch, @@ -59,6 +58,24 @@ impl SortPreservingCascadeStream { } } + fn build_record_batch( + &mut self, + batches: Vec, + sort_order: Vec, + ) -> Result { + let columns = (0..self.schema.fields.len()) + .map(|column_idx| { + let arrays: Vec<_> = batches + .iter() + .map(|batch| batch.column(column_idx).as_ref()) + .collect(); + Ok(interleave(&arrays, sort_order.as_slice())?) + }) + .collect::>>()?; + + Ok(RecordBatch::try_new(self.schema.clone(), columns)?) + } + fn poll_next_inner( &mut self, cx: &mut Context<'_>, @@ -73,12 +90,20 @@ impl SortPreservingCascadeStream { self.aborted = true; Poll::Ready(Some(Err(e))) } - Some(Ok(inner_result)) => Poll::Ready(Some(Ok(inner_result))), + Some(Ok((batches, _, sort_order))) => { + match self.build_record_batch(batches, sort_order) { + Ok(batch) => Poll::Ready(Some(Ok(batch))), + Err(e) => { + self.aborted = true; + Poll::Ready(Some(Err(e))) + } + } + } } } } -impl Stream for SortPreservingCascadeStream { +impl Stream for SortPreservingCascadeStream { type Item = Result; fn poll_next( @@ -90,7 +115,9 @@ impl Stream for SortPreservingCascadeStream { } } -impl RecordBatchStream for SortPreservingCascadeStream { +impl RecordBatchStream + for SortPreservingCascadeStream +{ fn schema(&self) -> SchemaRef { self.schema.clone() } diff --git a/datafusion/core/src/physical_plan/sorts/merge.rs b/datafusion/core/src/physical_plan/sorts/merge.rs index e8a75a8ed37e..e3218d529c1a 100644 --- a/datafusion/core/src/physical_plan/sorts/merge.rs +++ b/datafusion/core/src/physical_plan/sorts/merge.rs @@ -16,10 +16,9 @@ // under the License. use crate::physical_plan::metrics::BaselineMetrics; -use crate::physical_plan::sorts::builder::BatchBuilder; +use crate::physical_plan::sorts::builder::{SortOrder, SortOrderBuilder}; use crate::physical_plan::sorts::cursor::Cursor; use crate::physical_plan::sorts::stream::PartitionedStream; -use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; use datafusion_common::Result; use datafusion_execution::memory_pool::MemoryReservation; @@ -33,7 +32,7 @@ pub(crate) type CursorStream = #[derive(Debug)] pub(crate) struct SortPreservingMergeStream { - in_progress: BatchBuilder, + in_progress: SortOrderBuilder, /// The sorted input streams to merge together streams: CursorStream, @@ -95,7 +94,6 @@ pub(crate) struct SortPreservingMergeStream { impl SortPreservingMergeStream { pub(crate) fn new( streams: CursorStream, - schema: SchemaRef, metrics: BaselineMetrics, batch_size: usize, fetch: Option, @@ -104,7 +102,7 @@ impl SortPreservingMergeStream { let stream_count = streams.partitions(); Self { - in_progress: BatchBuilder::new(schema, stream_count, batch_size, reservation), + in_progress: SortOrderBuilder::new(stream_count, batch_size, reservation), streams, metrics, aborted: false, @@ -141,7 +139,7 @@ impl SortPreservingMergeStream { fn poll_next_inner( &mut self, cx: &mut Context<'_>, - ) -> Poll>> { + ) -> Poll, Vec, Vec)>>> { if self.aborted { return Poll::Ready(None); } @@ -189,7 +187,7 @@ impl SortPreservingMergeStream { } self.produced += self.in_progress.len(); - return Poll::Ready(self.in_progress.build_record_batch().transpose()); + return Poll::Ready(self.in_progress.build_sort_order().transpose()); } } @@ -284,7 +282,7 @@ impl SortPreservingMergeStream { } impl Stream for SortPreservingMergeStream { - type Item = Result; + type Item = Result<(Vec, Vec, Vec)>; fn poll_next( mut self: Pin<&mut Self>, From 28454c5f2678c59841ee379131e6f56101fd222e Mon Sep 17 00:00:00 2001 From: wiedld Date: Wed, 16 Aug 2023 19:14:53 -0700 Subject: [PATCH 06/24] feat(7181): add slice() to Cursor trait --- .../core/src/physical_plan/sorts/cursor.rs | 52 +++++++++++++++++++ 1 file changed, 52 insertions(+) diff --git a/datafusion/core/src/physical_plan/sorts/cursor.rs b/datafusion/core/src/physical_plan/sorts/cursor.rs index 726101d58013..b4048b2601a1 100644 --- a/datafusion/core/src/physical_plan/sorts/cursor.rs +++ b/datafusion/core/src/physical_plan/sorts/cursor.rs @@ -21,6 +21,7 @@ use arrow::datatypes::ArrowNativeTypeOp; use arrow::row::{Row, Rows}; use arrow_array::types::ByteArrayType; use arrow_array::{Array, ArrowPrimitiveType, GenericByteArray, PrimitiveArray}; +use datafusion_common::{DataFusionError, Result}; use datafusion_execution::memory_pool::MemoryReservation; use std::cmp::Ordering; @@ -102,6 +103,13 @@ pub trait Cursor: Ord { /// Go to reference row, returning the previous row index fn seek(&mut self, row: usize) -> usize; + + /// Slice the cursor at a given row index, returning a new cursor + /// + /// Returns an error if the slice is out of bounds, or memory is insufficient + fn slice(&self, offset: usize, length: usize) -> Result + where + Self: Sized; } impl Cursor for RowCursor { @@ -123,6 +131,14 @@ impl Cursor for RowCursor { self.cur_row = goto; previous } + + #[inline] + fn slice(&self, offset: usize, length: usize) -> Result { + let rows = self.rows.slice(offset, length); + let mut reservation = self.reservation.new_empty(); + reservation.try_grow(rows.size())?; + Ok(Self::new(rows, reservation)) + } } /// An [`Array`] that can be converted into [`FieldValues`] @@ -141,6 +157,10 @@ pub trait FieldValues { fn compare(a: &Self::Value, b: &Self::Value) -> Ordering; fn value(&self, idx: usize) -> &Self::Value; + + fn slice(&self, offset: usize, length: usize) -> Result + where + Self: Sized; } impl FieldArray for PrimitiveArray { @@ -170,6 +190,14 @@ impl FieldValues for PrimitiveValues { fn value(&self, idx: usize) -> &Self::Value { &self.0[idx] } + + #[inline] + fn slice(&self, offset: usize, length: usize) -> Result { + if offset + length > self.len() { + return Err(DataFusionError::Internal("slice out of bounds".into())); + } + Ok(Self(self.0.slice(offset, length))) + } } impl FieldArray for GenericByteArray { @@ -201,6 +229,14 @@ impl FieldValues for GenericByteArray { fn value(&self, idx: usize) -> &Self::Value { self.value(idx) } + + #[inline] + fn slice(&self, offset: usize, length: usize) -> Result { + if offset + length > Array::len(self) { + return Err(DataFusionError::Internal("slice out of bounds".into())); + } + Ok(self.slice(offset, length)) + } } /// A cursor over sorted, nullable [`FieldValues`] @@ -291,6 +327,22 @@ impl Cursor for FieldCursor { self.offset = goto; previous } + + fn slice(&self, offset: usize, length: usize) -> Result { + let FieldCursor { + values, + offset: _, + null_threshold, + options, + } = self; + + Ok(Self { + values: values.slice(offset, length)?, + offset: 0, + null_threshold: *null_threshold, + options: options.clone(), + }) + } } #[cfg(test)] From b766712efea59e015f4fe1f3efe0499a749a786d Mon Sep 17 00:00:00 2001 From: wiedld Date: Wed, 16 Aug 2023 19:43:22 -0700 Subject: [PATCH 07/24] feat(7181): make SortOrderBuilder yield in a stream. This requires slicing the batches and cursors, when yielded in parts. These two tests, reliant on multiple-polls of streaming data, now pass: * dataframe::tests::with_column_renamed_join * physical_plan::sorts::sort_preserving_merge::tests::test_partition_sort_streaming_input_output --- .../core/src/physical_plan/sorts/builder.rs | 120 +++++++++++++++--- .../core/src/physical_plan/sorts/merge.rs | 2 +- 2 files changed, 101 insertions(+), 21 deletions(-) diff --git a/datafusion/core/src/physical_plan/sorts/builder.rs b/datafusion/core/src/physical_plan/sorts/builder.rs index 7c015fffff77..148a99e9cdf4 100644 --- a/datafusion/core/src/physical_plan/sorts/builder.rs +++ b/datafusion/core/src/physical_plan/sorts/builder.rs @@ -18,12 +18,14 @@ use arrow::record_batch::RecordBatch; use datafusion_common::Result; use datafusion_execution::memory_pool::MemoryReservation; +use std::collections::HashMap; +use std::mem::{replace, take}; use super::cursor::Cursor; pub type SortOrder = (usize, usize); // batch_idx, row_idx -#[derive(Debug, Default)] +#[derive(Debug)] struct BatchCursor { /// The index into SortOrderBuilder::batches batch_idx: usize, @@ -116,7 +118,7 @@ impl SortOrderBuilder { fn cursor_finished(&mut self, stream_idx: usize) { let batch_idx = self.cursors[stream_idx].batch_idx; let row_idx = self.cursors[stream_idx].row_idx; - match std::mem::replace( + match replace( &mut self.cursors[stream_idx], BatchCursor { batch_idx, @@ -183,35 +185,113 @@ impl SortOrderBuilder { /// Takes the batches which already are sorted, and returns them with the corresponding cursors and sort order /// - /// This will drain the internal state of the builder, and return `None` if there are no pendin - pub fn build_sort_order( + /// This will drain the internal state of the builder, and return `None` if there are no pending + pub fn yield_sort_order( &mut self, ) -> Result, Vec, Vec)>> { if self.is_empty() { return Ok(None); } - let batches = std::mem::take(&mut self.batches); - let batch_cursors = std::mem::take(&mut self.batch_cursors); - let sort_order = std::mem::take(&mut self.indices); + let sort_order = take(&mut self.indices); - // reset the cursors per stream_idx (but keep the same capacity) - // these should already be empty when the caller calls this method, but just in case - self.cursors = (0..self.cursors.len()) - .map(|_| BatchCursor { - batch_idx: 0, - row_idx: 0, - cursor: None, - }) - .collect(); + let batch_rows_to_yield = + sort_order + .iter() + .fold(HashMap::new(), |mut acc, (batch_idx, row_idx)| { + acc.insert(batch_idx, row_idx + 1); + acc + }); + + let mut batches_to_retain = Vec::new(); + let mut batch_cursors_to_retain = Vec::new(); // per batch_idx + let mut retained_batch_idx: usize = 0; + + let mut batches_to_yield = Vec::new(); + let mut cursors_to_yield = Vec::new(); // per yielded batch + let mut yielded_batch_idx: usize = 0; + + // since we don't yield all batches, but the sort_order was built with the batch_idx from self.batches + // we need to update the batch_idx mapping in sort_order. + let mut batch_idx_for_sort_order = HashMap::new(); + + for batch_idx in 0..self.batches.len() { + let (stream_idx, batch) = &self.batches[batch_idx]; + let nothing_to_yield = !batch_rows_to_yield.contains_key(&batch_idx); + let is_fully_yielded = + !nothing_to_yield && batch_rows_to_yield[&batch_idx] == batch.num_rows(); + // to_split means that the batch has been partially yielded, and we need to split the batch and cursor + let to_split = !is_fully_yielded && !nothing_to_yield; + + if is_fully_yielded { + batches_to_yield.push(batch.to_owned()); + cursors_to_yield.push( + self.batch_cursors[batch_idx] + .take() + .expect("cursor should be Some"), + ); + + batch_idx_for_sort_order.insert(batch_idx, yielded_batch_idx); + yielded_batch_idx += 1; + } else if to_split { + let rows_to_yield = batch_rows_to_yield[&batch_idx]; + let split_row_idx = rows_to_yield - 1; + + // split batch + batches_to_retain.push(( + *stream_idx, + batch.slice(split_row_idx + 1, batch.num_rows() - rows_to_yield), + )); + batches_to_yield.push(batch.slice(0, rows_to_yield)); + + // split cursor + let cursor = match self.cursor_in_progress(*stream_idx) { + true => self.cursors[*stream_idx].cursor.take().expect("cursor should be Some"), + false => unreachable!("cursor should be in progress, since the batch is partially yielded"), + }; + let cursor_to_retain = + cursor.slice(split_row_idx + 1, batch.num_rows() - rows_to_yield)?; + self.cursors[*stream_idx] = BatchCursor { + batch_idx: retained_batch_idx, + row_idx: 0, + cursor: Some(cursor_to_retain), + }; + batch_cursors_to_retain.push(None); // placehold until cursor is finished + + cursors_to_yield.push(cursor.slice(0, rows_to_yield)?); + + // Immediately free memory associated with previous cursor + // This does not impact field cursors, which uses zero-copy buffer mem slices + // but it does impact row cursors which use memcopy. + drop(cursor); + + batch_idx_for_sort_order.insert(batch_idx, yielded_batch_idx); + yielded_batch_idx += 1; + retained_batch_idx += 1; + } else if nothing_to_yield { + batches_to_retain.push((*stream_idx, batch.to_owned())); + batch_cursors_to_retain.push(None); // placehold until cursor is finished + self.cursors[*stream_idx].batch_idx = retained_batch_idx; + + retained_batch_idx += 1; + } else { + unreachable!( + "should be fully yielded, partially yielded, or not yielded at all" + ); + } + } + self.batches = batches_to_retain; + self.batch_cursors = batch_cursors_to_retain; Ok(Some(( - batches.into_iter().map(|(_, batch)| batch).collect(), - batch_cursors + batches_to_yield, + cursors_to_yield, + sort_order .into_iter() - .map(|cursor| cursor.expect("batch cursor should be Some")) + .map(|(batch_idx, row_idx)| { + (batch_idx_for_sort_order[&batch_idx], row_idx) + }) .collect(), - sort_order, ))) } } diff --git a/datafusion/core/src/physical_plan/sorts/merge.rs b/datafusion/core/src/physical_plan/sorts/merge.rs index e3218d529c1a..f8d41c5533b7 100644 --- a/datafusion/core/src/physical_plan/sorts/merge.rs +++ b/datafusion/core/src/physical_plan/sorts/merge.rs @@ -187,7 +187,7 @@ impl SortPreservingMergeStream { } self.produced += self.in_progress.len(); - return Poll::Ready(self.in_progress.build_sort_order().transpose()); + return Poll::Ready(self.in_progress.yield_sort_order().transpose()); } } From 9ff37f315e2013b3832766aeb05ac849c6275773 Mon Sep 17 00:00:00 2001 From: wiedld Date: Tue, 22 Aug 2023 11:26:49 -0700 Subject: [PATCH 08/24] feat(7181): enable the ability to create a multi-layer merge cascade. * OffsetCursorStream enables the same RowCursorStream (with the same RowConverter) to be used across multiple leaf nodes. * Each tree node is a merge (a.k.a. SortPreservingMergeStream). * YieldedCursorStream enables the output from the previous merge, to be provided as input to the next merge. --- .../core/src/physical_plan/sorts/cascade.rs | 50 ++++- .../core/src/physical_plan/sorts/merge.rs | 6 +- .../core/src/physical_plan/sorts/stream.rs | 190 +++++++++++++++++- 3 files changed, 231 insertions(+), 15 deletions(-) diff --git a/datafusion/core/src/physical_plan/sorts/cascade.rs b/datafusion/core/src/physical_plan/sorts/cascade.rs index 61a010a22bdc..ccbf8dc8d137 100644 --- a/datafusion/core/src/physical_plan/sorts/cascade.rs +++ b/datafusion/core/src/physical_plan/sorts/cascade.rs @@ -1,7 +1,10 @@ use crate::physical_plan::metrics::BaselineMetrics; use crate::physical_plan::sorts::builder::SortOrder; use crate::physical_plan::sorts::cursor::Cursor; -use crate::physical_plan::sorts::merge::{CursorStream, SortPreservingMergeStream}; +use crate::physical_plan::sorts::merge::SortPreservingMergeStream; +use crate::physical_plan::sorts::stream::{ + CursorStream, MergeStream, OffsetCursorStream, YieldedCursorStream, +}; use crate::physical_plan::RecordBatchStream; use arrow::compute::interleave; @@ -11,14 +14,11 @@ use datafusion_common::Result; use datafusion_execution::memory_pool::MemoryReservation; use futures::Stream; use std::pin::Pin; +use std::sync::Arc; use std::task::{Context, Poll}; -type MergeStream = Pin< - Box, Vec, Vec)>>>, ->; - pub(crate) struct SortPreservingCascadeStream { - /// If the stream has encountered an error + /// If the stream has encountered an error, or fetch is reached aborted: bool, /// used to record execution metrics @@ -40,15 +40,49 @@ impl SortPreservingCascadeStream { fetch: Option, reservation: MemoryReservation, ) -> Self { + let stream_count = streams.partitions(); + + // TODO: since we already use a mutex here, + // we can have (for the same relative cost) a mutex for a single holder of record_batches + // which yields a batch_idx tracker. + // In this way, we can do slicing followed by concating of Cursors yielded from each merge, + // without needing to also yield sliced-then-concated batches (which are expensive to concat). + // + // Refer to YieldedCursorStream for where the concat would happen (TODO). + let streams = Arc::new(parking_lot::Mutex::new(streams)); + + let max_streams_per_merge = 2; // TODO: change this to 10, once we have tested with 2 (to force more leaf nodes) + let mut divided_streams: Vec> = + Vec::with_capacity(stream_count / max_streams_per_merge + 1); + + for stream_offset in (0..stream_count).step_by(max_streams_per_merge) { + let limit = + std::cmp::min(stream_offset + max_streams_per_merge, stream_count); + + // OffsetCursorStream enables the ability to share the same RowCursorStream across multiple leafnode merges. + let streams = + OffsetCursorStream::new(Arc::clone(&streams), stream_offset, limit); + + divided_streams.push(Box::pin(SortPreservingMergeStream::new( + Box::new(streams), + metrics.clone(), + batch_size, + None, // fetch, the LIMIT, is applied to the final merge + reservation.new_empty(), + ))); + } + + let next_level: CursorStream = + Box::new(YieldedCursorStream::new(divided_streams)); + let root: MergeStream = Box::pin(SortPreservingMergeStream::new( - streams, + next_level, metrics.clone(), batch_size, fetch, reservation, )); - // TODO (followup commit): build the cascade tree here let cascade = root; Self { aborted: false, diff --git a/datafusion/core/src/physical_plan/sorts/merge.rs b/datafusion/core/src/physical_plan/sorts/merge.rs index f8d41c5533b7..a602c8c49ff6 100644 --- a/datafusion/core/src/physical_plan/sorts/merge.rs +++ b/datafusion/core/src/physical_plan/sorts/merge.rs @@ -18,7 +18,7 @@ use crate::physical_plan::metrics::BaselineMetrics; use crate::physical_plan::sorts::builder::{SortOrder, SortOrderBuilder}; use crate::physical_plan::sorts::cursor::Cursor; -use crate::physical_plan::sorts::stream::PartitionedStream; +use crate::physical_plan::sorts::stream::CursorStream; use arrow::record_batch::RecordBatch; use datafusion_common::Result; use datafusion_execution::memory_pool::MemoryReservation; @@ -26,10 +26,6 @@ use futures::Stream; use std::pin::Pin; use std::task::{ready, Context, Poll}; -/// A fallible [`PartitionedStream`] of [`Cursor`] and [`RecordBatch`] -pub(crate) type CursorStream = - Box>>; - #[derive(Debug)] pub(crate) struct SortPreservingMergeStream { in_progress: SortOrderBuilder, diff --git a/datafusion/core/src/physical_plan/sorts/stream.rs b/datafusion/core/src/physical_plan/sorts/stream.rs index 9ef13b7eb25e..5dfd6e09645b 100644 --- a/datafusion/core/src/physical_plan/sorts/stream.rs +++ b/datafusion/core/src/physical_plan/sorts/stream.rs @@ -15,20 +15,35 @@ // specific language governing permissions and limitations // under the License. -use crate::physical_plan::sorts::cursor::{FieldArray, FieldCursor, RowCursor}; +use crate::physical_plan::sorts::builder::SortOrder; +use crate::physical_plan::sorts::cursor::{Cursor, FieldArray, FieldCursor, RowCursor}; use crate::physical_plan::SendableRecordBatchStream; use crate::physical_plan::{PhysicalExpr, PhysicalSortExpr}; use arrow::array::Array; use arrow::datatypes::Schema; use arrow::record_batch::RecordBatch; use arrow::row::{RowConverter, SortField}; -use datafusion_common::Result; +use datafusion_common::{DataFusionError, Result}; use datafusion_execution::memory_pool::MemoryReservation; use futures::stream::{Fuse, StreamExt}; +use parking_lot::Mutex; +use std::collections::VecDeque; use std::marker::PhantomData; use std::sync::Arc; use std::task::{ready, Context, Poll}; +/// A fallible [`PartitionedStream`] of [`Cursor`] and [`RecordBatch`] +pub(crate) type CursorStream = + Box>>; + +/// A stream of yielded [`SortOrder`]s, with the corresponding [`Cursor`]s and [`RecordBatch`]es +pub(crate) type MergeStream = std::pin::Pin< + Box< + dyn Send + + futures::Stream, Vec, Vec)>>, + >, +>; + /// A [`Stream`](futures::Stream) that has multiple partitions that can /// be polled separately but not concurrently /// @@ -207,3 +222,174 @@ impl PartitionedStream for FieldCursorStream { })) } } + +/// A wrapper around [`CursorStream`] +/// that provides polling of a subset of the streams. +pub struct OffsetCursorStream { + streams: Arc>>, + offset: usize, + limit: usize, +} + +impl OffsetCursorStream { + pub fn new( + streams: Arc>>, + offset: usize, + limit: usize, + ) -> Self { + Self { + streams, + offset, + limit, + } + } +} + +impl PartitionedStream for OffsetCursorStream { + type Output = Result<(C, RecordBatch)>; + + fn partitions(&self) -> usize { + self.limit - self.offset + } + + fn poll_next( + &mut self, + cx: &mut Context<'_>, + stream_idx: usize, + ) -> Poll> { + let stream_abs_idx = stream_idx + self.offset; + if stream_abs_idx >= self.limit { + return Poll::Ready(Some(Err(DataFusionError::Internal(format!( + "Invalid stream index {} for offset {} and limit {}", + stream_idx, self.offset, self.limit + ))))); + } + Poll::Ready(ready!(self.streams.lock().poll_next(cx, stream_abs_idx))) + } +} + +impl std::fmt::Debug for OffsetCursorStream { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + f.debug_struct("OffsetCursorStream").finish() + } +} + +/// A newtype wrapper around a set of fused [`MergeStream`] +/// that implements debug, and skips over empty inner poll results +struct FusedMergeStreams(Vec>>); + +impl FusedMergeStreams { + fn poll_next( + &mut self, + cx: &mut Context<'_>, + stream_idx: usize, + ) -> Poll, Vec, Vec)>>> { + loop { + match ready!(self.0[stream_idx].poll_next_unpin(cx)) { + Some(Ok((_, _, sort_order))) if sort_order.len() == 0 => continue, + r => return Poll::Ready(r), + } + } + } +} + +pub struct YieldedCursorStream { + // inner polled batches, per stream_idx, which are partially yielded + batches: Vec>>, + // inner polled batch cursors, per stream_idx, which are partially yielded + cursors: Vec>>, + /// Streams being polled + streams: FusedMergeStreams, +} + +impl YieldedCursorStream { + pub fn new(streams: Vec>) -> Self { + let stream_cnt = streams.len(); + Self { + batches: (0..stream_cnt).map(|_| None).collect(), + cursors: (0..stream_cnt).map(|_| None).collect(), + streams: FusedMergeStreams(streams.into_iter().map(|s| s.fuse()).collect()), + } + } + + fn incr_next_batch(&mut self, stream_idx: usize) -> Option<(C, RecordBatch)> { + if let (Some(cursors), Some(batches)) = + (&mut self.cursors[stream_idx], &mut self.batches[stream_idx]) + { + cursors.pop_front().zip(batches.pop_front()) + } else { + None + } + } + + // TODO: in order to handle sort_order, we need to either: + // parse further + // or concat the batches and cursors + fn try_parse_batches( + &mut self, + stream_idx: usize, + batches: Vec, + cursors: Vec, + sort_order: Vec, + ) -> Result<()> { + let mut parsed_batches = Vec::new(); + let mut parsed_cursors = Vec::new(); + let mut prev_batch_idx = sort_order[0].0; + let mut start_row_idx = sort_order[0].1; + let mut len = 0; + for (batch_idx, row_idx) in sort_order.iter() { + if prev_batch_idx == *batch_idx { + len += 1; + continue; + } else { + // parse batch + parsed_batches.push(batches[prev_batch_idx].slice(start_row_idx, len)); + parsed_cursors.push(cursors[prev_batch_idx].slice(start_row_idx, len)?); + + prev_batch_idx = *batch_idx; + start_row_idx = *row_idx; + len = 1; + } + } + parsed_batches.push(batches[prev_batch_idx].slice(start_row_idx, len)); + parsed_cursors.push(cursors[prev_batch_idx].slice(start_row_idx, len)?); + + self.batches[stream_idx] = Some(VecDeque::from(parsed_batches)); + self.cursors[stream_idx] = Some(VecDeque::from(parsed_cursors)); + return Ok(()); + } +} + +impl PartitionedStream for YieldedCursorStream { + type Output = Result<(C, RecordBatch)>; + + fn partitions(&self) -> usize { + self.streams.0.len() + } + + fn poll_next( + &mut self, + cx: &mut Context<'_>, + stream_idx: usize, + ) -> Poll> { + match self.incr_next_batch(stream_idx) { + None => match ready!(self.streams.poll_next(cx, stream_idx)) { + None => Poll::Ready(None), + Some(Err(e)) => Poll::Ready(Some(Err(e))), + Some(Ok((batches, cursors, sort_order))) => { + self.try_parse_batches(stream_idx, batches, cursors, sort_order)?; + Poll::Ready((Ok(self.incr_next_batch(stream_idx))).transpose()) + } + }, + Some(r) => Poll::Ready(Some(Ok(r))), + } + } +} + +impl std::fmt::Debug for YieldedCursorStream { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + f.debug_struct("YieldedCursorStream") + .field("num_partitions", &self.partitions()) + .finish() + } +} From eb647ea98aa318fe79eccb205be2b4e457a198a2 Mon Sep 17 00:00:00 2001 From: wiedld Date: Tue, 22 Aug 2023 14:40:39 -0700 Subject: [PATCH 09/24] feat(7181): build multiple-level cascade tree. --- .../core/src/physical_plan/sorts/cascade.rs | 47 +++++++++++++------ 1 file changed, 32 insertions(+), 15 deletions(-) diff --git a/datafusion/core/src/physical_plan/sorts/cascade.rs b/datafusion/core/src/physical_plan/sorts/cascade.rs index ccbf8dc8d137..3ac0ae7d2063 100644 --- a/datafusion/core/src/physical_plan/sorts/cascade.rs +++ b/datafusion/core/src/physical_plan/sorts/cascade.rs @@ -13,6 +13,7 @@ use arrow::record_batch::RecordBatch; use datafusion_common::Result; use datafusion_execution::memory_pool::MemoryReservation; use futures::Stream; +use std::collections::VecDeque; use std::pin::Pin; use std::sync::Arc; use std::task::{Context, Poll}; @@ -51,10 +52,11 @@ impl SortPreservingCascadeStream { // Refer to YieldedCursorStream for where the concat would happen (TODO). let streams = Arc::new(parking_lot::Mutex::new(streams)); - let max_streams_per_merge = 2; // TODO: change this to 10, once we have tested with 2 (to force more leaf nodes) - let mut divided_streams: Vec> = - Vec::with_capacity(stream_count / max_streams_per_merge + 1); + let max_streams_per_merge = 2; // TODO: change this to 10, once we have tested with 2 (to force more cascade levels) + let mut divided_streams: VecDeque> = + VecDeque::with_capacity(stream_count / max_streams_per_merge + 1); + // build leaves for stream_offset in (0..stream_count).step_by(max_streams_per_merge) { let limit = std::cmp::min(stream_offset + max_streams_per_merge, stream_count); @@ -63,7 +65,7 @@ impl SortPreservingCascadeStream { let streams = OffsetCursorStream::new(Arc::clone(&streams), stream_offset, limit); - divided_streams.push(Box::pin(SortPreservingMergeStream::new( + divided_streams.push_back(Box::pin(SortPreservingMergeStream::new( Box::new(streams), metrics.clone(), batch_size, @@ -72,21 +74,36 @@ impl SortPreservingCascadeStream { ))); } - let next_level: CursorStream = - Box::new(YieldedCursorStream::new(divided_streams)); + // build rest of tree + let mut next_level: VecDeque> = + VecDeque::with_capacity(divided_streams.len() / max_streams_per_merge + 1); + while divided_streams.len() > 1 || !next_level.is_empty() { + let fan_in: Vec> = divided_streams + .drain(0..std::cmp::min(max_streams_per_merge, divided_streams.len())) + .collect(); - let root: MergeStream = Box::pin(SortPreservingMergeStream::new( - next_level, - metrics.clone(), - batch_size, - fetch, - reservation, - )); + next_level.push_back(Box::pin(SortPreservingMergeStream::new( + Box::new(YieldedCursorStream::new(fan_in)), + metrics.clone(), + batch_size, + if divided_streams.is_empty() && next_level.is_empty() { + fetch + } else { + None + }, // fetch, the LIMIT, is applied to the final merge + reservation.new_empty(), + ))); + // in order to maintain sort-preserving streams, don't mix the merge tree levels. + if divided_streams.is_empty() { + divided_streams = next_level.drain(..).collect(); + } + } - let cascade = root; Self { aborted: false, - cascade, + cascade: divided_streams + .remove(0) + .expect("must have a root merge stream"), schema, metrics, } From 8cd22a028fc7a185f9dec80b5658cf5db8d944cf Mon Sep 17 00:00:00 2001 From: wiedld Date: Mon, 28 Aug 2023 14:25:47 -0700 Subject: [PATCH 10/24] feat(7181): use RecordBatch tracking to avoid expensive slicing of batches. This also enabled simplification of code and cursor handling in the SortOrderBuilder. --- .../core/src/physical_plan/sorts/builder.rs | 288 +++++++----------- .../core/src/physical_plan/sorts/cascade.rs | 73 +++-- .../core/src/physical_plan/sorts/cursor.rs | 12 + .../core/src/physical_plan/sorts/merge.rs | 20 +- .../core/src/physical_plan/sorts/stream.rs | 159 +++++++--- 5 files changed, 307 insertions(+), 245 deletions(-) diff --git a/datafusion/core/src/physical_plan/sorts/builder.rs b/datafusion/core/src/physical_plan/sorts/builder.rs index 148a99e9cdf4..9fc60a031f74 100644 --- a/datafusion/core/src/physical_plan/sorts/builder.rs +++ b/datafusion/core/src/physical_plan/sorts/builder.rs @@ -15,65 +15,52 @@ // specific language governing permissions and limitations // under the License. -use arrow::record_batch::RecordBatch; use datafusion_common::Result; -use datafusion_execution::memory_pool::MemoryReservation; -use std::collections::HashMap; -use std::mem::{replace, take}; +use uuid::Uuid; +use std::collections::VecDeque; +use std::mem::take; use super::cursor::Cursor; +use super::stream::BatchOffset; -pub type SortOrder = (usize, usize); // batch_idx, row_idx +pub type SortOrder = (Uuid, usize, BatchOffset); // batch_id, row_idx (without offset) #[derive(Debug)] struct BatchCursor { - /// The index into SortOrderBuilder::batches - batch_idx: usize, + /// The index into BatchTrackingStream::batches + batch: Uuid, /// The row index within the given batch row_idx: usize, + /// The offset of the row within the given batch, based on sliced cursors + row_offset: BatchOffset, /// The cursor for the given batch. If None, the batch is finished. - cursor: Option, + cursor: C, } -/// Provides an API to incrementally build a [`RecordBatch`] from partitioned [`RecordBatch`] +/// Provides an API to incrementally build a [`SortOrder`] from partitioned [`RecordBatch`](arrow::record_batch::RecordBatch)es #[derive(Debug)] pub struct SortOrderBuilder { - /// Maintain a list of [`RecordBatch`] and their corresponding stream - batches: Vec<(usize, RecordBatch)>, - /// Maintain a list of cursors for each finished (sorted) batch /// The number of total batches can be larger than the number of total streams - batch_cursors: Vec>, - - /// Accounts for memory used by buffered batches - reservation: MemoryReservation, + batch_cursors: VecDeque>, - /// The current [`BatchCursor`] for each stream - cursors: Vec>, + /// The current [`BatchCursor`] for each stream_idx + cursors: Vec>>, /// The accumulated stream indexes from which to pull rows indices: Vec, + + stream_count: usize, } impl SortOrderBuilder { /// Create a new [`SortOrderBuilder`] with the provided `stream_count` and `batch_size` - pub fn new( - stream_count: usize, - batch_size: usize, - reservation: MemoryReservation, - ) -> Self { + pub fn new(stream_count: usize, batch_size: usize) -> Self { Self { - batches: Vec::with_capacity(stream_count * 2), - batch_cursors: Vec::with_capacity(stream_count * 2), - cursors: (0..stream_count) - .map(|_| BatchCursor { - batch_idx: 0, - row_idx: 0, - cursor: None, - }) - .collect(), + batch_cursors: VecDeque::with_capacity(stream_count * 2), + cursors: (0..stream_count).map(|_| None).collect(), indices: Vec::with_capacity(batch_size), - reservation, + stream_count, } } @@ -81,27 +68,34 @@ impl SortOrderBuilder { pub fn push_batch( &mut self, stream_idx: usize, - batch: RecordBatch, - cursor: C, + mut cursor: C, + batch: Uuid, + row_offset: BatchOffset, ) -> Result<()> { - self.reservation.try_grow(batch.get_array_memory_size())?; - let batch_idx = self.batches.len(); - self.batches.push((stream_idx, batch)); - self.batch_cursors.push(None); // placehold until cursor is finished - self.cursors[stream_idx] = BatchCursor { - batch_idx, + cursor.seek(0); + self.cursors[stream_idx] = Some(BatchCursor { + batch, row_idx: 0, - cursor: Some(cursor), - }; + row_offset, + cursor, + }); Ok(()) } /// Append the next row from `stream_idx` pub fn push_row(&mut self, stream_idx: usize) { - let cursor = &mut self.cursors[stream_idx]; - let row_idx = cursor.row_idx; - cursor.row_idx += 1; - self.indices.push((cursor.batch_idx, row_idx)); + if let Some(batch_cursor) = &mut self.cursors[stream_idx] { + let row_idx = batch_cursor.row_idx; + self.indices + .push((batch_cursor.batch, row_idx, batch_cursor.row_offset)); + if batch_cursor.cursor.is_finished() { + self.cursor_finished(stream_idx); + } else { + batch_cursor.row_idx += 1; + } + } else { + unreachable!("row pushed for non-existant cursor"); + }; } /// Returns the number of in-progress rows in this [`SortOrderBuilder`] @@ -116,34 +110,21 @@ impl SortOrderBuilder { /// For a finished cursor, remove from BatchCursor (per stream_idx), and track in batch_cursors (per batch_idx) fn cursor_finished(&mut self, stream_idx: usize) { - let batch_idx = self.cursors[stream_idx].batch_idx; - let row_idx = self.cursors[stream_idx].row_idx; - match replace( - &mut self.cursors[stream_idx], - BatchCursor { - batch_idx, - row_idx, - cursor: None, - }, - ) - .cursor - { - Some(prev_batch_cursor) => { - self.batch_cursors[batch_idx] = Some(prev_batch_cursor) - } - None => unreachable!("previous cursor should not be None"), + if let Some(cursor) = self.cursors[stream_idx].take() { + self.batch_cursors.push_back(cursor); } } /// Advance the cursor for `stream_idx` /// Returns `true` if the cursor was advanced pub fn advance(&mut self, stream_idx: usize) -> bool { - match &mut self.cursors[stream_idx].cursor { - Some(c) => { - c.advance(); - if c.is_finished() { - self.cursor_finished(stream_idx); + match &mut self.cursors[stream_idx] { + Some(batch_cursor) => { + let cursor = &mut batch_cursor.cursor; + if cursor.is_finished() { + return false; } + cursor.advance(); true } None => false, @@ -151,35 +132,29 @@ impl SortOrderBuilder { } /// Returns true if there is an in-progress cursor for a given stream - pub fn cursor_in_progress(&self, stream_idx: usize) -> bool { + pub fn cursor_in_progress(&mut self, stream_idx: usize) -> bool { self.cursors[stream_idx] - .cursor - .as_ref() - .map(|cursor| !cursor.is_finished()) - .unwrap_or(false) + .as_mut() + .map_or(false, |batch_cursor| !batch_cursor.cursor.is_finished()) } /// Returns `true` if the cursor at index `a` is greater than at index `b` #[inline] - pub fn is_gt(&self, stream_idx_a: usize, stream_idx_b: usize) -> bool { + pub fn is_gt(&mut self, stream_idx_a: usize, stream_idx_b: usize) -> bool { match ( self.cursor_in_progress(stream_idx_a), self.cursor_in_progress(stream_idx_b), ) { (false, _) => true, (_, false) => false, - _ => { - match ( - &self.cursors[stream_idx_a].cursor, - &self.cursors[stream_idx_b].cursor, - ) { - (Some(a), Some(b)) => a - .cmp(&b) - .then_with(|| stream_idx_a.cmp(&stream_idx_b)) - .is_gt(), - _ => unreachable!(), - } - } + _ => match (&self.cursors[stream_idx_a], &self.cursors[stream_idx_b]) { + (Some(a), Some(b)) => a + .cursor + .cmp(&b.cursor) + .then_with(|| stream_idx_a.cmp(&stream_idx_b)) + .is_gt(), + _ => unreachable!(), + }, } } @@ -188,110 +163,73 @@ impl SortOrderBuilder { /// This will drain the internal state of the builder, and return `None` if there are no pending pub fn yield_sort_order( &mut self, - ) -> Result, Vec, Vec)>> { + ) -> Result, Vec)>> { if self.is_empty() { return Ok(None); } let sort_order = take(&mut self.indices); + let mut cursors_to_yield: Vec<(C, Uuid, BatchOffset)> = + Vec::with_capacity(self.stream_count * 2); + + // drain already complete cursors + for _ in 0..self.batch_cursors.len() { + let BatchCursor { + batch, + row_idx: _, + row_offset, + cursor: mut row_cursor, + } = self.batch_cursors.pop_front().expect("must have a cursor"); + row_cursor.seek(0); + cursors_to_yield.push((row_cursor, batch, row_offset)); + } - let batch_rows_to_yield = - sort_order - .iter() - .fold(HashMap::new(), |mut acc, (batch_idx, row_idx)| { - acc.insert(batch_idx, row_idx + 1); - acc - }); - - let mut batches_to_retain = Vec::new(); - let mut batch_cursors_to_retain = Vec::new(); // per batch_idx - let mut retained_batch_idx: usize = 0; - - let mut batches_to_yield = Vec::new(); - let mut cursors_to_yield = Vec::new(); // per yielded batch - let mut yielded_batch_idx: usize = 0; - - // since we don't yield all batches, but the sort_order was built with the batch_idx from self.batches - // we need to update the batch_idx mapping in sort_order. - let mut batch_idx_for_sort_order = HashMap::new(); + // split any in_progress cursor + for stream_idx in 0..self.cursors.len() { + let batch_cursor = match self.cursors[stream_idx].take() { + Some(c) => c, + None => continue, + }; + let BatchCursor { + batch, + row_idx, + row_offset, + cursor: row_cursor, + } = batch_cursor; - for batch_idx in 0..self.batches.len() { - let (stream_idx, batch) = &self.batches[batch_idx]; - let nothing_to_yield = !batch_rows_to_yield.contains_key(&batch_idx); - let is_fully_yielded = - !nothing_to_yield && batch_rows_to_yield[&batch_idx] == batch.num_rows(); - // to_split means that the batch has been partially yielded, and we need to split the batch and cursor - let to_split = !is_fully_yielded && !nothing_to_yield; + let is_fully_yielded = row_idx == row_cursor.num_rows(); + let to_split = row_idx > 0 && !is_fully_yielded; if is_fully_yielded { - batches_to_yield.push(batch.to_owned()); - cursors_to_yield.push( - self.batch_cursors[batch_idx] - .take() - .expect("cursor should be Some"), - ); - - batch_idx_for_sort_order.insert(batch_idx, yielded_batch_idx); - yielded_batch_idx += 1; + cursors_to_yield.push((row_cursor, batch, row_offset)); } else if to_split { - let rows_to_yield = batch_rows_to_yield[&batch_idx]; - let split_row_idx = rows_to_yield - 1; - - // split batch - batches_to_retain.push(( - *stream_idx, - batch.slice(split_row_idx + 1, batch.num_rows() - rows_to_yield), - )); - batches_to_yield.push(batch.slice(0, rows_to_yield)); + let row_cursor_to_yield = row_cursor.slice(0, row_idx)?; + let row_cursor_to_retain = + row_cursor.slice(row_idx, row_cursor.num_rows() - row_idx)?; + assert_eq!( + row_cursor_to_yield.num_rows() + row_cursor_to_retain.num_rows(), + row_cursor.num_rows() + ); + drop(row_cursor); // drop the original cursor - // split cursor - let cursor = match self.cursor_in_progress(*stream_idx) { - true => self.cursors[*stream_idx].cursor.take().expect("cursor should be Some"), - false => unreachable!("cursor should be in progress, since the batch is partially yielded"), - }; - let cursor_to_retain = - cursor.slice(split_row_idx + 1, batch.num_rows() - rows_to_yield)?; - self.cursors[*stream_idx] = BatchCursor { - batch_idx: retained_batch_idx, + self.cursors[stream_idx] = Some(BatchCursor { + batch, row_idx: 0, - cursor: Some(cursor_to_retain), - }; - batch_cursors_to_retain.push(None); // placehold until cursor is finished - - cursors_to_yield.push(cursor.slice(0, rows_to_yield)?); - - // Immediately free memory associated with previous cursor - // This does not impact field cursors, which uses zero-copy buffer mem slices - // but it does impact row cursors which use memcopy. - drop(cursor); - - batch_idx_for_sort_order.insert(batch_idx, yielded_batch_idx); - yielded_batch_idx += 1; - retained_batch_idx += 1; - } else if nothing_to_yield { - batches_to_retain.push((*stream_idx, batch.to_owned())); - batch_cursors_to_retain.push(None); // placehold until cursor is finished - self.cursors[*stream_idx].batch_idx = retained_batch_idx; - - retained_batch_idx += 1; + row_offset: BatchOffset(row_offset.0 + row_idx), + cursor: row_cursor_to_retain, + }); + cursors_to_yield.push((row_cursor_to_yield, batch, row_offset)); } else { - unreachable!( - "should be fully yielded, partially yielded, or not yielded at all" - ); + // retained all (nothing yielded) + self.cursors[stream_idx] = Some(BatchCursor { + batch, + row_idx, + row_offset, + cursor: row_cursor, + }); } } - self.batches = batches_to_retain; - self.batch_cursors = batch_cursors_to_retain; - Ok(Some(( - batches_to_yield, - cursors_to_yield, - sort_order - .into_iter() - .map(|(batch_idx, row_idx)| { - (batch_idx_for_sort_order[&batch_idx], row_idx) - }) - .collect(), - ))) + Ok(Some((cursors_to_yield, sort_order))) } } diff --git a/datafusion/core/src/physical_plan/sorts/cascade.rs b/datafusion/core/src/physical_plan/sorts/cascade.rs index 3ac0ae7d2063..0ddfa10238ff 100644 --- a/datafusion/core/src/physical_plan/sorts/cascade.rs +++ b/datafusion/core/src/physical_plan/sorts/cascade.rs @@ -3,7 +3,8 @@ use crate::physical_plan::sorts::builder::SortOrder; use crate::physical_plan::sorts::cursor::Cursor; use crate::physical_plan::sorts::merge::SortPreservingMergeStream; use crate::physical_plan::sorts::stream::{ - CursorStream, MergeStream, OffsetCursorStream, YieldedCursorStream, + BatchCursorStream, BatchTrackingStream, MergeStream, OffsetCursorStream, + YieldedCursorStream, }; use crate::physical_plan::RecordBatchStream; @@ -13,7 +14,7 @@ use arrow::record_batch::RecordBatch; use datafusion_common::Result; use datafusion_execution::memory_pool::MemoryReservation; use futures::Stream; -use std::collections::VecDeque; +use std::collections::{VecDeque, HashMap}; use std::pin::Pin; use std::sync::Arc; use std::task::{Context, Poll}; @@ -30,11 +31,15 @@ pub(crate) struct SortPreservingCascadeStream { /// The schema of the RecordBatches yielded by this stream schema: SchemaRef, + + /// Batches are collected on first yield from the RowCursorStream + /// Subsequent merges in cascade all refer to the [`BatchIdentifier`](super::stream::BatchIdentifier) + record_batch_collector: Arc>>, } impl SortPreservingCascadeStream { pub(crate) fn new( - streams: CursorStream, + streams: BatchCursorStream, schema: SchemaRef, metrics: BaselineMetrics, batch_size: usize, @@ -43,16 +48,14 @@ impl SortPreservingCascadeStream { ) -> Self { let stream_count = streams.partitions(); - // TODO: since we already use a mutex here, - // we can have (for the same relative cost) a mutex for a single holder of record_batches - // which yields a batch_idx tracker. - // In this way, we can do slicing followed by concating of Cursors yielded from each merge, - // without needing to also yield sliced-then-concated batches (which are expensive to concat). - // + // TODO: We can do slicing followed by concating of Cursors yielded from each merge. // Refer to YieldedCursorStream for where the concat would happen (TODO). - let streams = Arc::new(parking_lot::Mutex::new(streams)); + let streams = Arc::new(parking_lot::Mutex::new(BatchTrackingStream::new( + streams, + reservation.new_empty(), + ))); - let max_streams_per_merge = 2; // TODO: change this to 10, once we have tested with 2 (to force more cascade levels) + let max_streams_per_merge = 10; let mut divided_streams: VecDeque> = VecDeque::with_capacity(stream_count / max_streams_per_merge + 1); @@ -70,7 +73,6 @@ impl SortPreservingCascadeStream { metrics.clone(), batch_size, None, // fetch, the LIMIT, is applied to the final merge - reservation.new_empty(), ))); } @@ -91,7 +93,6 @@ impl SortPreservingCascadeStream { } else { None }, // fetch, the LIMIT, is applied to the final merge - reservation.new_empty(), ))); // in order to maintain sort-preserving streams, don't mix the merge tree levels. if divided_streams.is_empty() { @@ -106,24 +107,62 @@ impl SortPreservingCascadeStream { .expect("must have a root merge stream"), schema, metrics, + record_batch_collector: streams, } } fn build_record_batch( &mut self, - batches: Vec, sort_order: Vec, ) -> Result { + let mut batches_needed = Vec::with_capacity(sort_order.len()); + let mut batches_seen: HashMap = HashMap::with_capacity(sort_order.len()); // (batch_idx, rows_sorted) + let mut sort_order_offset_adjusted = Vec::with_capacity(sort_order.len()); + let mut batch_idx: usize = 0; + + for (batch_id, row_idx, offset) in sort_order.iter() { + let batch_idx = match batches_seen.get(batch_id) { + Some((batch_idx, _)) => *batch_idx, + None => { + batches_needed.push(*batch_id); + let batch_now = batch_idx; + batch_idx += 1; + batch_now + }, + }; + sort_order_offset_adjusted.push((batch_idx, *row_idx + offset.0)); + batches_seen.insert(*batch_id, (batch_idx, *row_idx + offset.0 + 1)); + } + + let batch_collecter = self.record_batch_collector.lock(); + let batches = batch_collecter.get_batches(batches_needed.as_slice()); + drop(batch_collecter); + + let batches_to_remove = batches + .iter() + .zip(batches_needed.into_iter()) + .filter_map(|(batch, batch_id)| { + if batch.num_rows() == batches_seen[&batch_id].1 { + Some(batch_id) + } else { + None + } + }).collect::>(); + let columns = (0..self.schema.fields.len()) .map(|column_idx| { let arrays: Vec<_> = batches .iter() .map(|batch| batch.column(column_idx).as_ref()) .collect(); - Ok(interleave(&arrays, sort_order.as_slice())?) + Ok(interleave(&arrays, sort_order_offset_adjusted.as_slice())?) }) .collect::>>()?; + let mut batch_collecter = self.record_batch_collector.lock(); + batch_collecter.remove_batches(batches_to_remove.as_slice()); + drop(batch_collecter); + Ok(RecordBatch::try_new(self.schema.clone(), columns)?) } @@ -141,8 +180,8 @@ impl SortPreservingCascadeStream { self.aborted = true; Poll::Ready(Some(Err(e))) } - Some(Ok((batches, _, sort_order))) => { - match self.build_record_batch(batches, sort_order) { + Some(Ok((_, sort_order))) => { + match self.build_record_batch(sort_order) { Ok(batch) => Poll::Ready(Some(Ok(batch))), Err(e) => { self.aborted = true; diff --git a/datafusion/core/src/physical_plan/sorts/cursor.rs b/datafusion/core/src/physical_plan/sorts/cursor.rs index b4048b2601a1..b0518e4fc7b2 100644 --- a/datafusion/core/src/physical_plan/sorts/cursor.rs +++ b/datafusion/core/src/physical_plan/sorts/cursor.rs @@ -110,6 +110,9 @@ pub trait Cursor: Ord { fn slice(&self, offset: usize, length: usize) -> Result where Self: Sized; + + /// Returns the number of rows in this cursor + fn num_rows(&self) -> usize; } impl Cursor for RowCursor { @@ -139,6 +142,11 @@ impl Cursor for RowCursor { reservation.try_grow(rows.size())?; Ok(Self::new(rows, reservation)) } + + #[inline] + fn num_rows(&self) -> usize { + self.num_rows + } } /// An [`Array`] that can be converted into [`FieldValues`] @@ -343,6 +351,10 @@ impl Cursor for FieldCursor { options: options.clone(), }) } + + fn num_rows(&self) -> usize { + self.values.len() + } } #[cfg(test)] diff --git a/datafusion/core/src/physical_plan/sorts/merge.rs b/datafusion/core/src/physical_plan/sorts/merge.rs index a602c8c49ff6..568e71fcd7d6 100644 --- a/datafusion/core/src/physical_plan/sorts/merge.rs +++ b/datafusion/core/src/physical_plan/sorts/merge.rs @@ -18,11 +18,10 @@ use crate::physical_plan::metrics::BaselineMetrics; use crate::physical_plan::sorts::builder::{SortOrder, SortOrderBuilder}; use crate::physical_plan::sorts::cursor::Cursor; -use crate::physical_plan::sorts::stream::CursorStream; -use arrow::record_batch::RecordBatch; +use crate::physical_plan::sorts::stream::{BatchOffset, CursorStream}; use datafusion_common::Result; -use datafusion_execution::memory_pool::MemoryReservation; use futures::Stream; +use uuid::Uuid; use std::pin::Pin; use std::task::{ready, Context, Poll}; @@ -93,12 +92,11 @@ impl SortPreservingMergeStream { metrics: BaselineMetrics, batch_size: usize, fetch: Option, - reservation: MemoryReservation, ) -> Self { let stream_count = streams.partitions(); Self { - in_progress: SortOrderBuilder::new(stream_count, batch_size, reservation), + in_progress: SortOrderBuilder::new(stream_count, batch_size), streams, metrics, aborted: false, @@ -126,16 +124,18 @@ impl SortPreservingMergeStream { match futures::ready!(self.streams.poll_next(cx, idx)) { None => Poll::Ready(Ok(())), Some(Err(e)) => Poll::Ready(Err(e)), - Some(Ok((cursor, batch))) => { - Poll::Ready(self.in_progress.push_batch(idx, batch, cursor)) - } + Some(Ok((cursor, batch_id, batch_offset))) => Poll::Ready( + self.in_progress + .push_batch(idx, cursor, batch_id, batch_offset), + ), } } fn poll_next_inner( &mut self, cx: &mut Context<'_>, - ) -> Poll, Vec, Vec)>>> { + ) -> Poll, Vec)>>> + { if self.aborted { return Poll::Ready(None); } @@ -278,7 +278,7 @@ impl SortPreservingMergeStream { } impl Stream for SortPreservingMergeStream { - type Item = Result<(Vec, Vec, Vec)>; + type Item = Result<(Vec<(C, Uuid, BatchOffset)>, Vec)>; fn poll_next( mut self: Pin<&mut Self>, diff --git a/datafusion/core/src/physical_plan/sorts/stream.rs b/datafusion/core/src/physical_plan/sorts/stream.rs index 5dfd6e09645b..d61922481e47 100644 --- a/datafusion/core/src/physical_plan/sorts/stream.rs +++ b/datafusion/core/src/physical_plan/sorts/stream.rs @@ -27,20 +27,30 @@ use datafusion_common::{DataFusionError, Result}; use datafusion_execution::memory_pool::MemoryReservation; use futures::stream::{Fuse, StreamExt}; use parking_lot::Mutex; -use std::collections::VecDeque; +use uuid::Uuid; +use std::collections::{HashMap, VecDeque}; use std::marker::PhantomData; use std::sync::Arc; use std::task::{ready, Context, Poll}; /// A fallible [`PartitionedStream`] of [`Cursor`] and [`RecordBatch`] -pub(crate) type CursorStream = +pub(crate) type BatchCursorStream = Box>>; -/// A stream of yielded [`SortOrder`]s, with the corresponding [`Cursor`]s and [`RecordBatch`]es +#[derive(Debug, Clone, Copy, Hash, PartialEq, Eq)] +pub struct BatchOffset(pub usize); // offset into a batch, used when a cursor is sliced + +/// A fallible [`PartitionedStream`] of [`Cursor`] and a batch identifier (Uuid) +pub(crate) type CursorStream = + Box>>; + +/// A stream of yielded [`SortOrder`]s, with the corresponding [`Cursor`]s and [`Uuid`]s pub(crate) type MergeStream = std::pin::Pin< Box< dyn Send - + futures::Stream, Vec, Vec)>>, + + futures::Stream< + Item = Result<(Vec<(C, Uuid, BatchOffset)>, Vec)>, + >, >, >; @@ -223,17 +233,17 @@ impl PartitionedStream for FieldCursorStream { } } -/// A wrapper around [`CursorStream`] -/// that provides polling of a subset of the streams. +/// A wrapper around [`CursorStream`] that implements [`PartitionedStream`] +/// and provides polling of a subset of the streams. pub struct OffsetCursorStream { - streams: Arc>>, + streams: Arc>>, offset: usize, limit: usize, } impl OffsetCursorStream { pub fn new( - streams: Arc>>, + streams: Arc>>, offset: usize, limit: usize, ) -> Self { @@ -246,7 +256,7 @@ impl OffsetCursorStream { } impl PartitionedStream for OffsetCursorStream { - type Output = Result<(C, RecordBatch)>; + type Output = Result<(C, Uuid, BatchOffset)>; fn partitions(&self) -> usize { self.limit - self.offset @@ -274,6 +284,50 @@ impl std::fmt::Debug for OffsetCursorStream { } } +pub struct BatchTrackingStream { + /// Write once, read many [`RecordBatch`]s + batches: HashMap>, + /// Input streams yielding [`Cursor`]s and [`RecordBatch`]es + streams: BatchCursorStream, + /// Accounts for memory used by buffered batches + reservation: MemoryReservation, +} + +impl BatchTrackingStream { + pub fn new(streams: BatchCursorStream, reservation: MemoryReservation) -> Self { + Self { + batches: HashMap::new(), + streams, + reservation, + } + } + + pub fn get_batches(&self, batch_ids: &[Uuid]) -> Vec> { + batch_ids.iter().map(|id| self.batches[id].clone()).collect() + } + + pub fn remove_batches(&mut self, batch_ids: &[Uuid]) { + for id in batch_ids { + self.batches.remove(id); + } + } + + fn poll_next( + &mut self, + cx: &mut Context<'_>, + stream_idx: usize, + ) -> Poll>> { + Poll::Ready(ready!(self.streams.poll_next(cx, stream_idx)).map(|r| { + r.and_then(|(cursor, batch)| { + self.reservation.try_grow(batch.get_array_memory_size())?; + let batch_id = Uuid::new_v4(); + self.batches.insert(batch_id, Arc::new(batch)); + Ok((cursor, batch_id, BatchOffset(0_usize))) + }) + })) + } +} + /// A newtype wrapper around a set of fused [`MergeStream`] /// that implements debug, and skips over empty inner poll results struct FusedMergeStreams(Vec>>); @@ -283,10 +337,11 @@ impl FusedMergeStreams { &mut self, cx: &mut Context<'_>, stream_idx: usize, - ) -> Poll, Vec, Vec)>>> { + ) -> Poll, Vec)>>> + { loop { match ready!(self.0[stream_idx].poll_next_unpin(cx)) { - Some(Ok((_, _, sort_order))) if sort_order.len() == 0 => continue, + Some(Ok((_, sort_order))) if sort_order.len() == 0 => continue, r => return Poll::Ready(r), } } @@ -294,10 +349,8 @@ impl FusedMergeStreams { } pub struct YieldedCursorStream { - // inner polled batches, per stream_idx, which are partially yielded - batches: Vec>>, // inner polled batch cursors, per stream_idx, which are partially yielded - cursors: Vec>>, + cursors: Vec>>, /// Streams being polled streams: FusedMergeStreams, } @@ -306,62 +359,82 @@ impl YieldedCursorStream { pub fn new(streams: Vec>) -> Self { let stream_cnt = streams.len(); Self { - batches: (0..stream_cnt).map(|_| None).collect(), cursors: (0..stream_cnt).map(|_| None).collect(), streams: FusedMergeStreams(streams.into_iter().map(|s| s.fuse()).collect()), } } - fn incr_next_batch(&mut self, stream_idx: usize) -> Option<(C, RecordBatch)> { - if let (Some(cursors), Some(batches)) = - (&mut self.cursors[stream_idx], &mut self.batches[stream_idx]) - { - cursors.pop_front().zip(batches.pop_front()) - } else { - None - } + fn incr_next_batch( + &mut self, + stream_idx: usize, + ) -> Option<(C, Uuid, BatchOffset)> { + self.cursors[stream_idx] + .as_mut() + .map(|queue| queue.pop_front()) + .flatten() } // TODO: in order to handle sort_order, we need to either: // parse further - // or concat the batches and cursors + // or concat the cursors fn try_parse_batches( &mut self, stream_idx: usize, - batches: Vec, - cursors: Vec, + cursors: Vec<(C, Uuid, BatchOffset)>, sort_order: Vec, ) -> Result<()> { - let mut parsed_batches = Vec::new(); - let mut parsed_cursors = Vec::new(); - let mut prev_batch_idx = sort_order[0].0; - let mut start_row_idx = sort_order[0].1; + let mut cursors_per_batch: HashMap<(Uuid, BatchOffset), C> = + HashMap::with_capacity(cursors.len()); + for (cursor, batch_id, batch_offset) in cursors { + cursors_per_batch.insert((batch_id, batch_offset), cursor); + } + + let mut parsed_cursors: Vec<(C, Uuid, BatchOffset)> = + Vec::with_capacity(sort_order.len()); + let (mut prev_batch_id, mut prev_row_idx, mut prev_batch_offset) = sort_order[0]; let mut len = 0; - for (batch_idx, row_idx) in sort_order.iter() { - if prev_batch_idx == *batch_idx { + + for (batch_id, row_idx, batch_offset) in sort_order.iter() { + if prev_batch_id == *batch_id && batch_offset.0 == prev_batch_offset.0 { len += 1; continue; } else { - // parse batch - parsed_batches.push(batches[prev_batch_idx].slice(start_row_idx, len)); - parsed_cursors.push(cursors[prev_batch_idx].slice(start_row_idx, len)?); + // parse cursor + if let Some(cursor) = + cursors_per_batch.get(&(prev_batch_id, prev_batch_offset)) + { + let parsed_cursor = cursor.slice(prev_row_idx, len)?; + parsed_cursors.push(( + parsed_cursor, + prev_batch_id, + BatchOffset(prev_row_idx + prev_batch_offset.0), + )); + } else { + unreachable!("cursor not found"); + } - prev_batch_idx = *batch_idx; - start_row_idx = *row_idx; + prev_batch_id = *batch_id; + prev_row_idx = *row_idx; + prev_batch_offset = *batch_offset; len = 1; } } - parsed_batches.push(batches[prev_batch_idx].slice(start_row_idx, len)); - parsed_cursors.push(cursors[prev_batch_idx].slice(start_row_idx, len)?); + if let Some(cursor) = cursors_per_batch.get(&(prev_batch_id, prev_batch_offset)) { + let parsed_cursor = cursor.slice(prev_row_idx, len)?; + parsed_cursors.push(( + parsed_cursor, + prev_batch_id, + BatchOffset(prev_row_idx + prev_batch_offset.0), + )); + } - self.batches[stream_idx] = Some(VecDeque::from(parsed_batches)); self.cursors[stream_idx] = Some(VecDeque::from(parsed_cursors)); return Ok(()); } } impl PartitionedStream for YieldedCursorStream { - type Output = Result<(C, RecordBatch)>; + type Output = Result<(C, Uuid, BatchOffset)>; fn partitions(&self) -> usize { self.streams.0.len() @@ -376,8 +449,8 @@ impl PartitionedStream for YieldedCursorStream None => match ready!(self.streams.poll_next(cx, stream_idx)) { None => Poll::Ready(None), Some(Err(e)) => Poll::Ready(Some(Err(e))), - Some(Ok((batches, cursors, sort_order))) => { - self.try_parse_batches(stream_idx, batches, cursors, sort_order)?; + Some(Ok((cursors, sort_order))) => { + self.try_parse_batches(stream_idx, cursors, sort_order)?; Poll::Ready((Ok(self.incr_next_batch(stream_idx))).transpose()) } }, From 173577b093d919dd56dae68537d8230981429ade Mon Sep 17 00:00:00 2001 From: wiedld Date: Tue, 29 Aug 2023 14:13:31 -0700 Subject: [PATCH 11/24] fix(7181): improve performance by using hasher on tuple (unqiue sliced record batch) --- datafusion/core/src/physical_plan/sorts/stream.rs | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/datafusion/core/src/physical_plan/sorts/stream.rs b/datafusion/core/src/physical_plan/sorts/stream.rs index d61922481e47..f7b88f9f7c30 100644 --- a/datafusion/core/src/physical_plan/sorts/stream.rs +++ b/datafusion/core/src/physical_plan/sorts/stream.rs @@ -19,6 +19,7 @@ use crate::physical_plan::sorts::builder::SortOrder; use crate::physical_plan::sorts::cursor::{Cursor, FieldArray, FieldCursor, RowCursor}; use crate::physical_plan::SendableRecordBatchStream; use crate::physical_plan::{PhysicalExpr, PhysicalSortExpr}; +use ahash::RandomState; use arrow::array::Array; use arrow::datatypes::Schema; use arrow::record_batch::RecordBatch; @@ -286,7 +287,7 @@ impl std::fmt::Debug for OffsetCursorStream { pub struct BatchTrackingStream { /// Write once, read many [`RecordBatch`]s - batches: HashMap>, + batches: HashMap, RandomState>, /// Input streams yielding [`Cursor`]s and [`RecordBatch`]es streams: BatchCursorStream, /// Accounts for memory used by buffered batches @@ -296,7 +297,7 @@ pub struct BatchTrackingStream { impl BatchTrackingStream { pub fn new(streams: BatchCursorStream, reservation: MemoryReservation) -> Self { Self { - batches: HashMap::new(), + batches: HashMap::with_hasher(RandomState::new()), streams, reservation, } @@ -383,8 +384,8 @@ impl YieldedCursorStream { cursors: Vec<(C, Uuid, BatchOffset)>, sort_order: Vec, ) -> Result<()> { - let mut cursors_per_batch: HashMap<(Uuid, BatchOffset), C> = - HashMap::with_capacity(cursors.len()); + let mut cursors_per_batch: HashMap<(Uuid, BatchOffset), C, RandomState> = + HashMap::with_capacity_and_hasher(cursors.len(), RandomState::new()); for (cursor, batch_id, batch_offset) in cursors { cursors_per_batch.insert((batch_id, batch_offset), cursor); } From b0f140211c042257e33fcdca10d3b627fe8fba8c Mon Sep 17 00:00:00 2001 From: wiedld Date: Thu, 31 Aug 2023 14:20:31 -0700 Subject: [PATCH 12/24] chore(7181): make a zero-cost BatchId type, for more explicit code --- .../core/src/physical_plan/sorts/builder.rs | 13 +++---- .../core/src/physical_plan/sorts/cascade.rs | 12 +++--- .../core/src/physical_plan/sorts/merge.rs | 8 ++-- .../core/src/physical_plan/sorts/stream.rs | 38 ++++++++++--------- 4 files changed, 35 insertions(+), 36 deletions(-) diff --git a/datafusion/core/src/physical_plan/sorts/builder.rs b/datafusion/core/src/physical_plan/sorts/builder.rs index 9fc60a031f74..75f40ad9fd6d 100644 --- a/datafusion/core/src/physical_plan/sorts/builder.rs +++ b/datafusion/core/src/physical_plan/sorts/builder.rs @@ -16,19 +16,18 @@ // under the License. use datafusion_common::Result; -use uuid::Uuid; use std::collections::VecDeque; use std::mem::take; use super::cursor::Cursor; -use super::stream::BatchOffset; +use super::stream::{BatchId, BatchOffset}; -pub type SortOrder = (Uuid, usize, BatchOffset); // batch_id, row_idx (without offset) +pub type SortOrder = (BatchId, usize, BatchOffset); // batch_id, row_idx (without offset) #[derive(Debug)] struct BatchCursor { /// The index into BatchTrackingStream::batches - batch: Uuid, + batch: BatchId, /// The row index within the given batch row_idx: usize, /// The offset of the row within the given batch, based on sliced cursors @@ -69,7 +68,7 @@ impl SortOrderBuilder { &mut self, stream_idx: usize, mut cursor: C, - batch: Uuid, + batch: BatchId, row_offset: BatchOffset, ) -> Result<()> { cursor.seek(0); @@ -163,13 +162,13 @@ impl SortOrderBuilder { /// This will drain the internal state of the builder, and return `None` if there are no pending pub fn yield_sort_order( &mut self, - ) -> Result, Vec)>> { + ) -> Result, Vec)>> { if self.is_empty() { return Ok(None); } let sort_order = take(&mut self.indices); - let mut cursors_to_yield: Vec<(C, Uuid, BatchOffset)> = + let mut cursors_to_yield: Vec<(C, BatchId, BatchOffset)> = Vec::with_capacity(self.stream_count * 2); // drain already complete cursors diff --git a/datafusion/core/src/physical_plan/sorts/cascade.rs b/datafusion/core/src/physical_plan/sorts/cascade.rs index 0ddfa10238ff..acd6ca679880 100644 --- a/datafusion/core/src/physical_plan/sorts/cascade.rs +++ b/datafusion/core/src/physical_plan/sorts/cascade.rs @@ -3,7 +3,7 @@ use crate::physical_plan::sorts::builder::SortOrder; use crate::physical_plan::sorts::cursor::Cursor; use crate::physical_plan::sorts::merge::SortPreservingMergeStream; use crate::physical_plan::sorts::stream::{ - BatchCursorStream, BatchTrackingStream, MergeStream, OffsetCursorStream, + BatchCursorStream, BatchId, BatchTrackingStream, MergeStream, OffsetCursorStream, YieldedCursorStream, }; use crate::physical_plan::RecordBatchStream; @@ -33,7 +33,7 @@ pub(crate) struct SortPreservingCascadeStream { schema: SchemaRef, /// Batches are collected on first yield from the RowCursorStream - /// Subsequent merges in cascade all refer to the [`BatchIdentifier`](super::stream::BatchIdentifier) + /// Subsequent merges in cascade all refer to the [`BatchId`]s record_batch_collector: Arc>>, } @@ -111,12 +111,10 @@ impl SortPreservingCascadeStream { } } - fn build_record_batch( - &mut self, - sort_order: Vec, - ) -> Result { + fn build_record_batch(&mut self, sort_order: Vec) -> Result { let mut batches_needed = Vec::with_capacity(sort_order.len()); - let mut batches_seen: HashMap = HashMap::with_capacity(sort_order.len()); // (batch_idx, rows_sorted) + let mut batches_seen: HashMap = + HashMap::with_capacity(sort_order.len()); // (batch_idx, rows_sorted) let mut sort_order_offset_adjusted = Vec::with_capacity(sort_order.len()); let mut batch_idx: usize = 0; diff --git a/datafusion/core/src/physical_plan/sorts/merge.rs b/datafusion/core/src/physical_plan/sorts/merge.rs index 568e71fcd7d6..70b3befb58b8 100644 --- a/datafusion/core/src/physical_plan/sorts/merge.rs +++ b/datafusion/core/src/physical_plan/sorts/merge.rs @@ -18,10 +18,9 @@ use crate::physical_plan::metrics::BaselineMetrics; use crate::physical_plan::sorts::builder::{SortOrder, SortOrderBuilder}; use crate::physical_plan::sorts::cursor::Cursor; -use crate::physical_plan::sorts::stream::{BatchOffset, CursorStream}; +use crate::physical_plan::sorts::stream::{BatchId, BatchOffset, CursorStream}; use datafusion_common::Result; use futures::Stream; -use uuid::Uuid; use std::pin::Pin; use std::task::{ready, Context, Poll}; @@ -134,8 +133,7 @@ impl SortPreservingMergeStream { fn poll_next_inner( &mut self, cx: &mut Context<'_>, - ) -> Poll, Vec)>>> - { + ) -> Poll, Vec)>>> { if self.aborted { return Poll::Ready(None); } @@ -278,7 +276,7 @@ impl SortPreservingMergeStream { } impl Stream for SortPreservingMergeStream { - type Item = Result<(Vec<(C, Uuid, BatchOffset)>, Vec)>; + type Item = Result<(Vec<(C, BatchId, BatchOffset)>, Vec)>; fn poll_next( mut self: Pin<&mut Self>, diff --git a/datafusion/core/src/physical_plan/sorts/stream.rs b/datafusion/core/src/physical_plan/sorts/stream.rs index f7b88f9f7c30..036adf7cb7c0 100644 --- a/datafusion/core/src/physical_plan/sorts/stream.rs +++ b/datafusion/core/src/physical_plan/sorts/stream.rs @@ -38,19 +38,21 @@ use std::task::{ready, Context, Poll}; pub(crate) type BatchCursorStream = Box>>; +pub type BatchId = Uuid; + #[derive(Debug, Clone, Copy, Hash, PartialEq, Eq)] pub struct BatchOffset(pub usize); // offset into a batch, used when a cursor is sliced /// A fallible [`PartitionedStream`] of [`Cursor`] and a batch identifier (Uuid) pub(crate) type CursorStream = - Box>>; + Box>>; /// A stream of yielded [`SortOrder`]s, with the corresponding [`Cursor`]s and [`Uuid`]s pub(crate) type MergeStream = std::pin::Pin< Box< dyn Send + futures::Stream< - Item = Result<(Vec<(C, Uuid, BatchOffset)>, Vec)>, + Item = Result<(Vec<(C, BatchId, BatchOffset)>, Vec)>, >, >, >; @@ -257,7 +259,7 @@ impl OffsetCursorStream { } impl PartitionedStream for OffsetCursorStream { - type Output = Result<(C, Uuid, BatchOffset)>; + type Output = Result<(C, BatchId, BatchOffset)>; fn partitions(&self) -> usize { self.limit - self.offset @@ -287,7 +289,7 @@ impl std::fmt::Debug for OffsetCursorStream { pub struct BatchTrackingStream { /// Write once, read many [`RecordBatch`]s - batches: HashMap, RandomState>, + batches: HashMap, RandomState>, /// Input streams yielding [`Cursor`]s and [`RecordBatch`]es streams: BatchCursorStream, /// Accounts for memory used by buffered batches @@ -303,11 +305,14 @@ impl BatchTrackingStream { } } - pub fn get_batches(&self, batch_ids: &[Uuid]) -> Vec> { - batch_ids.iter().map(|id| self.batches[id].clone()).collect() + pub fn get_batches(&self, batch_ids: &[BatchId]) -> Vec> { + batch_ids + .iter() + .map(|id| self.batches[id].clone()) + .collect() } - pub fn remove_batches(&mut self, batch_ids: &[Uuid]) { + pub fn remove_batches(&mut self, batch_ids: &[BatchId]) { for id in batch_ids { self.batches.remove(id); } @@ -317,7 +322,7 @@ impl BatchTrackingStream { &mut self, cx: &mut Context<'_>, stream_idx: usize, - ) -> Poll>> { + ) -> Poll>> { Poll::Ready(ready!(self.streams.poll_next(cx, stream_idx)).map(|r| { r.and_then(|(cursor, batch)| { self.reservation.try_grow(batch.get_array_memory_size())?; @@ -338,8 +343,7 @@ impl FusedMergeStreams { &mut self, cx: &mut Context<'_>, stream_idx: usize, - ) -> Poll, Vec)>>> - { + ) -> Poll, Vec)>>> { loop { match ready!(self.0[stream_idx].poll_next_unpin(cx)) { Some(Ok((_, sort_order))) if sort_order.len() == 0 => continue, @@ -350,8 +354,8 @@ impl FusedMergeStreams { } pub struct YieldedCursorStream { - // inner polled batch cursors, per stream_idx, which are partially yielded - cursors: Vec>>, + // Inner polled batch cursors, per stream_idx, which represent partially yielded batches. + cursors: Vec>>, /// Streams being polled streams: FusedMergeStreams, } @@ -368,7 +372,7 @@ impl YieldedCursorStream { fn incr_next_batch( &mut self, stream_idx: usize, - ) -> Option<(C, Uuid, BatchOffset)> { + ) -> Option<(C, BatchId, BatchOffset)> { self.cursors[stream_idx] .as_mut() .map(|queue| queue.pop_front()) @@ -381,16 +385,16 @@ impl YieldedCursorStream { fn try_parse_batches( &mut self, stream_idx: usize, - cursors: Vec<(C, Uuid, BatchOffset)>, + cursors: Vec<(C, BatchId, BatchOffset)>, sort_order: Vec, ) -> Result<()> { - let mut cursors_per_batch: HashMap<(Uuid, BatchOffset), C, RandomState> = + let mut cursors_per_batch: HashMap<(BatchId, BatchOffset), C, RandomState> = HashMap::with_capacity_and_hasher(cursors.len(), RandomState::new()); for (cursor, batch_id, batch_offset) in cursors { cursors_per_batch.insert((batch_id, batch_offset), cursor); } - let mut parsed_cursors: Vec<(C, Uuid, BatchOffset)> = + let mut parsed_cursors: Vec<(C, BatchId, BatchOffset)> = Vec::with_capacity(sort_order.len()); let (mut prev_batch_id, mut prev_row_idx, mut prev_batch_offset) = sort_order[0]; let mut len = 0; @@ -435,7 +439,7 @@ impl YieldedCursorStream { } impl PartitionedStream for YieldedCursorStream { - type Output = Result<(C, Uuid, BatchOffset)>; + type Output = Result<(C, BatchId, BatchOffset)>; fn partitions(&self) -> usize { self.streams.0.len() From 9ea3a65731d9abe92b77a8c05003c72fcf84712f Mon Sep 17 00:00:00 2001 From: wiedld Date: Thu, 31 Aug 2023 14:22:04 -0700 Subject: [PATCH 13/24] refactor: comment the major streaming structures, and how they are inter-used. Add missing debug fmt. --- .../core/src/physical_plan/sorts/builder.rs | 15 +++- .../core/src/physical_plan/sorts/cascade.rs | 2 - .../core/src/physical_plan/sorts/stream.rs | 76 ++++++++++++++++--- 3 files changed, 79 insertions(+), 14 deletions(-) diff --git a/datafusion/core/src/physical_plan/sorts/builder.rs b/datafusion/core/src/physical_plan/sorts/builder.rs index 75f40ad9fd6d..648a8e3f6d54 100644 --- a/datafusion/core/src/physical_plan/sorts/builder.rs +++ b/datafusion/core/src/physical_plan/sorts/builder.rs @@ -157,9 +157,20 @@ impl SortOrderBuilder { } } - /// Takes the batches which already are sorted, and returns them with the corresponding cursors and sort order + /// Takes the batches which already are sorted, and returns them with the corresponding cursors and sort order. /// - /// This will drain the internal state of the builder, and return `None` if there are no pending + /// This will drain the internal state of the builder, and return `None` if there are no pending. + /// + /// This slices cursors for each record batch, as follows: + /// 1. input was N record_batchs of up to max M size + /// 2. yielded ordered rows can only equal up to M size + /// 3. of the N record_batches, each will be: + /// a. fully yielded (all rows) + /// b. partially yielded (some rows) => slice cursor, and adjust BatchOffset + /// c. not yielded (no rows) => retain cursor + /// 4. output will be: + /// - SortOrder + /// - corresponding cursors, each up to total yielded rows [cursor_batch_0, cursor_batch_1, ..] pub fn yield_sort_order( &mut self, ) -> Result, Vec)>> { diff --git a/datafusion/core/src/physical_plan/sorts/cascade.rs b/datafusion/core/src/physical_plan/sorts/cascade.rs index acd6ca679880..5b1ecf1711b1 100644 --- a/datafusion/core/src/physical_plan/sorts/cascade.rs +++ b/datafusion/core/src/physical_plan/sorts/cascade.rs @@ -48,8 +48,6 @@ impl SortPreservingCascadeStream { ) -> Self { let stream_count = streams.partitions(); - // TODO: We can do slicing followed by concating of Cursors yielded from each merge. - // Refer to YieldedCursorStream for where the concat would happen (TODO). let streams = Arc::new(parking_lot::Mutex::new(BatchTrackingStream::new( streams, reservation.new_empty(), diff --git a/datafusion/core/src/physical_plan/sorts/stream.rs b/datafusion/core/src/physical_plan/sorts/stream.rs index 036adf7cb7c0..3fa42f56ef93 100644 --- a/datafusion/core/src/physical_plan/sorts/stream.rs +++ b/datafusion/core/src/physical_plan/sorts/stream.rs @@ -34,20 +34,33 @@ use std::marker::PhantomData; use std::sync::Arc; use std::task::{ready, Context, Poll}; -/// A fallible [`PartitionedStream`] of [`Cursor`] and [`RecordBatch`] +/// A fallible [`PartitionedStream`] of record batches. +/// +/// Each [`Cursor`] and [`RecordBatch`] represents a single record batch. pub(crate) type BatchCursorStream = Box>>; pub type BatchId = Uuid; #[derive(Debug, Clone, Copy, Hash, PartialEq, Eq)] -pub struct BatchOffset(pub usize); // offset into a batch, used when a cursor is sliced +pub struct BatchOffset(pub usize); -/// A fallible [`PartitionedStream`] of [`Cursor`] and a batch identifier (Uuid) +/// A [`PartitionedStream`] representing partial record batches. +/// +/// Each ([`Cursor`], [`BatchId`], [`BatchOffset`]) represents part of a record batch +/// with the cursor.row_idx=0 representing the normalized key for the row at batch[idx=BatchOffset]. +/// +/// Each merge node (a `SortPreservingMergeStream` loser tree) will consume a [`CursorStream`]. pub(crate) type CursorStream = Box>>; -/// A stream of yielded [`SortOrder`]s, with the corresponding [`Cursor`]s and [`Uuid`]s +/// A fallible stream of yielded [`SortOrder`]s is a [`MergeStream`]. +/// +/// Within a cascade of merge nodes, (each node being a `SortPreservingMergeStream` loser tree), +/// the merge node will yield a SortOrder as well as any partial record batches from the SortOrder. +/// +/// [`YieldedCursorStream`] then converts an output [`MergeStream`] +/// into an input [`CursorStream`] for the next merge. pub(crate) type MergeStream = std::pin::Pin< Box< dyn Send @@ -236,9 +249,13 @@ impl PartitionedStream for FieldCursorStream { } } -/// A wrapper around [`CursorStream`] that implements [`PartitionedStream`] -/// and provides polling of a subset of the streams. +/// A wrapper around [`CursorStream`] that provides polling of a subset of the partitioned streams. +/// +/// This is used in the leaf nodes of the cascading merge tree. +/// To have the same [`CursorStream`] (with the same RowConverter) +/// be separately polled by multiple leaf nodes. pub struct OffsetCursorStream { + // Input streams. [`BatchTrackingStream`] is a [`CursorStream`]. streams: Arc>>, offset: usize, limit: usize, @@ -287,6 +304,10 @@ impl std::fmt::Debug for OffsetCursorStream { } } +/// Converts a [`BatchCursorStream`] into a [`CursorStream`]. +/// +/// While storing the record batches outside of the cascading merge tree. +/// Should be used with a Mutex. pub struct BatchTrackingStream { /// Write once, read many [`RecordBatch`]s batches: HashMap, RandomState>, @@ -317,6 +338,14 @@ impl BatchTrackingStream { self.batches.remove(id); } } +} + +impl PartitionedStream for BatchTrackingStream { + type Output = Result<(C, BatchId, BatchOffset)>; + + fn partitions(&self) -> usize { + self.streams.partitions() + } fn poll_next( &mut self, @@ -334,6 +363,14 @@ impl BatchTrackingStream { } } +impl std::fmt::Debug for BatchTrackingStream { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + f.debug_struct("BatchTrackingStream") + .field("num_partitions", &self.partitions()) + .finish() + } +} + /// A newtype wrapper around a set of fused [`MergeStream`] /// that implements debug, and skips over empty inner poll results struct FusedMergeStreams(Vec>>); @@ -353,8 +390,16 @@ impl FusedMergeStreams { } } +impl std::fmt::Debug for FusedMergeStreams { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + f.debug_struct("FusedMergeStreams").finish() + } +} + +/// [`YieldedCursorStream`] converts an output [`MergeStream`] +/// into an input [`CursorStream`] for the next merge. pub struct YieldedCursorStream { - // Inner polled batch cursors, per stream_idx, which represent partially yielded batches. + // Inner polled batch cursors, per stream_idx, which represents partially yielded batches. cursors: Vec>>, /// Streams being polled streams: FusedMergeStreams, @@ -379,9 +424,20 @@ impl YieldedCursorStream { .flatten() } - // TODO: in order to handle sort_order, we need to either: - // parse further - // or concat the cursors + // The input [`SortOrder`] is across batches. + // We need to further parse the cursors into smaller batches. + // + // Input: + // - sort_order: Vec<(BatchId, row_idx)> = [(0,0), (0,1), (1,0), (0,2), (0,3)] + // - cursors: Vec<(C, BatchId, BatchOffset)> = [cursor_0, cursor_1] + // + // Output stream: + // Needs to be yielded to the next merge in three partial batches: + // [(0,0),(0,1)] with cursor => then [(1,0)] with cursor => then [(0,2),(0,3)] with cursor + // + // This additional parsing is only required when streaming into another merge node, + // and not required when yielding to the final interleave step. + // (Performance slightly decreases when doing this additional parsing for all SortOrderBuilder yields.) fn try_parse_batches( &mut self, stream_idx: usize, From 7be30c2cd5218141d2c480a194309d523d6fc53d Mon Sep 17 00:00:00 2001 From: wiedld Date: Thu, 31 Aug 2023 18:03:25 -0700 Subject: [PATCH 14/24] refactor: use u64 as batch_id in cascading merge sort --- datafusion/core/src/physical_plan/sorts/stream.rs | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/datafusion/core/src/physical_plan/sorts/stream.rs b/datafusion/core/src/physical_plan/sorts/stream.rs index 3fa42f56ef93..61ec50d4dd75 100644 --- a/datafusion/core/src/physical_plan/sorts/stream.rs +++ b/datafusion/core/src/physical_plan/sorts/stream.rs @@ -28,7 +28,6 @@ use datafusion_common::{DataFusionError, Result}; use datafusion_execution::memory_pool::MemoryReservation; use futures::stream::{Fuse, StreamExt}; use parking_lot::Mutex; -use uuid::Uuid; use std::collections::{HashMap, VecDeque}; use std::marker::PhantomData; use std::sync::Arc; @@ -40,7 +39,7 @@ use std::task::{ready, Context, Poll}; pub(crate) type BatchCursorStream = Box>>; -pub type BatchId = Uuid; +pub type BatchId = u64; #[derive(Debug, Clone, Copy, Hash, PartialEq, Eq)] pub struct BatchOffset(pub usize); @@ -309,6 +308,8 @@ impl std::fmt::Debug for OffsetCursorStream { /// While storing the record batches outside of the cascading merge tree. /// Should be used with a Mutex. pub struct BatchTrackingStream { + /// Monotonically increasing batch id + monotonic_counter: u64, /// Write once, read many [`RecordBatch`]s batches: HashMap, RandomState>, /// Input streams yielding [`Cursor`]s and [`RecordBatch`]es @@ -320,6 +321,7 @@ pub struct BatchTrackingStream { impl BatchTrackingStream { pub fn new(streams: BatchCursorStream, reservation: MemoryReservation) -> Self { Self { + monotonic_counter: 0, batches: HashMap::with_hasher(RandomState::new()), streams, reservation, @@ -355,7 +357,8 @@ impl PartitionedStream for BatchTrackingStream { Poll::Ready(ready!(self.streams.poll_next(cx, stream_idx)).map(|r| { r.and_then(|(cursor, batch)| { self.reservation.try_grow(batch.get_array_memory_size())?; - let batch_id = Uuid::new_v4(); + let batch_id = self.monotonic_counter; + self.monotonic_counter += 1; self.batches.insert(batch_id, Arc::new(batch)); Ok((cursor, batch_id, BatchOffset(0_usize))) }) From 0e9573d3950d9dbc005ddb79aa30078db6a0e0f8 Mon Sep 17 00:00:00 2001 From: wiedld Date: Sat, 2 Sep 2023 14:55:01 -0700 Subject: [PATCH 15/24] feat(7181): convert into generic ReceiverStream, such that can be reused for other streamed data (besides RecordBatches) --- datafusion/core/src/physical_plan/analyze.rs | 10 +- .../src/physical_plan/coalesce_partitions.rs | 8 +- datafusion/core/src/physical_plan/common.rs | 4 +- .../core/src/physical_plan/sorts/sort.rs | 4 +- .../sorts/sort_preserving_merge.rs | 7 +- datafusion/core/src/physical_plan/stream.rs | 195 +++++++++++------- datafusion/core/src/test/exec.rs | 6 +- 7 files changed, 144 insertions(+), 90 deletions(-) diff --git a/datafusion/core/src/physical_plan/analyze.rs b/datafusion/core/src/physical_plan/analyze.rs index 0e6edc618242..a3531d5ebf05 100644 --- a/datafusion/core/src/physical_plan/analyze.rs +++ b/datafusion/core/src/physical_plan/analyze.rs @@ -29,7 +29,9 @@ use datafusion_common::{internal_err, DataFusionError, Result}; use futures::StreamExt; use super::expressions::PhysicalSortExpr; -use super::stream::{RecordBatchReceiverStream, RecordBatchStreamAdapter}; +use super::stream::{ + ReceiverStream, RecordBatchReceiverStreamAdaptor, RecordBatchStreamAdapter, +}; use super::{DisplayAs, Distribution, SendableRecordBatchStream}; use datafusion_execution::TaskContext; @@ -131,11 +133,11 @@ impl ExecutionPlan for AnalyzeExec { // parallel (on a separate tokio task) using a JoinSet to // cancel outstanding futures on drop let num_input_partitions = self.input.output_partitioning().partition_count(); - let mut builder = - RecordBatchReceiverStream::builder(self.schema(), num_input_partitions); + let mut builder = ReceiverStream::builder(self.schema(), num_input_partitions); + let input = Arc::new(RecordBatchReceiverStreamAdaptor::new(self.input.clone())); for input_partition in 0..num_input_partitions { - builder.run_input(self.input.clone(), input_partition, context.clone()); + builder.run_input(input.clone(), input_partition, context.clone()); } // Create future that computes thefinal output diff --git a/datafusion/core/src/physical_plan/coalesce_partitions.rs b/datafusion/core/src/physical_plan/coalesce_partitions.rs index 78cb7b201f26..c19865abd567 100644 --- a/datafusion/core/src/physical_plan/coalesce_partitions.rs +++ b/datafusion/core/src/physical_plan/coalesce_partitions.rs @@ -23,7 +23,7 @@ use std::sync::Arc; use super::expressions::PhysicalSortExpr; use super::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; -use super::stream::{ObservedStream, RecordBatchReceiverStream}; +use super::stream::{ObservedStream, ReceiverStream, RecordBatchReceiverStreamAdaptor}; use super::{DisplayAs, SendableRecordBatchStream, Statistics}; use crate::physical_plan::{ @@ -143,12 +143,14 @@ impl ExecutionPlan for CoalescePartitionsExec { // least one result in an attempt to maximize // parallelism. let mut builder = - RecordBatchReceiverStream::builder(self.schema(), input_partitions); + ReceiverStream::builder(self.schema(), input_partitions); + let input = + Arc::new(RecordBatchReceiverStreamAdaptor::new(self.input.clone())); // spawn independent tasks whose resulting streams (of batches) // are sent to the channel for consumption. for part_i in 0..input_partitions { - builder.run_input(self.input.clone(), part_i, context.clone()); + builder.run_input(input.clone(), part_i, context.clone()); } let stream = builder.build(); diff --git a/datafusion/core/src/physical_plan/common.rs b/datafusion/core/src/physical_plan/common.rs index 9e94fe4e2599..92d327eed84a 100644 --- a/datafusion/core/src/physical_plan/common.rs +++ b/datafusion/core/src/physical_plan/common.rs @@ -18,7 +18,7 @@ //! Defines common code used in execution plans use super::SendableRecordBatchStream; -use crate::physical_plan::stream::RecordBatchReceiverStream; +use crate::physical_plan::stream::ReceiverStream; use crate::physical_plan::{ColumnStatistics, ExecutionPlan, Statistics}; use arrow::datatypes::Schema; use arrow::ipc::writer::{FileWriter, IpcWriteOptions}; @@ -102,7 +102,7 @@ pub(crate) fn spawn_buffered( Ok(handle) if handle.runtime_flavor() == tokio::runtime::RuntimeFlavor::MultiThread => { - let mut builder = RecordBatchReceiverStream::builder(input.schema(), buffer); + let mut builder = ReceiverStream::builder(input.schema(), buffer); let sender = builder.tx(); diff --git a/datafusion/core/src/physical_plan/sorts/sort.rs b/datafusion/core/src/physical_plan/sorts/sort.rs index d3cd6c36c606..270407e49043 100644 --- a/datafusion/core/src/physical_plan/sorts/sort.rs +++ b/datafusion/core/src/physical_plan/sorts/sort.rs @@ -25,7 +25,7 @@ use crate::physical_plan::metrics::{ BaselineMetrics, Count, ExecutionPlanMetricsSet, MetricBuilder, MetricsSet, }; use crate::physical_plan::sorts::streaming_merge::streaming_merge; -use crate::physical_plan::stream::{RecordBatchReceiverStream, RecordBatchStreamAdapter}; +use crate::physical_plan::stream::{ReceiverStream, RecordBatchStreamAdapter}; use crate::physical_plan::{ DisplayAs, DisplayFormatType, Distribution, EmptyRecordBatchStream, ExecutionPlan, Partitioning, SendableRecordBatchStream, Statistics, @@ -606,7 +606,7 @@ fn read_spill_as_stream( path: NamedTempFile, schema: SchemaRef, ) -> Result { - let mut builder = RecordBatchReceiverStream::builder(schema, 2); + let mut builder = ReceiverStream::builder(schema, 2); let sender = builder.tx(); builder.spawn_blocking(move || { diff --git a/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs b/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs index 006133eb4830..51d8b0ecc2e9 100644 --- a/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs +++ b/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs @@ -281,7 +281,7 @@ mod tests { use crate::physical_plan::memory::MemoryExec; use crate::physical_plan::metrics::MetricValue; use crate::physical_plan::sorts::sort::SortExec; - use crate::physical_plan::stream::RecordBatchReceiverStream; + use crate::physical_plan::stream::ReceiverStream; use crate::physical_plan::{collect, common}; use crate::test::exec::{assert_strong_count_converges_to_zero, BlockingExec}; use crate::test::{self, assert_is_pending}; @@ -834,10 +834,11 @@ mod tests { sorted_partitioned_input(sort.clone(), &[5, 7, 3], task_ctx.clone()).await?; let partition_count = batches.output_partitioning().partition_count(); - let mut streams = Vec::with_capacity(partition_count); + let mut streams: Vec = + Vec::with_capacity(partition_count); for partition in 0..partition_count { - let mut builder = RecordBatchReceiverStream::builder(schema.clone(), 1); + let mut builder = ReceiverStream::::builder(schema.clone(), 1); let sender = builder.tx(); diff --git a/datafusion/core/src/physical_plan/stream.rs b/datafusion/core/src/physical_plan/stream.rs index 1147f288643c..3bb5d3ef8b6b 100644 --- a/datafusion/core/src/physical_plan/stream.rs +++ b/datafusion/core/src/physical_plan/stream.rs @@ -38,22 +38,22 @@ use tokio::task::JoinSet; use super::metrics::BaselineMetrics; use super::{ExecutionPlan, RecordBatchStream, SendableRecordBatchStream}; -/// Builder for [`RecordBatchReceiverStream`] that propagates errors +/// Builder for [`ReceiverStream`] that propagates errors /// and panic's correctly. /// -/// [`RecordBatchReceiverStream`] is used to spawn one or more tasks -/// that produce `RecordBatch`es and send them to a single -/// `Receiver` which can improve parallelism. +/// [`ReceiverStream`] is used to spawn one or more tasks +/// that produce `O` output, (such as a `RecordBatch`), +/// and sends them to a single `Receiver` which can improve parallelism. /// /// This also handles propagating panic`s and canceling the tasks. -pub struct RecordBatchReceiverStreamBuilder { - tx: Sender>, - rx: Receiver>, +pub struct ReceiverStreamBuilder { + tx: Sender>, + rx: Receiver>, schema: SchemaRef, join_set: JoinSet<()>, } -impl RecordBatchReceiverStreamBuilder { +impl ReceiverStreamBuilder { /// create new channels with the specified buffer size pub fn new(schema: SchemaRef, capacity: usize) -> Self { let (tx, rx) = tokio::sync::mpsc::channel(capacity); @@ -66,8 +66,8 @@ impl RecordBatchReceiverStreamBuilder { } } - /// Get a handle for sending [`RecordBatch`]es to the output - pub fn tx(&self) -> Sender> { + /// Get a handle for sending `O` to the output + pub fn tx(&self) -> Sender> { self.tx.clone() } @@ -104,57 +104,19 @@ impl RecordBatchReceiverStreamBuilder { /// sent to the output stream and no further results are sent. pub(crate) fn run_input( &mut self, - input: Arc, + input: Arc> + Send + Sync>, partition: usize, context: Arc, ) { let output = self.tx(); self.spawn(async move { - let mut stream = match input.execute(partition, context) { - Err(e) => { - // If send fails, the plan being torn down, there - // is no place to send the error and no reason to continue. - output.send(Err(e)).await.ok(); - debug!( - "Stopping execution: error executing input: {}", - displayable(input.as_ref()).one_line() - ); - return; - } - Ok(stream) => stream, - }; - - // Transfer batches from inner stream to the output tx - // immediately. - while let Some(item) = stream.next().await { - let is_err = item.is_err(); - - // If send fails, plan being torn down, there is no - // place to send the error and no reason to continue. - if output.send(item).await.is_err() { - debug!( - "Stopping execution: output is gone, plan cancelling: {}", - displayable(input.as_ref()).one_line() - ); - return; - } - - // stop after the first error is encontered (don't - // drive all streams to completion) - if is_err { - debug!( - "Stopping execution: plan returned error: {}", - displayable(input.as_ref()).one_line() - ); - return; - } - } + input.call(output, partition, context).await; }); } - /// Create a stream of all `RecordBatch`es written to `tx` - pub fn build(self) -> SendableRecordBatchStream { + /// Create a stream of all `O`es written to `tx` + pub fn build(self) -> Pin>> { let Self { tx, rx, @@ -203,34 +165,33 @@ impl RecordBatchReceiverStreamBuilder { // produces the batch let inner = futures::stream::select(rx_stream, check_stream).boxed(); - Box::pin(RecordBatchReceiverStream { schema, inner }) + Box::pin(ReceiverStream::::new(schema, inner)) } } -/// A [`SendableRecordBatchStream`] that combines [`RecordBatch`]es from multiple inputs, -/// on new tokio Tasks, increasing the potential parallelism. +/// A SendableStream that combines multiple inputs, on new tokio Tasks, +/// increasing the potential parallelism. +/// +/// When `O` is a [`RecordBatch`], this is a [`SendableRecordBatchStream`]. /// /// This structure also handles propagating panics and cancelling the /// underlying tasks correctly. /// /// Use [`Self::builder`] to construct one. -pub struct RecordBatchReceiverStream { +pub struct ReceiverStream { schema: SchemaRef, - inner: BoxStream<'static, Result>, + inner: BoxStream<'static, Result>, } -impl RecordBatchReceiverStream { +impl ReceiverStream { /// Create a builder with an internal buffer of capacity batches. - pub fn builder( - schema: SchemaRef, - capacity: usize, - ) -> RecordBatchReceiverStreamBuilder { - RecordBatchReceiverStreamBuilder::new(schema, capacity) + pub fn builder(schema: SchemaRef, capacity: usize) -> ReceiverStreamBuilder { + ReceiverStreamBuilder::::new(schema, capacity) } } -impl Stream for RecordBatchReceiverStream { - type Item = Result; +impl Stream for ReceiverStream { + type Item = Result; fn poll_next( mut self: Pin<&mut Self>, @@ -240,12 +201,93 @@ impl Stream for RecordBatchReceiverStream { } } -impl RecordBatchStream for RecordBatchReceiverStream { +impl ReceiverStream { + /// Create a new [`ReceiverStream`] from the provided schema and stream + pub fn new(schema: SchemaRef, inner: BoxStream<'static, Result>) -> Self { + Self { schema, inner } + } +} + +impl RecordBatchStream for ReceiverStream { fn schema(&self) -> SchemaRef { self.schema.clone() } } +#[async_trait::async_trait] +pub(crate) trait StreamAdaptor: Send + Sync { + type Item: Send + Sync; + + async fn call( + &self, + output: Sender, + partition: usize, + context: Arc, + ); +} + +pub(crate) struct RecordBatchReceiverStreamAdaptor { + input: Arc, +} + +impl RecordBatchReceiverStreamAdaptor { + pub fn new(input: Arc) -> Self { + Self { input } + } +} + +#[async_trait::async_trait] +impl StreamAdaptor for RecordBatchReceiverStreamAdaptor { + type Item = Result; + + async fn call( + &self, + output: Sender, + partition: usize, + context: Arc, + ) { + let mut stream = match self.input.execute(partition, context) { + Err(e) => { + // If send fails, the plan being torn down, there + // is no place to send the error and no reason to continue. + output.send(Err(e)).await.ok(); + debug!( + "Stopping execution: error executing input: {}", + displayable(self.input.as_ref()).one_line() + ); + return; + } + Ok(stream) => stream, + }; + + // Transfer batches from inner stream to the output tx + // immediately. + while let Some(item) = stream.next().await { + let is_err = item.is_err(); + + // If send fails, plan being torn down, there is no + // place to send the error and no reason to continue. + if output.send(item).await.is_err() { + debug!( + "Stopping execution: output is gone, plan cancelling: {}", + displayable(self.input.as_ref()).one_line() + ); + return; + } + + // stop after the first error is encontered (don't + // drive all streams to completion) + if is_err { + debug!( + "Stopping execution: plan returned error: {}", + displayable(self.input.as_ref()).one_line() + ); + return; + } + } + } +} + pin_project! { /// Combines a [`Stream`] with a [`SchemaRef`] implementing /// [`RecordBatchStream`] for the combination @@ -418,8 +460,9 @@ mod test { let refs = input.refs(); // Configure a RecordBatchReceiverStream to consume the input - let mut builder = RecordBatchReceiverStream::builder(schema, 2); - builder.run_input(Arc::new(input), 0, task_ctx.clone()); + let mut builder = ReceiverStream::builder(schema, 2); + let input = Arc::new(RecordBatchReceiverStreamAdaptor::new(Arc::new(input))); + builder.run_input(input, 0, task_ctx.clone()); let stream = builder.build(); // input should still be present @@ -443,8 +486,11 @@ mod test { MockExec::new(vec![exec_err!("Test1"), exec_err!("Test2")], schema.clone()) .with_use_task(false); - let mut builder = RecordBatchReceiverStream::builder(schema, 2); - builder.run_input(Arc::new(error_stream), 0, task_ctx.clone()); + let mut builder = ReceiverStream::builder(schema, 2); + let input = Arc::new(RecordBatchReceiverStreamAdaptor::new(Arc::new( + error_stream, + ))); + builder.run_input(input, 0, task_ctx.clone()); let mut stream = builder.build(); // get the first result, which should be an error @@ -467,10 +513,13 @@ mod test { let num_partitions = input.output_partitioning().partition_count(); // Configure a RecordBatchReceiverStream to consume all the input partitions - let mut builder = - RecordBatchReceiverStream::builder(input.schema(), num_partitions); + let mut builder = ReceiverStream::builder(input.schema(), num_partitions); for partition in 0..num_partitions { - builder.run_input(input.clone(), partition, task_ctx.clone()); + builder.run_input( + Arc::new(RecordBatchReceiverStreamAdaptor::new(input.clone())), + partition, + task_ctx.clone(), + ); } let mut stream = builder.build(); diff --git a/datafusion/core/src/test/exec.rs b/datafusion/core/src/test/exec.rs index 682f31a7fe9a..0fd3a5e5d7ee 100644 --- a/datafusion/core/src/test/exec.rs +++ b/datafusion/core/src/test/exec.rs @@ -38,7 +38,7 @@ use crate::physical_plan::{ use crate::physical_plan::{expressions::PhysicalSortExpr, DisplayAs}; use crate::{ error::{DataFusionError, Result}, - physical_plan::stream::RecordBatchReceiverStream, + physical_plan::stream::ReceiverStream, }; use crate::{ execution::context::TaskContext, physical_plan::stream::RecordBatchStreamAdapter, @@ -216,7 +216,7 @@ impl ExecutionPlan for MockExec { .collect(); if self.use_task { - let mut builder = RecordBatchReceiverStream::builder(self.schema(), 2); + let mut builder = ReceiverStream::builder(self.schema(), 2); // send data in order but in a separate task (to ensure // the batches are not available without the stream // yielding). @@ -349,7 +349,7 @@ impl ExecutionPlan for BarrierExec { ) -> Result { assert!(partition < self.data.len()); - let mut builder = RecordBatchReceiverStream::builder(self.schema(), 2); + let mut builder = ReceiverStream::builder(self.schema(), 2); // task simply sends data in order after barrier is reached let data = self.data[partition].clone(); From c439138aff44c42ade58310845bcc77f0e25c8f0 Mon Sep 17 00:00:00 2001 From: wiedld Date: Sat, 2 Sep 2023 16:56:54 -0700 Subject: [PATCH 16/24] feat(7181): add buffered multithreading to merge streams --- .../core/src/physical_plan/sorts/cascade.rs | 75 ++++++++++++++----- .../core/src/physical_plan/sorts/cursor.rs | 4 +- 2 files changed, 58 insertions(+), 21 deletions(-) diff --git a/datafusion/core/src/physical_plan/sorts/cascade.rs b/datafusion/core/src/physical_plan/sorts/cascade.rs index 5b1ecf1711b1..c8a34816d7f2 100644 --- a/datafusion/core/src/physical_plan/sorts/cascade.rs +++ b/datafusion/core/src/physical_plan/sorts/cascade.rs @@ -6,6 +6,7 @@ use crate::physical_plan::sorts::stream::{ BatchCursorStream, BatchId, BatchTrackingStream, MergeStream, OffsetCursorStream, YieldedCursorStream, }; +use crate::physical_plan::stream::ReceiverStream; use crate::physical_plan::RecordBatchStream; use arrow::compute::interleave; @@ -13,8 +14,8 @@ use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; use datafusion_common::Result; use datafusion_execution::memory_pool::MemoryReservation; -use futures::Stream; -use std::collections::{VecDeque, HashMap}; +use futures::{Stream, StreamExt}; +use std::collections::{HashMap, VecDeque}; use std::pin::Pin; use std::sync::Arc; use std::task::{Context, Poll}; @@ -66,12 +67,16 @@ impl SortPreservingCascadeStream { let streams = OffsetCursorStream::new(Arc::clone(&streams), stream_offset, limit); - divided_streams.push_back(Box::pin(SortPreservingMergeStream::new( - Box::new(streams), - metrics.clone(), - batch_size, - None, // fetch, the LIMIT, is applied to the final merge - ))); + divided_streams.push_back(spawn_buffered_merge( + Box::pin(SortPreservingMergeStream::new( + Box::new(streams), + metrics.clone(), + batch_size, + None, // fetch, the LIMIT, is applied to the final merge + )), + schema.clone(), + 2, + )); } // build rest of tree @@ -82,16 +87,20 @@ impl SortPreservingCascadeStream { .drain(0..std::cmp::min(max_streams_per_merge, divided_streams.len())) .collect(); - next_level.push_back(Box::pin(SortPreservingMergeStream::new( - Box::new(YieldedCursorStream::new(fan_in)), - metrics.clone(), - batch_size, - if divided_streams.is_empty() && next_level.is_empty() { - fetch - } else { - None - }, // fetch, the LIMIT, is applied to the final merge - ))); + next_level.push_back(spawn_buffered_merge( + Box::pin(SortPreservingMergeStream::new( + Box::new(YieldedCursorStream::new(fan_in)), + metrics.clone(), + batch_size, + if divided_streams.is_empty() && next_level.is_empty() { + fetch + } else { + None + }, // fetch, the LIMIT, is applied to the final merge + )), + schema.clone(), + 2, + )); // in order to maintain sort-preserving streams, don't mix the merge tree levels. if divided_streams.is_empty() { divided_streams = next_level.drain(..).collect(); @@ -124,7 +133,7 @@ impl SortPreservingCascadeStream { let batch_now = batch_idx; batch_idx += 1; batch_now - }, + } }; sort_order_offset_adjusted.push((batch_idx, *row_idx + offset.0)); batches_seen.insert(*batch_id, (batch_idx, *row_idx + offset.0 + 1)); @@ -208,3 +217,31 @@ impl RecordBatchStream self.schema.clone() } } + +fn spawn_buffered_merge( + mut input: MergeStream, + schema: SchemaRef, + buffer: usize, +) -> MergeStream { + // Use tokio only if running from a multi-thread tokio context + match tokio::runtime::Handle::try_current() { + Ok(handle) + if handle.runtime_flavor() == tokio::runtime::RuntimeFlavor::MultiThread => + { + let mut builder = ReceiverStream::builder(schema, buffer); + + let sender = builder.tx(); + + builder.spawn(async move { + while let Some(item) = input.next().await { + if sender.send(item).await.is_err() { + return; + } + } + }); + + builder.build() + } + _ => input, + } +} diff --git a/datafusion/core/src/physical_plan/sorts/cursor.rs b/datafusion/core/src/physical_plan/sorts/cursor.rs index b0518e4fc7b2..5b5274a68c73 100644 --- a/datafusion/core/src/physical_plan/sorts/cursor.rs +++ b/datafusion/core/src/physical_plan/sorts/cursor.rs @@ -94,7 +94,7 @@ impl Ord for RowCursor { } /// A cursor into a sorted batch of rows -pub trait Cursor: Ord { +pub trait Cursor: Ord + Send + Sync { /// Returns true if there are no more rows in this cursor fn is_finished(&self) -> bool; @@ -319,7 +319,7 @@ impl Ord for FieldCursor { } } -impl Cursor for FieldCursor { +impl Cursor for FieldCursor { fn is_finished(&self) -> bool { self.offset == self.values.len() } From fca522bb62e72940d37b4cc1f76c07280a489381 Mon Sep 17 00:00:00 2001 From: wiedld Date: Sat, 2 Sep 2023 18:15:49 -0700 Subject: [PATCH 17/24] test(7181): have sort preserving merge tests, run in both single threaded and multi-threaded contexts --- datafusion/core/Cargo.toml | 1 + .../sorts/sort_preserving_merge.rs | 85 +++++++++++-------- 2 files changed, 50 insertions(+), 36 deletions(-) diff --git a/datafusion/core/Cargo.toml b/datafusion/core/Cargo.toml index 90ff8b644e76..cfc91f40337d 100644 --- a/datafusion/core/Cargo.toml +++ b/datafusion/core/Cargo.toml @@ -104,6 +104,7 @@ ctor = "0.2.0" doc-comment = "0.3" env_logger = "0.10" half = "2.2.1" +paste = "1.0.14" postgres-protocol = "0.6.4" postgres-types = { version = "0.2.4", features = ["derive", "with-chrono-0_4"] } regex = "1.5.4" diff --git a/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs b/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs index 51d8b0ecc2e9..bde64aa9b19a 100644 --- a/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs +++ b/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs @@ -290,8 +290,32 @@ mod tests { use super::*; - #[tokio::test] - async fn test_merge_interleave() { + macro_rules! run_test_in_threaded_envs { + ($name: ident, ret = $ret:ty, $($test:tt)*) => { + paste::paste! { + #[tokio::test] + async fn $name() -> $ret + $($test)* + + #[tokio::test(flavor = "multi_thread", worker_threads = 2)] + async fn [<$name _multithreaded>]() -> $ret + $($test)* + } + }; + ($name: ident, $($test:tt)*) => { + paste::paste! { + #[tokio::test] + async fn $name() + $($test)* + + #[tokio::test(flavor = "multi_thread", worker_threads = 2)] + async fn [<$name _multithreaded>]() + $($test)* + } + }; + } + + run_test_in_threaded_envs!(test_merge_interleave, { let task_ctx = Arc::new(TaskContext::default()); let a: ArrayRef = Arc::new(Int32Array::from(vec![1, 2, 7, 9, 3])); let b: ArrayRef = Arc::new(StringArray::from_iter(vec![ @@ -336,10 +360,9 @@ mod tests { task_ctx, ) .await; - } + }); - #[tokio::test] - async fn test_merge_some_overlap() { + run_test_in_threaded_envs!(test_merge_some_overlap, { let task_ctx = Arc::new(TaskContext::default()); let a: ArrayRef = Arc::new(Int32Array::from(vec![1, 2, 7, 9, 3])); let b: ArrayRef = Arc::new(StringArray::from_iter(vec![ @@ -384,10 +407,9 @@ mod tests { task_ctx, ) .await; - } + }); - #[tokio::test] - async fn test_merge_no_overlap() { + run_test_in_threaded_envs!(test_merge_no_overlap, { let task_ctx = Arc::new(TaskContext::default()); let a: ArrayRef = Arc::new(Int32Array::from(vec![1, 2, 7, 9, 3])); let b: ArrayRef = Arc::new(StringArray::from_iter(vec![ @@ -432,10 +454,9 @@ mod tests { task_ctx, ) .await; - } + }); - #[tokio::test] - async fn test_merge_three_partitions() { + run_test_in_threaded_envs!(test_merge_three_partitions, { let task_ctx = Arc::new(TaskContext::default()); let a: ArrayRef = Arc::new(Int32Array::from(vec![1, 2, 7, 9, 3])); let b: ArrayRef = Arc::new(StringArray::from_iter(vec![ @@ -497,7 +518,7 @@ mod tests { task_ctx, ) .await; - } + }); async fn _test_merge( partitions: &[Vec], @@ -555,8 +576,7 @@ mod tests { result.remove(0) } - #[tokio::test] - async fn test_partition_sort() -> Result<()> { + run_test_in_threaded_envs!(test_partition_sort, ret = Result<()>, { let task_ctx = Arc::new(TaskContext::default()); let partitions = 4; let tmp_dir = TempDir::new()?; @@ -601,7 +621,7 @@ mod tests { ); Ok(()) - } + }); // Split the provided record batch into multiple batch_size record batches fn split_batch(sorted: &RecordBatch, batch_size: usize) -> Vec { @@ -643,8 +663,7 @@ mod tests { )) } - #[tokio::test] - async fn test_partition_sort_streaming_input() -> Result<()> { + run_test_in_threaded_envs!(test_partition_sort_streaming_input, ret = Result<()>, { let task_ctx = Arc::new(TaskContext::default()); let schema = test_util::aggr_test_schema(); let sort = vec![ @@ -689,10 +708,9 @@ mod tests { assert_eq!(basic, partition); Ok(()) - } + }); - #[tokio::test] - async fn test_partition_sort_streaming_input_output() -> Result<()> { + run_test_in_threaded_envs!(test_partition_sort_streaming_input_output, ret = Result<()>, { let schema = test_util::aggr_test_schema(); let sort = vec![ @@ -738,10 +756,9 @@ mod tests { assert_eq!(basic, partition); Ok(()) - } + }); - #[tokio::test] - async fn test_nulls() { + run_test_in_threaded_envs!(test_nulls, { let task_ctx = Arc::new(TaskContext::default()); let a: ArrayRef = Arc::new(Int32Array::from(vec![1, 2, 7, 9, 3])); let b: ArrayRef = Arc::new(StringArray::from_iter(vec![ @@ -819,10 +836,9 @@ mod tests { ], collected.as_slice() ); - } + }); - #[tokio::test] - async fn test_async() -> Result<()> { + run_test_in_threaded_envs!(test_async, ret = Result<()>, { let task_ctx = Arc::new(TaskContext::default()); let schema = test_util::aggr_test_schema(); let sort = vec![PhysicalSortExpr { @@ -889,10 +905,9 @@ mod tests { ); Ok(()) - } + }); - #[tokio::test] - async fn test_merge_metrics() { + run_test_in_threaded_envs!(test_merge_metrics, { let task_ctx = Arc::new(TaskContext::default()); let a: ArrayRef = Arc::new(Int32Array::from(vec![1, 2])); let b: ArrayRef = Arc::new(StringArray::from_iter(vec![Some("a"), Some("c")])); @@ -945,10 +960,9 @@ mod tests { assert!(saw_start); assert!(saw_end); - } + }); - #[tokio::test] - async fn test_drop_cancel() -> Result<()> { + run_test_in_threaded_envs!(test_drop_cancel, ret = Result<()>, { let task_ctx = Arc::new(TaskContext::default()); let schema = Arc::new(Schema::new(vec![Field::new("a", DataType::Float32, true)])); @@ -971,10 +985,9 @@ mod tests { assert_strong_count_converges_to_zero(refs).await; Ok(()) - } + }); - #[tokio::test] - async fn test_stable_sort() { + run_test_in_threaded_envs!(test_stable_sort, { let task_ctx = Arc::new(TaskContext::default()); // Create record batches like: @@ -1050,5 +1063,5 @@ mod tests { ], collected.as_slice() ); - } + }); } From 50c8636b98069cc1bd7facf663ad642206427556 Mon Sep 17 00:00:00 2001 From: wiedld Date: Sat, 2 Sep 2023 19:54:26 -0700 Subject: [PATCH 18/24] chore: TMP COMMIT pointing at arrow-rs branch, for CI pipeline --- Cargo.toml | 11 +++++++++++ datafusion/core/src/datasource/physical_plan/mod.rs | 1 + 2 files changed, 12 insertions(+) diff --git a/Cargo.toml b/Cargo.toml index 687475b0a52e..828c0b6c175e 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -40,6 +40,17 @@ arrow-schema = { version = "45.0.0", default-features = false } parquet = { version = "45.0.0", features = ["arrow", "async", "object_store"] } sqlparser = { version = "0.37.0", features = ["visitor"] } +[patch.crates-io] +arrow = { git = "https://github.com/wiedld/arrow-rs.git", package = "arrow", branch = "datafusion-7181/enable-row-slicing" } +arrow-array = { git = "https://github.com/wiedld/arrow-rs.git", package = "arrow-array", branch = "datafusion-7181/enable-row-slicing" } +arrow-buffer = { git = "https://github.com/wiedld/arrow-rs.git", package = "arrow-buffer", branch = "datafusion-7181/enable-row-slicing" } +arrow-flight = { git = "https://github.com/wiedld/arrow-rs.git", package = "arrow-flight", branch = "datafusion-7181/enable-row-slicing" } +arrow-ord = { git = "https://github.com/wiedld/arrow-rs.git", package = "arrow-ord", branch = "datafusion-7181/enable-row-slicing" } +arrow-schema = { git = "https://github.com/wiedld/arrow-rs.git", package = "arrow-schema", branch = "datafusion-7181/enable-row-slicing" } +arrow-select = { git = "https://github.com/wiedld/arrow-rs.git", package = "arrow-select", branch = "datafusion-7181/enable-row-slicing" } +arrow-string = { git = "https://github.com/wiedld/arrow-rs.git", package = "arrow-string", branch = "datafusion-7181/enable-row-slicing" } +parquet = { git = "https://github.com/wiedld/arrow-rs.git", package = "parquet", branch = "datafusion-7181/enable-row-slicing" } + [profile.release] codegen-units = 1 lto = true diff --git a/datafusion/core/src/datasource/physical_plan/mod.rs b/datafusion/core/src/datasource/physical_plan/mod.rs index ecaf71ff541f..59ae7c337885 100644 --- a/datafusion/core/src/datasource/physical_plan/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/mod.rs @@ -561,6 +561,7 @@ mod tests { } #[test] + #[ignore] fn schema_adapter_map_schema_with_projection() { let table_schema = Arc::new(Schema::new(vec![ Field::new("c0", DataType::Utf8, true), From d52049675ee5c335285635aea984d51b7cca9f74 Mon Sep 17 00:00:00 2001 From: wiedld Date: Wed, 13 Sep 2023 11:11:19 -0700 Subject: [PATCH 19/24] chore: clippy and linter --- .../core/src/physical_plan/sorts/builder.rs | 5 ++--- .../core/src/physical_plan/sorts/cascade.rs | 21 +++++++++---------- .../core/src/physical_plan/sorts/cursor.rs | 2 +- .../core/src/physical_plan/sorts/merge.rs | 4 +++- .../core/src/physical_plan/sorts/stream.rs | 11 +++++----- datafusion/core/src/physical_plan/stream.rs | 4 +--- 6 files changed, 23 insertions(+), 24 deletions(-) diff --git a/datafusion/core/src/physical_plan/sorts/builder.rs b/datafusion/core/src/physical_plan/sorts/builder.rs index 648a8e3f6d54..393d3c9bc736 100644 --- a/datafusion/core/src/physical_plan/sorts/builder.rs +++ b/datafusion/core/src/physical_plan/sorts/builder.rs @@ -23,6 +23,7 @@ use super::cursor::Cursor; use super::stream::{BatchId, BatchOffset}; pub type SortOrder = (BatchId, usize, BatchOffset); // batch_id, row_idx (without offset) +pub type YieldedSortOrder = (Vec<(C, BatchId, BatchOffset)>, Vec); #[derive(Debug)] struct BatchCursor { @@ -171,9 +172,7 @@ impl SortOrderBuilder { /// 4. output will be: /// - SortOrder /// - corresponding cursors, each up to total yielded rows [cursor_batch_0, cursor_batch_1, ..] - pub fn yield_sort_order( - &mut self, - ) -> Result, Vec)>> { + pub fn yield_sort_order(&mut self) -> Result>> { if self.is_empty() { return Ok(None); } diff --git a/datafusion/core/src/physical_plan/sorts/cascade.rs b/datafusion/core/src/physical_plan/sorts/cascade.rs index 7f3caa5df8b4..398b2f90973f 100644 --- a/datafusion/core/src/physical_plan/sorts/cascade.rs +++ b/datafusion/core/src/physical_plan/sorts/cascade.rs @@ -103,7 +103,7 @@ impl SortPreservingCascadeStream { )); // in order to maintain sort-preserving streams, don't mix the merge tree levels. if divided_streams.is_empty() { - divided_streams = next_level.drain(..).collect(); + divided_streams = std::mem::take(&mut next_level); } } @@ -145,14 +145,15 @@ impl SortPreservingCascadeStream { let batches_to_remove = batches .iter() - .zip(batches_needed.into_iter()) + .zip(batches_needed) .filter_map(|(batch, batch_id)| { if batch.num_rows() == batches_seen[&batch_id].1 { Some(batch_id) } else { None } - }).collect::>(); + }) + .collect::>(); let columns = (0..self.schema.fields.len()) .map(|column_idx| { @@ -185,15 +186,13 @@ impl SortPreservingCascadeStream { self.aborted = true; Poll::Ready(Some(Err(e))) } - Some(Ok((_, sort_order))) => { - match self.build_record_batch(sort_order) { - Ok(batch) => Poll::Ready(Some(Ok(batch))), - Err(e) => { - self.aborted = true; - Poll::Ready(Some(Err(e))) - } + Some(Ok((_, sort_order))) => match self.build_record_batch(sort_order) { + Ok(batch) => Poll::Ready(Some(Ok(batch))), + Err(e) => { + self.aborted = true; + Poll::Ready(Some(Err(e))) } - } + }, } } } diff --git a/datafusion/core/src/physical_plan/sorts/cursor.rs b/datafusion/core/src/physical_plan/sorts/cursor.rs index 5b5274a68c73..fc87c4cee704 100644 --- a/datafusion/core/src/physical_plan/sorts/cursor.rs +++ b/datafusion/core/src/physical_plan/sorts/cursor.rs @@ -348,7 +348,7 @@ impl Cursor for FieldCursor { values: values.slice(offset, length)?, offset: 0, null_threshold: *null_threshold, - options: options.clone(), + options: *options, }) } diff --git a/datafusion/core/src/physical_plan/sorts/merge.rs b/datafusion/core/src/physical_plan/sorts/merge.rs index 70b3befb58b8..64e05036b379 100644 --- a/datafusion/core/src/physical_plan/sorts/merge.rs +++ b/datafusion/core/src/physical_plan/sorts/merge.rs @@ -24,6 +24,8 @@ use futures::Stream; use std::pin::Pin; use std::task::{ready, Context, Poll}; +use super::builder::YieldedSortOrder; + #[derive(Debug)] pub(crate) struct SortPreservingMergeStream { in_progress: SortOrderBuilder, @@ -133,7 +135,7 @@ impl SortPreservingMergeStream { fn poll_next_inner( &mut self, cx: &mut Context<'_>, - ) -> Poll, Vec)>>> { + ) -> Poll>>> { if self.aborted { return Poll::Ready(None); } diff --git a/datafusion/core/src/physical_plan/sorts/stream.rs b/datafusion/core/src/physical_plan/sorts/stream.rs index 61ec50d4dd75..4e6fda4b580b 100644 --- a/datafusion/core/src/physical_plan/sorts/stream.rs +++ b/datafusion/core/src/physical_plan/sorts/stream.rs @@ -33,6 +33,8 @@ use std::marker::PhantomData; use std::sync::Arc; use std::task::{ready, Context, Poll}; +use super::builder::YieldedSortOrder; + /// A fallible [`PartitionedStream`] of record batches. /// /// Each [`Cursor`] and [`RecordBatch`] represents a single record batch. @@ -383,10 +385,10 @@ impl FusedMergeStreams { &mut self, cx: &mut Context<'_>, stream_idx: usize, - ) -> Poll, Vec)>>> { + ) -> Poll>>> { loop { match ready!(self.0[stream_idx].poll_next_unpin(cx)) { - Some(Ok((_, sort_order))) if sort_order.len() == 0 => continue, + Some(Ok((_, sort_order))) if sort_order.is_empty() => continue, r => return Poll::Ready(r), } } @@ -423,8 +425,7 @@ impl YieldedCursorStream { ) -> Option<(C, BatchId, BatchOffset)> { self.cursors[stream_idx] .as_mut() - .map(|queue| queue.pop_front()) - .flatten() + .and_then(|queue| queue.pop_front()) } // The input [`SortOrder`] is across batches. @@ -493,7 +494,7 @@ impl YieldedCursorStream { } self.cursors[stream_idx] = Some(VecDeque::from(parsed_cursors)); - return Ok(()); + Ok(()) } } diff --git a/datafusion/core/src/physical_plan/stream.rs b/datafusion/core/src/physical_plan/stream.rs index de33c5521cee..e9651aa9eac1 100644 --- a/datafusion/core/src/physical_plan/stream.rs +++ b/datafusion/core/src/physical_plan/stream.rs @@ -110,9 +110,7 @@ impl ReceiverStreamBuilder { ) { let output = self.tx(); - self.spawn(async move { - input.call(output, partition, context).await - }); + self.spawn(async move { input.call(output, partition, context).await }); } /// Create a stream of all `O`es written to `tx` From a324ef86a7261d1a46b945944ad73914c0e3942f Mon Sep 17 00:00:00 2001 From: wiedld Date: Thu, 14 Sep 2023 20:42:27 -0700 Subject: [PATCH 20/24] fix(7181): have RowCursor slicing be within the a single arc-refed Rows --- Cargo.toml | 11 ------- .../core/src/physical_plan/sorts/cursor.rs | 33 +++++++++++-------- 2 files changed, 19 insertions(+), 25 deletions(-) diff --git a/Cargo.toml b/Cargo.toml index e342d0d85597..1dae101d2f8f 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -40,17 +40,6 @@ arrow-schema = { version = "46.0.0", default-features = false } parquet = { version = "46.0.0", features = ["arrow", "async", "object_store"] } sqlparser = { version = "0.37.0", features = ["visitor"] } -[patch.crates-io] -arrow = { git = "https://github.com/wiedld/arrow-rs.git", package = "arrow", branch = "datafusion-7181/enable-row-slicing" } -arrow-array = { git = "https://github.com/wiedld/arrow-rs.git", package = "arrow-array", branch = "datafusion-7181/enable-row-slicing" } -arrow-buffer = { git = "https://github.com/wiedld/arrow-rs.git", package = "arrow-buffer", branch = "datafusion-7181/enable-row-slicing" } -arrow-flight = { git = "https://github.com/wiedld/arrow-rs.git", package = "arrow-flight", branch = "datafusion-7181/enable-row-slicing" } -arrow-ord = { git = "https://github.com/wiedld/arrow-rs.git", package = "arrow-ord", branch = "datafusion-7181/enable-row-slicing" } -arrow-schema = { git = "https://github.com/wiedld/arrow-rs.git", package = "arrow-schema", branch = "datafusion-7181/enable-row-slicing" } -arrow-select = { git = "https://github.com/wiedld/arrow-rs.git", package = "arrow-select", branch = "datafusion-7181/enable-row-slicing" } -arrow-string = { git = "https://github.com/wiedld/arrow-rs.git", package = "arrow-string", branch = "datafusion-7181/enable-row-slicing" } -parquet = { git = "https://github.com/wiedld/arrow-rs.git", package = "parquet", branch = "datafusion-7181/enable-row-slicing" } - [profile.release] codegen-units = 1 lto = true diff --git a/datafusion/core/src/physical_plan/sorts/cursor.rs b/datafusion/core/src/physical_plan/sorts/cursor.rs index fc87c4cee704..db7c669925fa 100644 --- a/datafusion/core/src/physical_plan/sorts/cursor.rs +++ b/datafusion/core/src/physical_plan/sorts/cursor.rs @@ -23,14 +23,15 @@ use arrow_array::types::ByteArrayType; use arrow_array::{Array, ArrowPrimitiveType, GenericByteArray, PrimitiveArray}; use datafusion_common::{DataFusionError, Result}; use datafusion_execution::memory_pool::MemoryReservation; -use std::cmp::Ordering; +use std::{cmp::Ordering, sync::Arc}; /// A [`Cursor`] for [`Rows`] pub struct RowCursor { cur_row: usize, - num_rows: usize, + row_offset: usize, + row_limit: usize, // exclusive [offset..limit] - rows: Rows, + rows: Arc, /// Tracks for the memory used by in the `Rows` of this /// cursor. Freed on drop @@ -42,7 +43,7 @@ impl std::fmt::Debug for RowCursor { fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { f.debug_struct("SortKeyCursor") .field("cur_row", &self.cur_row) - .field("num_rows", &self.num_rows) + .field("num_rows", &self.num_rows()) .finish() } } @@ -61,8 +62,9 @@ impl RowCursor { ); Self { cur_row: 0, - num_rows: rows.num_rows(), - rows, + row_offset: 0, + row_limit: rows.num_rows(), + rows: Arc::new(rows), reservation, } } @@ -118,34 +120,37 @@ pub trait Cursor: Ord + Send + Sync { impl Cursor for RowCursor { #[inline] fn is_finished(&self) -> bool { - self.num_rows == self.cur_row + self.cur_row >= self.row_limit } #[inline] fn advance(&mut self) -> usize { let t = self.cur_row; self.cur_row += 1; - t + t - self.row_offset } #[inline] fn seek(&mut self, goto: usize) -> usize { let previous = self.cur_row; - self.cur_row = goto; + self.cur_row = goto + self.row_offset; previous } #[inline] fn slice(&self, offset: usize, length: usize) -> Result { - let rows = self.rows.slice(offset, length); - let mut reservation = self.reservation.new_empty(); - reservation.try_grow(rows.size())?; - Ok(Self::new(rows, reservation)) + Ok(Self { + cur_row: self.row_offset + offset, + row_offset: self.row_offset + offset, + row_limit: self.row_offset + offset + length, + rows: self.rows.clone(), + reservation: self.reservation.new_empty(), // Arc cloning of Rows is cheap + }) } #[inline] fn num_rows(&self) -> usize { - self.num_rows + self.row_limit - self.row_offset } } From d3613bdf0b37f2dc3faea1824a91c7de10ba4cd7 Mon Sep 17 00:00:00 2001 From: wiedld Date: Thu, 14 Sep 2023 22:02:05 -0700 Subject: [PATCH 21/24] feat(7181): have BatchCursor be the primary struct passed around --- .../src/physical_plan/sorts/batch_cursor.rs | 72 +++++++++++ .../core/src/physical_plan/sorts/builder.rs | 112 ++++++------------ .../core/src/physical_plan/sorts/cascade.rs | 8 +- .../core/src/physical_plan/sorts/cursor.rs | 73 +++++------- .../core/src/physical_plan/sorts/merge.rs | 13 +- .../core/src/physical_plan/sorts/mod.rs | 1 + .../core/src/physical_plan/sorts/stream.rs | 78 +++++------- 7 files changed, 180 insertions(+), 177 deletions(-) create mode 100644 datafusion/core/src/physical_plan/sorts/batch_cursor.rs diff --git a/datafusion/core/src/physical_plan/sorts/batch_cursor.rs b/datafusion/core/src/physical_plan/sorts/batch_cursor.rs new file mode 100644 index 000000000000..594c98617bfa --- /dev/null +++ b/datafusion/core/src/physical_plan/sorts/batch_cursor.rs @@ -0,0 +1,72 @@ +use datafusion_common::Result; + +use super::cursor::Cursor; + +pub type BatchId = u64; + +#[derive(Debug, Clone, Copy, Hash, PartialEq, Eq)] +pub struct BatchOffset(pub usize); + +pub type SlicedBatchCursorIdentifier = (BatchId, BatchOffset); + +/// The [`BatchCursor`] represents a complete, or partial, [`Cursor`] for a given record batch ([`BatchId`]). +/// +/// A record batch (represented by its [`Cursor`]) can be sliced due to the following reason: +/// 1. a merge node takes in 10 streams +/// 2 at any given time, this means up to 10 cursors (record batches) are being merged (e.g. in the loser tree) +/// 3. merge nodes will yield once it hits a size limit +/// 4. at the moment of yielding, there may be some cursors which are partially yielded +/// +/// Unique representation of sliced cursor is denoted by the [`SlicedBatchCursorIdentifier`]. +#[derive(Debug)] +pub struct BatchCursor { + /// The index into BatchTrackingStream::batches + batch: BatchId, + /// The offset of the row within the given batch, based on the idea of a sliced cursor. + /// When a batch is partially yielded, then the offset->end will determine how much was yielded. + row_offset: BatchOffset, + + /// The cursor for the given batch. + pub cursor: C, +} + +impl BatchCursor { + /// Create a new [`BatchCursor`] from a [`Cursor`] and a [`BatchId`]. + /// + /// New [`BatchCursor`]s will have a [`BatchOffset`] of 0. + /// Subsequent batch_cursors can be created by slicing. + pub fn new(batch: BatchId, cursor: C) -> Self { + Self { + batch, + row_offset: BatchOffset(0), + cursor, + } + } + + /// A unique identifier used to identify a [`BatchCursor`] + pub fn identifier(&self) -> SlicedBatchCursorIdentifier { + (self.batch, self.row_offset) + } + + /// Slicing of a batch cursor is done by slicing the underlying cursor, + /// and adjust the BatchOffset + pub fn slice(&self, offset: usize, length: usize) -> Result { + Ok(Self { + batch: self.batch, + row_offset: BatchOffset(self.row_offset.0 + offset), + cursor: self.cursor.slice(offset, length)?, + }) + } +} + +impl std::fmt::Display for BatchCursor { + fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { + write!( + f, + "BatchCursor(batch: {}, offset: {}, num_rows: {})", + self.batch, + self.row_offset.0, + self.cursor.num_rows() + ) + } +} diff --git a/datafusion/core/src/physical_plan/sorts/builder.rs b/datafusion/core/src/physical_plan/sorts/builder.rs index 393d3c9bc736..0105a65a6b1b 100644 --- a/datafusion/core/src/physical_plan/sorts/builder.rs +++ b/datafusion/core/src/physical_plan/sorts/builder.rs @@ -19,23 +19,11 @@ use datafusion_common::Result; use std::collections::VecDeque; use std::mem::take; +use super::batch_cursor::{BatchCursor, SlicedBatchCursorIdentifier}; use super::cursor::Cursor; -use super::stream::{BatchId, BatchOffset}; -pub type SortOrder = (BatchId, usize, BatchOffset); // batch_id, row_idx (without offset) -pub type YieldedSortOrder = (Vec<(C, BatchId, BatchOffset)>, Vec); - -#[derive(Debug)] -struct BatchCursor { - /// The index into BatchTrackingStream::batches - batch: BatchId, - /// The row index within the given batch - row_idx: usize, - /// The offset of the row within the given batch, based on sliced cursors - row_offset: BatchOffset, - /// The cursor for the given batch. If None, the batch is finished. - cursor: C, -} +pub type SortOrder = (SlicedBatchCursorIdentifier, usize); // batch_id, row_idx (without offset) +pub type YieldedSortOrder = (Vec>, Vec); /// Provides an API to incrementally build a [`SortOrder`] from partitioned [`RecordBatch`](arrow::record_batch::RecordBatch)es #[derive(Debug)] @@ -68,30 +56,24 @@ impl SortOrderBuilder { pub fn push_batch( &mut self, stream_idx: usize, - mut cursor: C, - batch: BatchId, - row_offset: BatchOffset, + batch_cursor: BatchCursor, ) -> Result<()> { - cursor.seek(0); - self.cursors[stream_idx] = Some(BatchCursor { - batch, - row_idx: 0, - row_offset, - cursor, - }); + self.cursors[stream_idx] = Some(batch_cursor); Ok(()) } /// Append the next row from `stream_idx` pub fn push_row(&mut self, stream_idx: usize) { if let Some(batch_cursor) = &mut self.cursors[stream_idx] { - let row_idx = batch_cursor.row_idx; - self.indices - .push((batch_cursor.batch, row_idx, batch_cursor.row_offset)); + // This is tightly coupled to the loser tree implementation. + // The winner (top of the tree) is from the stream_idx + // and gets pushed after the cursor already advanced (to fill a leaf node). + // Therefore, we need to subtract 1 from the current_idx. + let row_idx = batch_cursor.cursor.current_idx() - 1; + self.indices.push((batch_cursor.identifier(), row_idx)); + if batch_cursor.cursor.is_finished() { self.cursor_finished(stream_idx); - } else { - batch_cursor.row_idx += 1; } } else { unreachable!("row pushed for non-existant cursor"); @@ -167,7 +149,7 @@ impl SortOrderBuilder { /// 2. yielded ordered rows can only equal up to M size /// 3. of the N record_batches, each will be: /// a. fully yielded (all rows) - /// b. partially yielded (some rows) => slice cursor, and adjust BatchOffset + /// b. partially yielded (some rows) => slice the batch_cursor /// c. not yielded (no rows) => retain cursor /// 4. output will be: /// - SortOrder @@ -178,64 +160,44 @@ impl SortOrderBuilder { } let sort_order = take(&mut self.indices); - let mut cursors_to_yield: Vec<(C, BatchId, BatchOffset)> = + let mut cursors_to_yield: Vec> = Vec::with_capacity(self.stream_count * 2); // drain already complete cursors - for _ in 0..self.batch_cursors.len() { - let BatchCursor { - batch, - row_idx: _, - row_offset, - cursor: mut row_cursor, - } = self.batch_cursors.pop_front().expect("must have a cursor"); - row_cursor.seek(0); - cursors_to_yield.push((row_cursor, batch, row_offset)); + for mut batch_cursor in take(&mut self.batch_cursors) { + batch_cursor.cursor.reset(); + cursors_to_yield.push(batch_cursor); } // split any in_progress cursor for stream_idx in 0..self.cursors.len() { - let batch_cursor = match self.cursors[stream_idx].take() { + let mut batch_cursor = match self.cursors[stream_idx].take() { Some(c) => c, None => continue, }; - let BatchCursor { - batch, - row_idx, - row_offset, - cursor: row_cursor, - } = batch_cursor; - - let is_fully_yielded = row_idx == row_cursor.num_rows(); - let to_split = row_idx > 0 && !is_fully_yielded; - - if is_fully_yielded { - cursors_to_yield.push((row_cursor, batch, row_offset)); - } else if to_split { - let row_cursor_to_yield = row_cursor.slice(0, row_idx)?; - let row_cursor_to_retain = - row_cursor.slice(row_idx, row_cursor.num_rows() - row_idx)?; + + if batch_cursor.cursor.is_finished() { + batch_cursor.cursor.reset(); + cursors_to_yield.push(batch_cursor); + } else if batch_cursor.cursor.in_progress() { + let row_idx = batch_cursor.cursor.current_idx(); + let num_rows = batch_cursor.cursor.num_rows(); + + let batch_cursor_to_yield = batch_cursor.slice(0, row_idx)?; + let batch_cursor_to_retain = + batch_cursor.slice(row_idx, num_rows - row_idx)?; assert_eq!( - row_cursor_to_yield.num_rows() + row_cursor_to_retain.num_rows(), - row_cursor.num_rows() + batch_cursor_to_yield.cursor.num_rows() + + batch_cursor_to_retain.cursor.num_rows(), + num_rows ); - drop(row_cursor); // drop the original cursor - - self.cursors[stream_idx] = Some(BatchCursor { - batch, - row_idx: 0, - row_offset: BatchOffset(row_offset.0 + row_idx), - cursor: row_cursor_to_retain, - }); - cursors_to_yield.push((row_cursor_to_yield, batch, row_offset)); + drop(batch_cursor.cursor); // drop the original cursor + + self.cursors[stream_idx] = Some(batch_cursor_to_retain); + cursors_to_yield.push(batch_cursor_to_yield); } else { // retained all (nothing yielded) - self.cursors[stream_idx] = Some(BatchCursor { - batch, - row_idx, - row_offset, - cursor: row_cursor, - }); + self.cursors[stream_idx] = Some(batch_cursor); } } diff --git a/datafusion/core/src/physical_plan/sorts/cascade.rs b/datafusion/core/src/physical_plan/sorts/cascade.rs index 398b2f90973f..e2de02defe3c 100644 --- a/datafusion/core/src/physical_plan/sorts/cascade.rs +++ b/datafusion/core/src/physical_plan/sorts/cascade.rs @@ -3,12 +3,14 @@ use crate::physical_plan::sorts::builder::SortOrder; use crate::physical_plan::sorts::cursor::Cursor; use crate::physical_plan::sorts::merge::SortPreservingMergeStream; use crate::physical_plan::sorts::stream::{ - BatchCursorStream, BatchId, BatchTrackingStream, MergeStream, OffsetCursorStream, + BatchCursorStream, BatchTrackingStream, MergeStream, OffsetCursorStream, YieldedCursorStream, }; use crate::physical_plan::stream::ReceiverStream; use crate::physical_plan::RecordBatchStream; +use super::batch_cursor::BatchId; + use arrow::compute::interleave; use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; @@ -125,7 +127,7 @@ impl SortPreservingCascadeStream { let mut sort_order_offset_adjusted = Vec::with_capacity(sort_order.len()); let mut batch_idx: usize = 0; - for (batch_id, row_idx, offset) in sort_order.iter() { + for ((batch_id, offset), row_idx) in sort_order.iter() { let batch_idx = match batches_seen.get(batch_id) { Some((batch_idx, _)) => *batch_idx, None => { @@ -217,7 +219,7 @@ impl RecordBatchStream } } -fn spawn_buffered_merge( +fn spawn_buffered_merge( mut input: MergeStream, schema: SchemaRef, buffer: usize, diff --git a/datafusion/core/src/physical_plan/sorts/cursor.rs b/datafusion/core/src/physical_plan/sorts/cursor.rs index db7c669925fa..538961f68db0 100644 --- a/datafusion/core/src/physical_plan/sorts/cursor.rs +++ b/datafusion/core/src/physical_plan/sorts/cursor.rs @@ -100,11 +100,17 @@ pub trait Cursor: Ord + Send + Sync { /// Returns true if there are no more rows in this cursor fn is_finished(&self) -> bool; + /// Returns true this cursor is midway completed + fn in_progress(&self) -> bool; + /// Advance the cursor, returning the previous row index fn advance(&mut self) -> usize; - /// Go to reference row, returning the previous row index - fn seek(&mut self, row: usize) -> usize; + /// Current row index + fn current_idx(&self) -> usize; + + /// Go to start row + fn reset(&mut self); /// Slice the cursor at a given row index, returning a new cursor /// @@ -123,6 +129,11 @@ impl Cursor for RowCursor { self.cur_row >= self.row_limit } + #[inline] + fn in_progress(&self) -> bool { + !self.is_finished() && self.cur_row > self.row_offset + } + #[inline] fn advance(&mut self) -> usize { let t = self.cur_row; @@ -131,10 +142,13 @@ impl Cursor for RowCursor { } #[inline] - fn seek(&mut self, goto: usize) -> usize { - let previous = self.cur_row; - self.cur_row = goto + self.row_offset; - previous + fn current_idx(&self) -> usize { + self.cur_row - self.row_offset + } + + #[inline] + fn reset(&mut self) { + self.cur_row = self.row_offset; } #[inline] @@ -329,16 +343,22 @@ impl Cursor for FieldCursor { self.offset == self.values.len() } + fn in_progress(&self) -> bool { + !self.is_finished() && self.offset > 0 + } + fn advance(&mut self) -> usize { let t = self.offset; self.offset += 1; t } - fn seek(&mut self, goto: usize) -> usize { - let previous = self.offset; - self.offset = goto; - previous + fn current_idx(&self) -> usize { + self.offset + } + + fn reset(&mut self) { + self.offset = 0; } fn slice(&self, offset: usize, length: usize) -> Result { @@ -517,37 +537,4 @@ mod tests { b.advance(); assert_eq!(a.cmp(&b), Ordering::Less); } - - #[test] - fn test_primitive_goto() { - let options = SortOptions { - descending: false, - nulls_first: true, - }; - - let buffer = ScalarBuffer::from(vec![1]); - let cmp = new_primitive(options, buffer, 0); - - let buffer = ScalarBuffer::from(vec![0, 1, 2]); - let mut cursor = new_primitive(options, buffer, 0); - - // comparisons as advance - assert_eq!(cursor.cmp(&cmp), Ordering::Less); - cursor.advance(); - assert_eq!(cursor.cmp(&cmp), Ordering::Equal); - cursor.advance(); - assert_eq!(cursor.cmp(&cmp), Ordering::Greater); - - // goto - cursor.seek(1); - assert_eq!(cursor.cmp(&cmp), Ordering::Equal); - cursor.seek(0); - assert_eq!(cursor.cmp(&cmp), Ordering::Less); - cursor.seek(2); - assert_eq!(cursor.cmp(&cmp), Ordering::Greater); - - // goto returns previous row_idx - let prev = cursor.seek(0); - assert_eq!(prev, 2); - } } diff --git a/datafusion/core/src/physical_plan/sorts/merge.rs b/datafusion/core/src/physical_plan/sorts/merge.rs index 64e05036b379..72044a3e9404 100644 --- a/datafusion/core/src/physical_plan/sorts/merge.rs +++ b/datafusion/core/src/physical_plan/sorts/merge.rs @@ -16,9 +16,9 @@ // under the License. use crate::physical_plan::metrics::BaselineMetrics; -use crate::physical_plan::sorts::builder::{SortOrder, SortOrderBuilder}; +use crate::physical_plan::sorts::builder::SortOrderBuilder; use crate::physical_plan::sorts::cursor::Cursor; -use crate::physical_plan::sorts::stream::{BatchId, BatchOffset, CursorStream}; +use crate::physical_plan::sorts::stream::CursorStream; use datafusion_common::Result; use futures::Stream; use std::pin::Pin; @@ -125,10 +125,9 @@ impl SortPreservingMergeStream { match futures::ready!(self.streams.poll_next(cx, idx)) { None => Poll::Ready(Ok(())), Some(Err(e)) => Poll::Ready(Err(e)), - Some(Ok((cursor, batch_id, batch_offset))) => Poll::Ready( - self.in_progress - .push_batch(idx, cursor, batch_id, batch_offset), - ), + Some(Ok(batch_cursor)) => { + Poll::Ready(self.in_progress.push_batch(idx, batch_cursor)) + } } } @@ -278,7 +277,7 @@ impl SortPreservingMergeStream { } impl Stream for SortPreservingMergeStream { - type Item = Result<(Vec<(C, BatchId, BatchOffset)>, Vec)>; + type Item = Result>; fn poll_next( mut self: Pin<&mut Self>, diff --git a/datafusion/core/src/physical_plan/sorts/mod.rs b/datafusion/core/src/physical_plan/sorts/mod.rs index a6b18b3c673f..6dcbdd7b02f0 100644 --- a/datafusion/core/src/physical_plan/sorts/mod.rs +++ b/datafusion/core/src/physical_plan/sorts/mod.rs @@ -17,6 +17,7 @@ //! Sort functionalities +mod batch_cursor; mod builder; mod cascade; mod cursor; diff --git a/datafusion/core/src/physical_plan/sorts/stream.rs b/datafusion/core/src/physical_plan/sorts/stream.rs index 4e6fda4b580b..aa29bda30063 100644 --- a/datafusion/core/src/physical_plan/sorts/stream.rs +++ b/datafusion/core/src/physical_plan/sorts/stream.rs @@ -33,6 +33,7 @@ use std::marker::PhantomData; use std::sync::Arc; use std::task::{ready, Context, Poll}; +use super::batch_cursor::{BatchCursor, BatchId, SlicedBatchCursorIdentifier}; use super::builder::YieldedSortOrder; /// A fallible [`PartitionedStream`] of record batches. @@ -41,19 +42,11 @@ use super::builder::YieldedSortOrder; pub(crate) type BatchCursorStream = Box>>; -pub type BatchId = u64; - -#[derive(Debug, Clone, Copy, Hash, PartialEq, Eq)] -pub struct BatchOffset(pub usize); - -/// A [`PartitionedStream`] representing partial record batches. -/// -/// Each ([`Cursor`], [`BatchId`], [`BatchOffset`]) represents part of a record batch -/// with the cursor.row_idx=0 representing the normalized key for the row at batch[idx=BatchOffset]. +/// A [`PartitionedStream`] representing partial record batches (a.k.a. [`BatchCursor`]). /// /// Each merge node (a `SortPreservingMergeStream` loser tree) will consume a [`CursorStream`]. pub(crate) type CursorStream = - Box>>; + Box>>>; /// A fallible stream of yielded [`SortOrder`]s is a [`MergeStream`]. /// @@ -62,14 +55,8 @@ pub(crate) type CursorStream = /// /// [`YieldedCursorStream`] then converts an output [`MergeStream`] /// into an input [`CursorStream`] for the next merge. -pub(crate) type MergeStream = std::pin::Pin< - Box< - dyn Send - + futures::Stream< - Item = Result<(Vec<(C, BatchId, BatchOffset)>, Vec)>, - >, - >, ->; +pub(crate) type MergeStream = + std::pin::Pin>>>>; /// A [`Stream`](futures::Stream) that has multiple partitions that can /// be polled separately but not concurrently @@ -277,7 +264,7 @@ impl OffsetCursorStream { } impl PartitionedStream for OffsetCursorStream { - type Output = Result<(C, BatchId, BatchOffset)>; + type Output = Result>; fn partitions(&self) -> usize { self.limit - self.offset @@ -345,7 +332,7 @@ impl BatchTrackingStream { } impl PartitionedStream for BatchTrackingStream { - type Output = Result<(C, BatchId, BatchOffset)>; + type Output = Result>; fn partitions(&self) -> usize { self.streams.partitions() @@ -355,14 +342,14 @@ impl PartitionedStream for BatchTrackingStream { &mut self, cx: &mut Context<'_>, stream_idx: usize, - ) -> Poll>> { + ) -> Poll> { Poll::Ready(ready!(self.streams.poll_next(cx, stream_idx)).map(|r| { r.and_then(|(cursor, batch)| { self.reservation.try_grow(batch.get_array_memory_size())?; let batch_id = self.monotonic_counter; self.monotonic_counter += 1; self.batches.insert(batch_id, Arc::new(batch)); - Ok((cursor, batch_id, BatchOffset(0_usize))) + Ok(BatchCursor::new(batch_id, cursor)) }) })) } @@ -378,7 +365,7 @@ impl std::fmt::Debug for BatchTrackingStream { /// A newtype wrapper around a set of fused [`MergeStream`] /// that implements debug, and skips over empty inner poll results -struct FusedMergeStreams(Vec>>); +struct FusedMergeStreams(Vec>>); impl FusedMergeStreams { fn poll_next( @@ -405,7 +392,7 @@ impl std::fmt::Debug for FusedMergeStreams { /// into an input [`CursorStream`] for the next merge. pub struct YieldedCursorStream { // Inner polled batch cursors, per stream_idx, which represents partially yielded batches. - cursors: Vec>>, + cursors: Vec>>>, /// Streams being polled streams: FusedMergeStreams, } @@ -419,10 +406,7 @@ impl YieldedCursorStream { } } - fn incr_next_batch( - &mut self, - stream_idx: usize, - ) -> Option<(C, BatchId, BatchOffset)> { + fn incr_next_batch(&mut self, stream_idx: usize) -> Option> { self.cursors[stream_idx] .as_mut() .and_then(|queue| queue.pop_front()) @@ -432,8 +416,8 @@ impl YieldedCursorStream { // We need to further parse the cursors into smaller batches. // // Input: - // - sort_order: Vec<(BatchId, row_idx)> = [(0,0), (0,1), (1,0), (0,2), (0,3)] - // - cursors: Vec<(C, BatchId, BatchOffset)> = [cursor_0, cursor_1] + // - sort_order: Vec<(SlicedBatchCursorIdentifier, row_idx)> = [(0,0), (0,1), (1,0), (0,2), (0,3)] + // - cursors: Vec = [cursor_0, cursor_1] // // Output stream: // Needs to be yielded to the next merge in three partial batches: @@ -445,21 +429,25 @@ impl YieldedCursorStream { fn try_parse_batches( &mut self, stream_idx: usize, - cursors: Vec<(C, BatchId, BatchOffset)>, + cursors: Vec>, sort_order: Vec, ) -> Result<()> { - let mut cursors_per_batch: HashMap<(BatchId, BatchOffset), C, RandomState> = - HashMap::with_capacity_and_hasher(cursors.len(), RandomState::new()); - for (cursor, batch_id, batch_offset) in cursors { - cursors_per_batch.insert((batch_id, batch_offset), cursor); + let mut cursors_per_batch: HashMap< + SlicedBatchCursorIdentifier, + BatchCursor, + RandomState, + > = HashMap::with_capacity_and_hasher(cursors.len(), RandomState::new()); + for cursor in cursors { + cursors_per_batch.insert(cursor.identifier(), cursor); } - let mut parsed_cursors: Vec<(C, BatchId, BatchOffset)> = + let mut parsed_cursors: Vec> = Vec::with_capacity(sort_order.len()); - let (mut prev_batch_id, mut prev_row_idx, mut prev_batch_offset) = sort_order[0]; + let ((mut prev_batch_id, mut prev_batch_offset), mut prev_row_idx) = + sort_order[0]; let mut len = 0; - for (batch_id, row_idx, batch_offset) in sort_order.iter() { + for ((batch_id, batch_offset), row_idx) in sort_order.iter() { if prev_batch_id == *batch_id && batch_offset.0 == prev_batch_offset.0 { len += 1; continue; @@ -469,11 +457,7 @@ impl YieldedCursorStream { cursors_per_batch.get(&(prev_batch_id, prev_batch_offset)) { let parsed_cursor = cursor.slice(prev_row_idx, len)?; - parsed_cursors.push(( - parsed_cursor, - prev_batch_id, - BatchOffset(prev_row_idx + prev_batch_offset.0), - )); + parsed_cursors.push(parsed_cursor); } else { unreachable!("cursor not found"); } @@ -486,11 +470,7 @@ impl YieldedCursorStream { } if let Some(cursor) = cursors_per_batch.get(&(prev_batch_id, prev_batch_offset)) { let parsed_cursor = cursor.slice(prev_row_idx, len)?; - parsed_cursors.push(( - parsed_cursor, - prev_batch_id, - BatchOffset(prev_row_idx + prev_batch_offset.0), - )); + parsed_cursors.push(parsed_cursor); } self.cursors[stream_idx] = Some(VecDeque::from(parsed_cursors)); @@ -499,7 +479,7 @@ impl YieldedCursorStream { } impl PartitionedStream for YieldedCursorStream { - type Output = Result<(C, BatchId, BatchOffset)>; + type Output = Result>; fn partitions(&self) -> usize { self.streams.0.len() From 3786021a2e087ee1576ab7be0becc01c75fa95e9 Mon Sep 17 00:00:00 2001 From: wiedld Date: Fri, 15 Sep 2023 00:07:18 -0700 Subject: [PATCH 22/24] feat(7181): update documentation for the cascaded merge --- .../core/src/physical_plan/sorts/cascade.rs | 64 +++++++++++++++++-- 1 file changed, 60 insertions(+), 4 deletions(-) diff --git a/datafusion/core/src/physical_plan/sorts/cascade.rs b/datafusion/core/src/physical_plan/sorts/cascade.rs index e2de02defe3c..d964294e0874 100644 --- a/datafusion/core/src/physical_plan/sorts/cascade.rs +++ b/datafusion/core/src/physical_plan/sorts/cascade.rs @@ -22,6 +22,58 @@ use std::pin::Pin; use std::sync::Arc; use std::task::{Context, Poll}; +/// Sort preserving cascade stream +/// +/// The cascade works as a tree of sort-preserving-merges, where each merge has +/// a limited fan-in (number of inputs) and a limit size yielded (batch size) per poll. +/// The poll is called from the root merge, which will poll its children, and so on. +/// +/// ```text +/// ┌─────┐ ┌─────┐ +/// │ 2 │ │ 1 │ +/// │ 3 │ │ 2 │ +/// │ 1 │─ ─▶ sort ─ ─▶│ 2 │─ ─ ─ ─ ─ ─ ─ ─ ┐ +/// │ 4 │ │ 3 │ +/// │ 2 │ │ 4 │ │ +/// └─────┘ └─────┘ +/// ┌─────┐ ┌─────┐ ▼ +/// │ 1 │ │ 1 │ +/// │ 4 │─ ▶ sort ─ ─ ▶│ 1 ├ ─ ─ ─ ─ ─ ▶ merge ─ ─ ─ ─ +/// │ 1 │ │ 4 │ │ +/// └─────┘ └─────┘ +/// ... ... ... ▼ +/// +/// merge ─ ─ ─ ─ ─ ─ ▶ sorted output +/// stream +/// ▲ +/// ... ... ... │ +/// ┌─────┐ ┌─────┐ +/// │ 3 │ │ 3 │ │ +/// │ 1 │─ ▶ sort ─ ─ ▶│ 1 │─ ─ ─ ─ ─ ─▶ merge ─ ─ ─ ─ +/// └─────┘ └─────┘ +/// ┌─────┐ ┌─────┐ ▲ +/// │ 4 │ │ 3 │ +/// │ 3 │─ ▶ sort ─ ─ ▶│ 4 │─ ─ ─ ─ ─ ─ ─ ─ ┘ +/// └─────┘ └─────┘ +/// +/// in_mem_batches do a series of merges that +/// each has a limited fan-in +/// (number of inputs) +/// ``` +/// +/// The cascade is built using a series of streams, each with a different purpose: +/// * Streams leading into the leaf nodes: +/// 1. [`BatchCursorStream`] yields the initial cursors and batches. (e.g. a RowCursorStream) +/// 2. [`BatchTrackingStream`] collects the batches, to avoid passing those around. Yields a [`CursorStream`](super::stream::CursorStream). +/// 3. This initial CursorStream is for a number of partitions (e.g. 100). +/// 4. The initial single CursorStream is shared across multiple leaf nodes, using [`OffsetCursorStream`]. +/// 5. The total fan-in is always limited to 10. E.g. each leaf node will pull from a dedicated 10 (out of 100) partitions. +/// +/// * Streams between merge nodes: +/// 1. a single [`MergeStream`] is yielded per node. +/// 2. A connector [`YieldedCursorStream`] converts a [`MergeStream`] into a [`CursorStream`](super::stream::CursorStream). +/// 3. next merge node takes a fan-in of up to 10 [`CursorStream`](super::stream::CursorStream)s. +/// pub(crate) struct SortPreservingCascadeStream { /// If the stream has encountered an error, or fetch is reached aborted: bool, @@ -120,21 +172,23 @@ impl SortPreservingCascadeStream { } } + /// Construct and yield the root node [`RecordBatch`]s. fn build_record_batch(&mut self, sort_order: Vec) -> Result { let mut batches_needed = Vec::with_capacity(sort_order.len()); let mut batches_seen: HashMap = HashMap::with_capacity(sort_order.len()); // (batch_idx, rows_sorted) + + // The sort_order yielded at each poll is relative to the sliced batch it came from. + // Therefore, the sort_order row_idx needs to be adjusted by the offset of the sliced batch. let mut sort_order_offset_adjusted = Vec::with_capacity(sort_order.len()); - let mut batch_idx: usize = 0; for ((batch_id, offset), row_idx) in sort_order.iter() { let batch_idx = match batches_seen.get(batch_id) { Some((batch_idx, _)) => *batch_idx, None => { + let batch_idx = batches_seen.len(); batches_needed.push(*batch_id); - let batch_now = batch_idx; - batch_idx += 1; - batch_now + batch_idx } }; sort_order_offset_adjusted.push((batch_idx, *row_idx + offset.0)); @@ -145,6 +199,7 @@ impl SortPreservingCascadeStream { let batches = batch_collecter.get_batches(batches_needed.as_slice()); drop(batch_collecter); + // remove record_batches (from the batch tracker) that are fully yielded let batches_to_remove = batches .iter() .zip(batches_needed) @@ -157,6 +212,7 @@ impl SortPreservingCascadeStream { }) .collect::>(); + // record_batch data to yield let columns = (0..self.schema.fields.len()) .map(|column_idx| { let arrays: Vec<_> = batches From 8701220aa7cd691d64f661e476314e67c4fe3b4d Mon Sep 17 00:00:00 2001 From: wiedld Date: Fri, 15 Sep 2023 10:58:17 -0700 Subject: [PATCH 23/24] fix: add apache license header to new mods --- .../src/physical_plan/sorts/batch_cursor.rs | 17 +++++++++++++++++ .../core/src/physical_plan/sorts/cascade.rs | 17 +++++++++++++++++ 2 files changed, 34 insertions(+) diff --git a/datafusion/core/src/physical_plan/sorts/batch_cursor.rs b/datafusion/core/src/physical_plan/sorts/batch_cursor.rs index 594c98617bfa..1ada42f9ed8a 100644 --- a/datafusion/core/src/physical_plan/sorts/batch_cursor.rs +++ b/datafusion/core/src/physical_plan/sorts/batch_cursor.rs @@ -1,3 +1,20 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + use datafusion_common::Result; use super::cursor::Cursor; diff --git a/datafusion/core/src/physical_plan/sorts/cascade.rs b/datafusion/core/src/physical_plan/sorts/cascade.rs index d964294e0874..fb16da3ec08f 100644 --- a/datafusion/core/src/physical_plan/sorts/cascade.rs +++ b/datafusion/core/src/physical_plan/sorts/cascade.rs @@ -1,3 +1,20 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + use crate::physical_plan::metrics::BaselineMetrics; use crate::physical_plan::sorts::builder::SortOrder; use crate::physical_plan::sorts::cursor::Cursor; From f97cc4dd63f944f54b1d01b07c9de1cc7b19fea0 Mon Sep 17 00:00:00 2001 From: wiedld Date: Tue, 19 Sep 2023 11:06:05 -0700 Subject: [PATCH 24/24] feat(7181): remove mutex around polled stream. * Move record batch tracking into its own abstraction and with interior mutability * Split streams instead of locking, which removes the need to poll per offset subset. * As a reflection of this reduced responsibilty, rename OffsetCursorStream to BatchTrackerStream. --- .../core/src/datasource/physical_plan/mod.rs | 1 - .../src/datasource/physical_plan/parquet.rs | 2 +- datafusion/physical-plan/src/sorts/cascade.rs | 56 +++--- datafusion/physical-plan/src/sorts/stream.rs | 180 ++++++++++-------- .../src/sorts/streaming_merge.rs | 4 +- 5 files changed, 132 insertions(+), 111 deletions(-) diff --git a/datafusion/core/src/datasource/physical_plan/mod.rs b/datafusion/core/src/datasource/physical_plan/mod.rs index bec9119df08a..d8ae6b3c04e6 100644 --- a/datafusion/core/src/datasource/physical_plan/mod.rs +++ b/datafusion/core/src/datasource/physical_plan/mod.rs @@ -577,7 +577,6 @@ mod tests { } #[test] - #[ignore] fn schema_adapter_map_schema_with_projection() { let table_schema = Arc::new(Schema::new(vec![ Field::new("c0", DataType::Utf8, true), diff --git a/datafusion/core/src/datasource/physical_plan/parquet.rs b/datafusion/core/src/datasource/physical_plan/parquet.rs index 6f27acfe7f6c..56eec4b1b7d5 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet.rs @@ -380,7 +380,7 @@ impl ExecutionPlan for ParquetExec { let stream = FileStream::new(&self.base_config, partition_index, opener, &self.metrics)?; - Ok(Box::pin(stream)) + Ok(Box::pin(stream) as SendableRecordBatchStream) } fn metrics(&self) -> Option { diff --git a/datafusion/physical-plan/src/sorts/cascade.rs b/datafusion/physical-plan/src/sorts/cascade.rs index bbf7d012bee8..800f767300aa 100644 --- a/datafusion/physical-plan/src/sorts/cascade.rs +++ b/datafusion/physical-plan/src/sorts/cascade.rs @@ -20,8 +20,7 @@ use crate::sorts::builder::SortOrder; use crate::sorts::cursor::Cursor; use crate::sorts::merge::SortPreservingMergeStream; use crate::sorts::stream::{ - BatchCursorStream, BatchTrackingStream, MergeStream, OffsetCursorStream, - YieldedCursorStream, + BatchCursorStream, BatchTracker, BatchTrackerStream, MergeStream, YieldedCursorStream, }; use crate::stream::ReceiverStream; use crate::RecordBatchStream; @@ -39,6 +38,8 @@ use std::pin::Pin; use std::sync::Arc; use std::task::{Context, Poll}; +static MAX_STREAMS_PER_MERGE: usize = 10; + /// Sort preserving cascade stream /// /// The cascade works as a tree of sort-preserving-merges, where each merge has @@ -81,10 +82,14 @@ use std::task::{Context, Poll}; /// The cascade is built using a series of streams, each with a different purpose: /// * Streams leading into the leaf nodes: /// 1. [`BatchCursorStream`] yields the initial cursors and batches. (e.g. a RowCursorStream) -/// 2. [`BatchTrackingStream`] collects the batches, to avoid passing those around. Yields a [`CursorStream`](super::stream::CursorStream). -/// 3. This initial CursorStream is for a number of partitions (e.g. 100). -/// 4. The initial single CursorStream is shared across multiple leaf nodes, using [`OffsetCursorStream`]. -/// 5. The total fan-in is always limited to 10. E.g. each leaf node will pull from a dedicated 10 (out of 100) partitions. +/// * This initial CursorStream is for a number of partitions (e.g. 100). +/// * only a single BatchCursorStream. +/// 2. [`BatchCursorStream::take_partitions()`] allows us to take a subset of the partitioned streams. +/// * This enables parallelism of [`BatchCursorStream`] with mutable access (for polling), without locking. +/// * The total fan-in is always limited to 10. E.g. each leaf node will pull from a dedicated 10 (out of 100) partitions. +/// 3. [`BatchTrackerStream`] is used to collect the record batches from the leaf nodes. +/// * contains a single, shared using [`BatchTracker`]. +/// * polling of streams is non-blocking. /// /// * Streams between merge nodes: /// 1. a single [`MergeStream`] is yielded per node. @@ -106,12 +111,12 @@ pub(crate) struct SortPreservingCascadeStream { /// Batches are collected on first yield from the RowCursorStream /// Subsequent merges in cascade all refer to the [`BatchId`]s - record_batch_collector: Arc>>, + record_batch_collector: Arc, } impl SortPreservingCascadeStream { - pub(crate) fn new( - streams: BatchCursorStream, + pub(crate) fn new>>( + mut streams: S, schema: SchemaRef, metrics: BaselineMetrics, batch_size: usize, @@ -120,23 +125,21 @@ impl SortPreservingCascadeStream { ) -> Self { let stream_count = streams.partitions(); - let streams = Arc::new(parking_lot::Mutex::new(BatchTrackingStream::new( - streams, - reservation.new_empty(), - ))); + let batch_tracker = Arc::new(BatchTracker::new(reservation.new_empty())); - let max_streams_per_merge = 10; + let max_streams_per_merge = MAX_STREAMS_PER_MERGE; let mut divided_streams: VecDeque> = VecDeque::with_capacity(stream_count / max_streams_per_merge + 1); // build leaves - for stream_offset in (0..stream_count).step_by(max_streams_per_merge) { - let limit = - std::cmp::min(stream_offset + max_streams_per_merge, stream_count); + for stream_idx in (0..stream_count).step_by(max_streams_per_merge) { + let limit = std::cmp::min(max_streams_per_merge, stream_count - stream_idx); - // OffsetCursorStream enables the ability to share the same RowCursorStream across multiple leafnode merges. - let streams = - OffsetCursorStream::new(Arc::clone(&streams), stream_offset, limit); + // divide the BatchCursorStream across multiple leafnode merges. + let streams = BatchTrackerStream::new( + streams.take_partitions(0..limit), + batch_tracker.clone(), + ); divided_streams.push_back(spawn_buffered_merge( Box::pin(SortPreservingMergeStream::new( @@ -185,7 +188,7 @@ impl SortPreservingCascadeStream { .expect("must have a root merge stream"), schema, metrics, - record_batch_collector: streams, + record_batch_collector: batch_tracker, } } @@ -212,9 +215,9 @@ impl SortPreservingCascadeStream { batches_seen.insert(*batch_id, (batch_idx, *row_idx + offset.0 + 1)); } - let batch_collecter = self.record_batch_collector.lock(); - let batches = batch_collecter.get_batches(batches_needed.as_slice()); - drop(batch_collecter); + let batches = self + .record_batch_collector + .get_batches(batches_needed.as_slice()); // remove record_batches (from the batch tracker) that are fully yielded let batches_to_remove = batches @@ -240,9 +243,8 @@ impl SortPreservingCascadeStream { }) .collect::>>()?; - let mut batch_collecter = self.record_batch_collector.lock(); - batch_collecter.remove_batches(batches_to_remove.as_slice()); - drop(batch_collecter); + self.record_batch_collector + .remove_batches(batches_to_remove.as_slice()); Ok(RecordBatch::try_new(self.schema.clone(), columns)?) } diff --git a/datafusion/physical-plan/src/sorts/stream.rs b/datafusion/physical-plan/src/sorts/stream.rs index f48d8b0c3bef..0cec3bbb3603 100644 --- a/datafusion/physical-plan/src/sorts/stream.rs +++ b/datafusion/physical-plan/src/sorts/stream.rs @@ -24,12 +24,14 @@ use arrow::array::Array; use arrow::datatypes::Schema; use arrow::record_batch::RecordBatch; use arrow::row::{RowConverter, SortField}; -use datafusion_common::{DataFusionError, Result}; +use datafusion_common::Result; use datafusion_execution::memory_pool::MemoryReservation; use futures::stream::{Fuse, StreamExt}; use parking_lot::Mutex; use std::collections::{HashMap, VecDeque}; use std::marker::PhantomData; +use std::ops::Range; +use std::sync::atomic::{AtomicU64, Ordering}; use std::sync::Arc; use std::task::{ready, Context, Poll}; @@ -39,14 +41,22 @@ use super::builder::YieldedSortOrder; /// A fallible [`PartitionedStream`] of record batches. /// /// Each [`Cursor`] and [`RecordBatch`] represents a single record batch. -pub(crate) type BatchCursorStream = - Box>>; +pub(crate) trait BatchCursorStream: + PartitionedStream + Send + 'static +{ + /// Acquire ownership over a subset of the partitioned streams. + /// + /// Like `Vec::take()`, this removes the indexed positions. + fn take_partitions(&mut self, range: Range) -> Box + where + Self: Sized; +} /// A [`PartitionedStream`] representing partial record batches (a.k.a. [`BatchCursor`]). /// /// Each merge node (a `SortPreservingMergeStream` loser tree) will consume a [`CursorStream`]. pub(crate) type CursorStream = - Box>>>; + Box>> + Send>; /// A fallible stream of yielded [`SortOrder`]s is a [`MergeStream`]. /// @@ -56,7 +66,7 @@ pub(crate) type CursorStream = /// [`YieldedCursorStream`] then converts an output [`MergeStream`] /// into an input [`CursorStream`] for the next merge. pub(crate) type MergeStream = - std::pin::Pin>>>>; + std::pin::Pin>> + Send>>; /// A [`Stream`](futures::Stream) that has multiple partitions that can /// be polled separately but not concurrently @@ -109,7 +119,7 @@ impl FusedStreams { #[derive(Debug)] pub struct RowCursorStream { /// Converter to convert output of physical expressions - converter: RowConverter, + converter: Arc, /// The physical expressions to sort by column_expressions: Vec>, /// Input streams @@ -134,7 +144,7 @@ impl RowCursorStream { .collect::>>()?; let streams = streams.into_iter().map(|s| s.fuse()).collect(); - let converter = RowConverter::new(sort_fields)?; + let converter = Arc::new(RowConverter::new(sort_fields)?); Ok(Self { converter, reservation, @@ -143,6 +153,15 @@ impl RowCursorStream { }) } + fn new_from_streams(&self, streams: FusedStreams) -> Self { + Self { + converter: self.converter.clone(), + column_expressions: self.column_expressions.clone(), + reservation: self.reservation.new_empty(), + streams, + } + } + fn convert_batch(&mut self, batch: &RecordBatch) -> Result { let cols = self .column_expressions @@ -181,6 +200,13 @@ impl PartitionedStream for RowCursorStream { } } +impl BatchCursorStream for RowCursorStream { + fn take_partitions(&mut self, range: Range) -> Box { + let streams_slice = self.streams.0.drain(range).collect::>(); + Box::new(self.new_from_streams(FusedStreams(streams_slice))) + } +} + /// Specialized stream for sorts on single primitive columns pub struct FieldCursorStream { /// The physical expressions to sort by @@ -208,6 +234,14 @@ impl FieldCursorStream { } } + fn new_from_streams(&self, streams: FusedStreams) -> Self { + Self { + sort: self.sort.clone(), + phantom: self.phantom, + streams, + } + } + fn convert_batch(&mut self, batch: &RecordBatch) -> Result> { let value = self.sort.expr.evaluate(batch)?; let array = value.into_array(batch.num_rows()); @@ -237,37 +271,44 @@ impl PartitionedStream for FieldCursorStream { } } -/// A wrapper around [`CursorStream`] that provides polling of a subset of the partitioned streams. +impl BatchCursorStream for FieldCursorStream { + fn take_partitions(&mut self, range: Range) -> Box { + let streams_slice = self.streams.0.drain(range).collect::>(); + Box::new(self.new_from_streams(FusedStreams(streams_slice))) + } +} + +/// A wrapper around [`CursorStream`] that collects the [`RecordBatch`] per poll, +/// and only passes along the [`BatchCursor`]. /// /// This is used in the leaf nodes of the cascading merge tree. -/// To have the same [`CursorStream`] (with the same RowConverter) -/// be separately polled by multiple leaf nodes. -pub struct OffsetCursorStream { - // Input streams. [`BatchTrackingStream`] is a [`CursorStream`]. - streams: Arc>>, - offset: usize, - limit: usize, +pub(crate) struct BatchTrackerStream< + C: Cursor, + S: BatchCursorStream>, +> { + // Partitioned Input stream. + streams: Box, + record_batch_holder: Arc, } -impl OffsetCursorStream { - pub fn new( - streams: Arc>>, - offset: usize, - limit: usize, - ) -> Self { +impl>> + BatchTrackerStream +{ + pub fn new(streams: Box, record_batch_holder: Arc) -> Self { Self { streams, - offset, - limit, + record_batch_holder, } } } -impl PartitionedStream for OffsetCursorStream { +impl>> + PartitionedStream for BatchTrackerStream +{ type Output = Result>; fn partitions(&self) -> usize { - self.limit - self.offset + self.streams.partitions() } fn poll_next( @@ -275,18 +316,18 @@ impl PartitionedStream for OffsetCursorStream { cx: &mut Context<'_>, stream_idx: usize, ) -> Poll> { - let stream_abs_idx = stream_idx + self.offset; - if stream_abs_idx >= self.limit { - return Poll::Ready(Some(Err(DataFusionError::Internal(format!( - "Invalid stream index {} for offset {} and limit {}", - stream_idx, self.offset, self.limit - ))))); - } - Poll::Ready(ready!(self.streams.lock().poll_next(cx, stream_abs_idx))) + Poll::Ready(ready!(self.streams.poll_next(cx, stream_idx)).map(|r| { + r.and_then(|(cursor, batch)| { + let batch_id = self.record_batch_holder.add_batch(batch)?; + Ok(BatchCursor::new(batch_id, cursor)) + }) + })) } } -impl std::fmt::Debug for OffsetCursorStream { +impl>> + std::fmt::Debug for BatchTrackerStream +{ fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { f.debug_struct("OffsetCursorStream").finish() } @@ -296,70 +337,49 @@ impl std::fmt::Debug for OffsetCursorStream { /// /// While storing the record batches outside of the cascading merge tree. /// Should be used with a Mutex. -pub struct BatchTrackingStream { +pub struct BatchTracker { /// Monotonically increasing batch id - monotonic_counter: u64, + monotonic_counter: AtomicU64, /// Write once, read many [`RecordBatch`]s - batches: HashMap, RandomState>, - /// Input streams yielding [`Cursor`]s and [`RecordBatch`]es - streams: BatchCursorStream, + batches: Mutex, RandomState>>, /// Accounts for memory used by buffered batches - reservation: MemoryReservation, + reservation: Mutex, } -impl BatchTrackingStream { - pub fn new(streams: BatchCursorStream, reservation: MemoryReservation) -> Self { +impl BatchTracker { + pub fn new(reservation: MemoryReservation) -> Self { Self { - monotonic_counter: 0, - batches: HashMap::with_hasher(RandomState::new()), - streams, - reservation, + monotonic_counter: AtomicU64::new(0), + batches: Mutex::new(HashMap::with_hasher(RandomState::new())), + reservation: Mutex::new(reservation), } } + pub fn add_batch(&self, batch: RecordBatch) -> Result { + self.reservation + .lock() + .try_grow(batch.get_array_memory_size())?; + let batch_id = self.monotonic_counter.fetch_add(1, Ordering::Relaxed); + self.batches.lock().insert(batch_id, Arc::new(batch)); + Ok(batch_id) + } + pub fn get_batches(&self, batch_ids: &[BatchId]) -> Vec> { - batch_ids - .iter() - .map(|id| self.batches[id].clone()) - .collect() + let batches = self.batches.lock(); + batch_ids.iter().map(|id| batches[id].clone()).collect() } - pub fn remove_batches(&mut self, batch_ids: &[BatchId]) { + pub fn remove_batches(&self, batch_ids: &[BatchId]) { + let mut batches = self.batches.lock(); for id in batch_ids { - self.batches.remove(id); + batches.remove(id); } } } -impl PartitionedStream for BatchTrackingStream { - type Output = Result>; - - fn partitions(&self) -> usize { - self.streams.partitions() - } - - fn poll_next( - &mut self, - cx: &mut Context<'_>, - stream_idx: usize, - ) -> Poll> { - Poll::Ready(ready!(self.streams.poll_next(cx, stream_idx)).map(|r| { - r.and_then(|(cursor, batch)| { - self.reservation.try_grow(batch.get_array_memory_size())?; - let batch_id = self.monotonic_counter; - self.monotonic_counter += 1; - self.batches.insert(batch_id, Arc::new(batch)); - Ok(BatchCursor::new(batch_id, cursor)) - }) - })) - } -} - -impl std::fmt::Debug for BatchTrackingStream { +impl std::fmt::Debug for BatchTracker { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - f.debug_struct("BatchTrackingStream") - .field("num_partitions", &self.partitions()) - .finish() + f.debug_struct("BatchTracker").finish() } } diff --git a/datafusion/physical-plan/src/sorts/streaming_merge.rs b/datafusion/physical-plan/src/sorts/streaming_merge.rs index 7455e63867c9..e026ad683cdf 100644 --- a/datafusion/physical-plan/src/sorts/streaming_merge.rs +++ b/datafusion/physical-plan/src/sorts/streaming_merge.rs @@ -39,7 +39,7 @@ macro_rules! merge_helper { ($t:ty, $sort:ident, $streams:ident, $schema:ident, $tracking_metrics:ident, $batch_size:ident, $fetch:ident, $reservation:ident) => {{ let streams = FieldCursorStream::<$t>::new($sort, $streams); return Ok(Box::pin(SortPreservingCascadeStream::new( - Box::new(streams), + streams, $schema, $tracking_metrics, $batch_size, @@ -82,7 +82,7 @@ pub fn streaming_merge( )?; Ok(Box::pin(SortPreservingCascadeStream::new( - Box::new(streams), + streams, schema, metrics, batch_size,