Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Simplify streaming_merge function parameters #12719

Merged
merged 3 commits into from
Oct 4, 2024
Merged
Show file tree
Hide file tree
Changes from 2 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,7 @@ mod sp_repartition_fuzz_tests {
metrics::{BaselineMetrics, ExecutionPlanMetricsSet},
repartition::RepartitionExec,
sorts::sort_preserving_merge::SortPreservingMergeExec,
sorts::streaming_merge::streaming_merge,
sorts::streaming_merge::{streaming_merge, StreamingMergeConfig},
stream::RecordBatchStreamAdapter,
ExecutionPlan, Partitioning,
};
Expand Down Expand Up @@ -246,15 +246,15 @@ mod sp_repartition_fuzz_tests {
MemoryConsumer::new("test".to_string()).register(context.memory_pool());

// Internally SortPreservingMergeExec uses this function for merging.
let res = streaming_merge(
let res = streaming_merge(StreamingMergeConfig {
streams,
schema,
&exprs,
BaselineMetrics::new(&ExecutionPlanMetricsSet::new(), 0),
1,
None,
mem_reservation,
)?;
expressions: &exprs,
metrics: BaselineMetrics::new(&ExecutionPlanMetricsSet::new(), 0),
batch_size: 1,
fetch: None,
reservation: mem_reservation,
})?;
let res = collect(res).await?;
// Contains the merged result.
let res = concat_batches(&res[0].schema(), &res)?;
Expand Down
16 changes: 8 additions & 8 deletions datafusion/physical-plan/src/aggregates/row_hash.rs
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,7 @@ use crate::aggregates::{
};
use crate::metrics::{BaselineMetrics, MetricBuilder, RecordOutput};
use crate::sorts::sort::sort_batch;
use crate::sorts::streaming_merge;
use crate::sorts::streaming_merge::{streaming_merge, StreamingMergeConfig};
use crate::spill::{read_spill_as_stream, spill_record_batch_by_size};
use crate::stream::RecordBatchStreamAdapter;
use crate::{aggregates, metrics, ExecutionPlan, PhysicalExpr};
Expand Down Expand Up @@ -1001,15 +1001,15 @@ impl GroupedHashAggregateStream {
streams.push(stream);
}
self.spill_state.is_stream_merging = true;
self.input = streaming_merge(
self.input = streaming_merge(StreamingMergeConfig {
streams,
schema,
&self.spill_state.spill_expr,
self.baseline_metrics.clone(),
self.batch_size,
None,
self.reservation.new_empty(),
)?;
expressions: &self.spill_state.spill_expr,
metrics: self.baseline_metrics.clone(),
batch_size: self.batch_size,
fetch: None,
reservation: self.reservation.new_empty(),
})?;
self.input_done = false;
self.group_ordering = GroupOrdering::Full(GroupOrderingFull::new());
Ok(())
Expand Down
18 changes: 9 additions & 9 deletions datafusion/physical-plan/src/repartition/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,7 @@ use crate::metrics::BaselineMetrics;
use crate::repartition::distributor_channels::{
channels, partition_aware_channels, DistributionReceiver, DistributionSender,
};
use crate::sorts::streaming_merge;
use crate::sorts::streaming_merge::{streaming_merge, StreamingMergeConfig};
use crate::stream::RecordBatchStreamAdapter;
use crate::{DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, Statistics};

Expand Down Expand Up @@ -637,15 +637,15 @@ impl ExecutionPlan for RepartitionExec {
let merge_reservation =
MemoryConsumer::new(format!("{}[Merge {partition}]", name))
.register(context.memory_pool());
streaming_merge(
input_streams,
schema_captured,
&sort_exprs,
BaselineMetrics::new(&metrics, partition),
context.session_config().batch_size(),
streaming_merge(StreamingMergeConfig {
streams: input_streams,
schema: schema_captured,
expressions: &sort_exprs,
metrics: BaselineMetrics::new(&metrics, partition),
batch_size: context.session_config().batch_size(),
fetch,
merge_reservation,
)
reservation: merge_reservation,
})
} else {
Ok(Box::pin(RepartitionStream {
num_input_partitions,
Expand Down
1 change: 0 additions & 1 deletion datafusion/physical-plan/src/sorts/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -28,4 +28,3 @@ mod stream;
pub mod streaming_merge;

pub use index::RowIndex;
pub(crate) use streaming_merge::streaming_merge;
32 changes: 16 additions & 16 deletions datafusion/physical-plan/src/sorts/sort.rs
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,7 @@ use crate::limit::LimitStream;
use crate::metrics::{
BaselineMetrics, Count, ExecutionPlanMetricsSet, MetricBuilder, MetricsSet,
};
use crate::sorts::streaming_merge::streaming_merge;
use crate::sorts::streaming_merge::{streaming_merge, StreamingMergeConfig};
use crate::spill::{read_spill_as_stream, spill_record_batches};
use crate::stream::RecordBatchStreamAdapter;
use crate::topk::TopK;
Expand Down Expand Up @@ -342,15 +342,15 @@ impl ExternalSorter {
streams.push(stream);
}

streaming_merge(
streaming_merge(StreamingMergeConfig {
streams,
Arc::clone(&self.schema),
&self.expr,
self.metrics.baseline.clone(),
self.batch_size,
self.fetch,
self.reservation.new_empty(),
)
schema: Arc::clone(&self.schema),
expressions: &self.expr,
metrics: self.metrics.baseline.clone(),
batch_size: self.batch_size,
fetch: self.fetch,
reservation: self.reservation.new_empty(),
})
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Another potential API to consider might be a builder style

Suggested change
streaming_merge(StreamingMergeConfig {
streams,
Arc::clone(&self.schema),
&self.expr,
self.metrics.baseline.clone(),
self.batch_size,
self.fetch,
self.reservation.new_empty(),
)
schema: Arc::clone(&self.schema),
expressions: &self.expr,
metrics: self.metrics.baseline.clone(),
batch_size: self.batch_size,
fetch: self.fetch,
reservation: self.reservation.new_empty(),
})
StreamingMergeBuilder::new()
.with_streams(streams)
.with_schema(Arc::clone(&self.schema))
.with_expressions(&self.expr)
.with_metrics(self.metrics.baseline.clone())
.with_batch_size(self.batch_size)
.with_fetch(self.fetch)
.with_reservation(self.reservation.new_empty())
.build()?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

👍 for the builder

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thank you for the review and suggestion! I've implemented the builder pattern like you suggested.

} else {
self.in_mem_sort_stream(self.metrics.baseline.clone())
}
Expand Down Expand Up @@ -534,15 +534,15 @@ impl ExternalSorter {
})
.collect::<Result<_>>()?;

streaming_merge(
streaming_merge(StreamingMergeConfig {
streams,
Arc::clone(&self.schema),
&self.expr,
schema: Arc::clone(&self.schema),
expressions: &self.expr,
metrics,
self.batch_size,
self.fetch,
self.merge_reservation.new_empty(),
)
batch_size: self.batch_size,
fetch: self.fetch,
reservation: self.merge_reservation.new_empty(),
})
}

/// Sorts a single `RecordBatch` into a single stream.
Expand Down
28 changes: 14 additions & 14 deletions datafusion/physical-plan/src/sorts/sort_preserving_merge.rs
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@ use crate::common::spawn_buffered;
use crate::expressions::PhysicalSortExpr;
use crate::limit::LimitStream;
use crate::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet};
use crate::sorts::streaming_merge;
use crate::sorts::streaming_merge::{streaming_merge, StreamingMergeConfig};
use crate::{
DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, ExecutionPlanProperties,
Partitioning, PlanProperties, SendableRecordBatchStream, Statistics,
Expand Down Expand Up @@ -273,15 +273,15 @@ impl ExecutionPlan for SortPreservingMergeExec {

debug!("Done setting up sender-receiver for SortPreservingMergeExec::execute");

let result = streaming_merge(
receivers,
let result = streaming_merge(StreamingMergeConfig {
streams: receivers,
schema,
&self.expr,
BaselineMetrics::new(&self.metrics, partition),
context.session_config().batch_size(),
self.fetch,
expressions: &self.expr,
metrics: BaselineMetrics::new(&self.metrics, partition),
batch_size: context.session_config().batch_size(),
fetch: self.fetch,
reservation,
)?;
})?;

debug!("Got stream result from SortPreservingMergeStream::new_from_receivers");

Expand Down Expand Up @@ -960,15 +960,15 @@ mod tests {
MemoryConsumer::new("test").register(&task_ctx.runtime_env().memory_pool);

let fetch = None;
let merge_stream = streaming_merge(
let merge_stream = streaming_merge(StreamingMergeConfig {
streams,
batches.schema(),
sort.as_slice(),
BaselineMetrics::new(&metrics, 0),
task_ctx.session_config().batch_size(),
schema: batches.schema(),
expressions: sort.as_slice(),
metrics: BaselineMetrics::new(&metrics, 0),
batch_size: task_ctx.session_config().batch_size(),
fetch,
reservation,
)
})
.unwrap();

let mut merged = common::collect(merge_stream).await.unwrap();
Expand Down
69 changes: 37 additions & 32 deletions datafusion/physical-plan/src/sorts/streaming_merge.rs
Original file line number Diff line number Diff line change
Expand Up @@ -36,62 +36,67 @@ 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);
($t:ty, $sort:ident, $config:ident) => {{
let streams = FieldCursorStream::<$t>::new($sort, $config.streams);
return Ok(Box::pin(SortPreservingMergeStream::new(
Box::new(streams),
$schema,
$tracking_metrics,
$batch_size,
$fetch,
$reservation,
$config.schema,
$config.metrics,
$config.batch_size,
$config.fetch,
$config.reservation,
)));
}};
}

/// Configuration parameters to initialize a `SortPreservingMergeStream`
pub struct StreamingMergeConfig<'a> {
pub streams: Vec<SendableRecordBatchStream>,
pub schema: SchemaRef,
pub expressions: &'a [PhysicalSortExpr],
pub metrics: BaselineMetrics,
pub batch_size: usize,
pub fetch: Option<usize>,
pub reservation: MemoryReservation,
}

/// Perform a streaming merge of [`SendableRecordBatchStream`] based on provided sort expressions
/// while preserving order.
pub fn streaming_merge(
streams: Vec<SendableRecordBatchStream>,
schema: SchemaRef,
expressions: &[PhysicalSortExpr],
metrics: BaselineMetrics,
batch_size: usize,
fetch: Option<usize>,
reservation: MemoryReservation,
config: StreamingMergeConfig,
) -> Result<SendableRecordBatchStream> {
// If there are no sort expressions, preserving the order
// doesn't mean anything (and result in infinite loops)
if expressions.is_empty() {
if config.expressions.is_empty() {
return internal_err!("Sort expressions cannot be empty for streaming merge");
}
// 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())?;
if config.expressions.len() == 1 {
let sort = config.expressions[0].clone();
let data_type = sort.expr.data_type(config.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)
data_type => (primitive_merge_helper, sort, config),
DataType::Utf8 => merge_helper!(StringArray, sort, config)
DataType::LargeUtf8 => merge_helper!(LargeStringArray, sort, config)
DataType::Binary => merge_helper!(BinaryArray, sort, config)
DataType::LargeBinary => merge_helper!(LargeBinaryArray, sort, config)
_ => {}
}
}

let streams = RowCursorStream::try_new(
schema.as_ref(),
expressions,
streams,
reservation.new_empty(),
config.schema.as_ref(),
config.expressions,
config.streams,
config.reservation.new_empty(),
)?;

Ok(Box::pin(SortPreservingMergeStream::new(
Box::new(streams),
schema,
metrics,
batch_size,
fetch,
reservation,
config.schema,
config.metrics,
config.batch_size,
config.fetch,
config.reservation,
)))
}