From ced6c74599135142691bc45d7b8c912dd768efeb Mon Sep 17 00:00:00 2001 From: Weijun-H Date: Sat, 14 Dec 2024 16:31:37 +0800 Subject: [PATCH 01/25] draft --- datafusion/physical-plan/src/sorts/merge.rs | 4 ++ .../src/sorts/sort_preserving_merge.rs | 41 ++++++++++++++----- .../src/sorts/streaming_merge.rs | 26 ++++++++---- 3 files changed, 54 insertions(+), 17 deletions(-) diff --git a/datafusion/physical-plan/src/sorts/merge.rs b/datafusion/physical-plan/src/sorts/merge.rs index 1c2b8cd0c91b7..e0b736ead155f 100644 --- a/datafusion/physical-plan/src/sorts/merge.rs +++ b/datafusion/physical-plan/src/sorts/merge.rs @@ -148,6 +148,8 @@ pub(crate) struct SortPreservingMergeStream { /// vector to ensure the next iteration starts with a different partition, preventing the same partition /// from being continuously polled. uninitiated_partitions: VecDeque, + + enable_pull_based_execution: bool, } impl SortPreservingMergeStream { @@ -159,6 +161,7 @@ impl SortPreservingMergeStream { fetch: Option, reservation: MemoryReservation, enable_round_robin_tie_breaker: bool, + enable_pull_based_execution: bool, ) -> Self { let stream_count = streams.partitions(); @@ -180,6 +183,7 @@ impl SortPreservingMergeStream { produced: 0, uninitiated_partitions: (0..stream_count).collect(), enable_round_robin_tie_breaker, + enable_pull_based_execution, } } diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index 454a068551754..d25b107cd00da 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -37,6 +37,7 @@ use datafusion_physical_expr::PhysicalSortExpr; use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; use log::{debug, trace}; +use tokio::stream; /// Sort preserving merge execution plan /// @@ -98,6 +99,7 @@ pub struct SortPreservingMergeExec { /// /// See [`Self::with_round_robin_repartition`] for more information. enable_round_robin_repartition: bool, + enable_pull_based_execution: bool, } impl SortPreservingMergeExec { @@ -111,6 +113,7 @@ impl SortPreservingMergeExec { fetch: None, cache, enable_round_robin_repartition: true, + enable_pull_based_execution: true, } } @@ -216,6 +219,7 @@ impl ExecutionPlan for SortPreservingMergeExec { fetch: limit, cache: self.cache.clone(), enable_round_robin_repartition: true, + enable_pull_based_execution: true, })) } @@ -297,18 +301,34 @@ impl ExecutionPlan for SortPreservingMergeExec { } }, _ => { - let receivers = (0..input_partitions) - .map(|partition| { - let stream = - self.input.execute(partition, Arc::clone(&context))?; - Ok(spawn_buffered(stream, 1)) - }) - .collect::>()?; - - debug!("Done setting up sender-receiver for SortPreservingMergeExec::execute"); + let streams = if self.enable_pull_based_execution { + // Direct stream connection without channels + let streams = (0..input_partitions) + .map(|partition| { + self.input.execute(partition, Arc::clone(&context)) + }) + .collect::>()?; + + debug!( + "Setting up direct streams for SortPreservingMergeExec::execute" + ); + streams + } else { + // Channel based stream connection + let receivers = (0..input_partitions) + .map(|partition| { + let stream = + self.input.execute(partition, Arc::clone(&context))?; + Ok(spawn_buffered(stream, 1)) + }) + .collect::>()?; + + debug!("Done setting up sender-receiver for SortPreservingMergeExec::execute"); + receivers + }; let result = StreamingMergeBuilder::new() - .with_streams(receivers) + .with_streams(streams) .with_schema(schema) .with_expressions(self.expr.as_ref()) .with_metrics(BaselineMetrics::new(&self.metrics, partition)) @@ -316,6 +336,7 @@ impl ExecutionPlan for SortPreservingMergeExec { .with_fetch(self.fetch) .with_reservation(reservation) .with_round_robin_tie_breaker(self.enable_round_robin_repartition) + .with_pull_based_execution(self.enable_pull_based_execution) .build()?; debug!("Got stream result from SortPreservingMergeStream::new_from_receivers"); diff --git a/datafusion/physical-plan/src/sorts/streaming_merge.rs b/datafusion/physical-plan/src/sorts/streaming_merge.rs index a541f79dc7174..bd29d5a88dd0c 100644 --- a/datafusion/physical-plan/src/sorts/streaming_merge.rs +++ b/datafusion/physical-plan/src/sorts/streaming_merge.rs @@ -37,8 +37,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, $enable_round_robin_tie_breaker:ident) => {{ - let streams = + ($t:ty, $sort:ident, $streams:ident, $schema:ident, $tracking_metrics:ident, $batch_size:ident, $fetch:ident, $reservation:ident, $enable_round_robin_tie_breaker:ident, $enable_pull_based_execution:ident) => {{ FieldCursorStream::<$t>::new($sort, $streams, $reservation.new_empty()); return Ok(Box::pin(SortPreservingMergeStream::new( Box::new(streams), @@ -48,6 +47,7 @@ macro_rules! merge_helper { $fetch, $reservation, $enable_round_robin_tie_breaker, + $enable_pull_based_execution, ))); }}; } @@ -61,6 +61,7 @@ pub struct StreamingMergeBuilder<'a> { fetch: Option, reservation: Option, enable_round_robin_tie_breaker: bool, + enable_pull_based_execution: bool, } impl Default for StreamingMergeBuilder<'_> { @@ -74,6 +75,7 @@ impl Default for StreamingMergeBuilder<'_> { fetch: None, reservation: None, enable_round_robin_tie_breaker: false, + enable_pull_based_execution: false, } } } @@ -133,6 +135,14 @@ impl<'a> StreamingMergeBuilder<'a> { self } + pub fn with_pull_based_execution( + mut self, + enable_pull_based_execution: bool, + ) -> Self { + self.enable_pull_based_execution = enable_pull_based_execution; + self + } + pub fn build(self) -> Result { let Self { streams, @@ -143,6 +153,7 @@ impl<'a> StreamingMergeBuilder<'a> { fetch, expressions, enable_round_robin_tie_breaker, + enable_pull_based_execution, } = self; // Early return if streams or expressions are empty @@ -175,11 +186,11 @@ impl<'a> StreamingMergeBuilder<'a> { 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, enable_round_robin_tie_breaker), - DataType::Utf8 => merge_helper!(StringArray, sort, streams, schema, metrics, batch_size, fetch, reservation, enable_round_robin_tie_breaker) - DataType::LargeUtf8 => merge_helper!(LargeStringArray, sort, streams, schema, metrics, batch_size, fetch, reservation, enable_round_robin_tie_breaker) - DataType::Binary => merge_helper!(BinaryArray, sort, streams, schema, metrics, batch_size, fetch, reservation, enable_round_robin_tie_breaker) - DataType::LargeBinary => merge_helper!(LargeBinaryArray, sort, streams, schema, metrics, batch_size, fetch, reservation, enable_round_robin_tie_breaker) + data_type => (primitive_merge_helper, sort, streams, schema, metrics, batch_size, fetch, reservation, enable_round_robin_tie_breaker, enable_pull_based_execution), + DataType::Utf8 => merge_helper!(StringArray, sort, streams, schema, metrics, batch_size, fetch, reservation, enable_round_robin_tie_breaker, enable_pull_based_execution), + DataType::LargeUtf8 => merge_helper!(LargeStringArray, sort, streams, schema, metrics, batch_size, fetch, reservation, enable_round_robin_tie_breaker, enable_pull_based_execution) + DataType::Binary => merge_helper!(BinaryArray, sort, streams, schema, metrics, batch_size, fetch, reservation, enable_round_robin_tie_breaker, enable_pull_based_execution) + DataType::LargeBinary => merge_helper!(LargeBinaryArray, sort, streams, schema, metrics, batch_size, fetch, reservation, enable_round_robin_tie_breaker, enable_pull_based_execution) _ => {} } } @@ -198,6 +209,7 @@ impl<'a> StreamingMergeBuilder<'a> { fetch, reservation, enable_round_robin_tie_breaker, + enable_pull_based_execution, ))) } } From dc2d89fe37be7a776d13d92aba52c88039450220 Mon Sep 17 00:00:00 2001 From: Weijun-H Date: Sat, 14 Dec 2024 17:07:45 +0800 Subject: [PATCH 02/25] clean up --- datafusion/physical-plan/src/sorts/merge.rs | 4 --- .../src/sorts/sort_preserving_merge.rs | 2 -- .../src/sorts/streaming_merge.rs | 27 +++++-------------- 3 files changed, 7 insertions(+), 26 deletions(-) diff --git a/datafusion/physical-plan/src/sorts/merge.rs b/datafusion/physical-plan/src/sorts/merge.rs index e0b736ead155f..1c2b8cd0c91b7 100644 --- a/datafusion/physical-plan/src/sorts/merge.rs +++ b/datafusion/physical-plan/src/sorts/merge.rs @@ -148,8 +148,6 @@ pub(crate) struct SortPreservingMergeStream { /// vector to ensure the next iteration starts with a different partition, preventing the same partition /// from being continuously polled. uninitiated_partitions: VecDeque, - - enable_pull_based_execution: bool, } impl SortPreservingMergeStream { @@ -161,7 +159,6 @@ impl SortPreservingMergeStream { fetch: Option, reservation: MemoryReservation, enable_round_robin_tie_breaker: bool, - enable_pull_based_execution: bool, ) -> Self { let stream_count = streams.partitions(); @@ -183,7 +180,6 @@ impl SortPreservingMergeStream { produced: 0, uninitiated_partitions: (0..stream_count).collect(), enable_round_robin_tie_breaker, - enable_pull_based_execution, } } diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index d25b107cd00da..4555f59e4c945 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -37,7 +37,6 @@ use datafusion_physical_expr::PhysicalSortExpr; use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; use log::{debug, trace}; -use tokio::stream; /// Sort preserving merge execution plan /// @@ -336,7 +335,6 @@ impl ExecutionPlan for SortPreservingMergeExec { .with_fetch(self.fetch) .with_reservation(reservation) .with_round_robin_tie_breaker(self.enable_round_robin_repartition) - .with_pull_based_execution(self.enable_pull_based_execution) .build()?; debug!("Got stream result from SortPreservingMergeStream::new_from_receivers"); diff --git a/datafusion/physical-plan/src/sorts/streaming_merge.rs b/datafusion/physical-plan/src/sorts/streaming_merge.rs index bd29d5a88dd0c..e1c3b2cb8bb81 100644 --- a/datafusion/physical-plan/src/sorts/streaming_merge.rs +++ b/datafusion/physical-plan/src/sorts/streaming_merge.rs @@ -37,8 +37,8 @@ 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, $enable_round_robin_tie_breaker:ident, $enable_pull_based_execution:ident) => {{ - FieldCursorStream::<$t>::new($sort, $streams, $reservation.new_empty()); + ($t:ty, $sort:ident, $streams:ident, $schema:ident, $tracking_metrics:ident, $batch_size:ident, $fetch:ident, $reservation:ident, $enable_round_robin_tie_breaker:ident) => {{ + FieldCursorStream::<$t>::new($sort, $streams, $reservation.new_empty()); return Ok(Box::pin(SortPreservingMergeStream::new( Box::new(streams), $schema, @@ -47,7 +47,6 @@ macro_rules! merge_helper { $fetch, $reservation, $enable_round_robin_tie_breaker, - $enable_pull_based_execution, ))); }}; } @@ -61,7 +60,6 @@ pub struct StreamingMergeBuilder<'a> { fetch: Option, reservation: Option, enable_round_robin_tie_breaker: bool, - enable_pull_based_execution: bool, } impl Default for StreamingMergeBuilder<'_> { @@ -75,7 +73,6 @@ impl Default for StreamingMergeBuilder<'_> { fetch: None, reservation: None, enable_round_robin_tie_breaker: false, - enable_pull_based_execution: false, } } } @@ -135,14 +132,6 @@ impl<'a> StreamingMergeBuilder<'a> { self } - pub fn with_pull_based_execution( - mut self, - enable_pull_based_execution: bool, - ) -> Self { - self.enable_pull_based_execution = enable_pull_based_execution; - self - } - pub fn build(self) -> Result { let Self { streams, @@ -153,7 +142,6 @@ impl<'a> StreamingMergeBuilder<'a> { fetch, expressions, enable_round_robin_tie_breaker, - enable_pull_based_execution, } = self; // Early return if streams or expressions are empty @@ -186,11 +174,11 @@ impl<'a> StreamingMergeBuilder<'a> { 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, enable_round_robin_tie_breaker, enable_pull_based_execution), - DataType::Utf8 => merge_helper!(StringArray, sort, streams, schema, metrics, batch_size, fetch, reservation, enable_round_robin_tie_breaker, enable_pull_based_execution), - DataType::LargeUtf8 => merge_helper!(LargeStringArray, sort, streams, schema, metrics, batch_size, fetch, reservation, enable_round_robin_tie_breaker, enable_pull_based_execution) - DataType::Binary => merge_helper!(BinaryArray, sort, streams, schema, metrics, batch_size, fetch, reservation, enable_round_robin_tie_breaker, enable_pull_based_execution) - DataType::LargeBinary => merge_helper!(LargeBinaryArray, sort, streams, schema, metrics, batch_size, fetch, reservation, enable_round_robin_tie_breaker, enable_pull_based_execution) + data_type => (primitive_merge_helper, sort, streams, schema, metrics, batch_size, fetch, reservation, enable_round_robin_tie_breaker), + DataType::Utf8 => merge_helper!(StringArray, sort, streams, schema, metrics, batch_size, fetch, reservation, enable_round_robin_tie_breaker), + DataType::LargeUtf8 => merge_helper!(LargeStringArray, sort, streams, schema, metrics, batch_size, fetch, reservation, enable_round_robin_tie_breaker) + DataType::Binary => merge_helper!(BinaryArray, sort, streams, schema, metrics, batch_size, fetch, reservation, enable_round_robin_tie_breaker) + DataType::LargeBinary => merge_helper!(LargeBinaryArray, sort, streams, schema, metrics, batch_size, fetch, reservation, enable_round_robin_tie_breaker) _ => {} } } @@ -209,7 +197,6 @@ impl<'a> StreamingMergeBuilder<'a> { fetch, reservation, enable_round_robin_tie_breaker, - enable_pull_based_execution, ))) } } From 532bfcac4ae558de628cbf4e4cb4fe900988066f Mon Sep 17 00:00:00 2001 From: Weijun-H Date: Sun, 15 Dec 2024 16:55:11 +0800 Subject: [PATCH 03/25] clean up --- datafusion/physical-plan/src/sorts/streaming_merge.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/physical-plan/src/sorts/streaming_merge.rs b/datafusion/physical-plan/src/sorts/streaming_merge.rs index e1c3b2cb8bb81..090ec498d71d3 100644 --- a/datafusion/physical-plan/src/sorts/streaming_merge.rs +++ b/datafusion/physical-plan/src/sorts/streaming_merge.rs @@ -175,7 +175,7 @@ impl<'a> StreamingMergeBuilder<'a> { 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, enable_round_robin_tie_breaker), - DataType::Utf8 => merge_helper!(StringArray, sort, streams, schema, metrics, batch_size, fetch, reservation, enable_round_robin_tie_breaker), + DataType::Utf8 => merge_helper!(StringArray, sort, streams, schema, metrics, batch_size, fetch, reservation, enable_round_robin_tie_breaker) DataType::LargeUtf8 => merge_helper!(LargeStringArray, sort, streams, schema, metrics, batch_size, fetch, reservation, enable_round_robin_tie_breaker) DataType::Binary => merge_helper!(BinaryArray, sort, streams, schema, metrics, batch_size, fetch, reservation, enable_round_robin_tie_breaker) DataType::LargeBinary => merge_helper!(LargeBinaryArray, sort, streams, schema, metrics, batch_size, fetch, reservation, enable_round_robin_tie_breaker) From 280e09a7cd1189d7d9858bb8e5f34c6bd74ea63d Mon Sep 17 00:00:00 2001 From: Weijun-H Date: Tue, 24 Dec 2024 10:45:38 +0200 Subject: [PATCH 04/25] initial start of OnDemandRepartitionExec --- .../physical-plan/src/repartition/mod.rs | 2 + .../src/repartition/on_demand_repartition.rs | 1214 +++++++++++++++++ 2 files changed, 1216 insertions(+) create mode 100644 datafusion/physical-plan/src/repartition/on_demand_repartition.rs diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 40e68cfcae837..546527beee446 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -58,6 +58,8 @@ use log::trace; use parking_lot::Mutex; mod distributor_channels; +mod on_demand_repartition; + type MaybeBatch = Option>; type InputPartitionsToCurrentPartitionSender = Vec>; diff --git a/datafusion/physical-plan/src/repartition/on_demand_repartition.rs b/datafusion/physical-plan/src/repartition/on_demand_repartition.rs new file mode 100644 index 0000000000000..354e82d926017 --- /dev/null +++ b/datafusion/physical-plan/src/repartition/on_demand_repartition.rs @@ -0,0 +1,1214 @@ +// 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. + +//! This file implements the [`RepartitionExec`] operator, which maps N input +//! partitions to M output partitions based on a partitioning scheme, optionally +//! maintaining the order of the input rows in the output. + +use std::pin::Pin; +use std::sync::Arc; +use std::task::{Context, Poll}; +use std::{any::Any, vec}; + +use super::metrics::{self, ExecutionPlanMetricsSet, MetricBuilder, MetricsSet}; +use super::{ + BatchPartitioner, DisplayAs, ExecutionPlanProperties, + InputPartitionsToCurrentPartitionReceiver, InputPartitionsToCurrentPartitionSender, + MaybeBatch, RecordBatchStream, RepartitionMetrics, SendableRecordBatchStream, +}; +use crate::common::SharedMemoryReservation; +use crate::execution_plan::CardinalityEffect; +use crate::hash_utils::create_hashes; +use crate::metrics::BaselineMetrics; +use crate::repartition::distributor_channels::{ + channels, partition_aware_channels, DistributionReceiver, DistributionSender, +}; +use crate::repartition::PerPartitionStream; +use crate::sorts::streaming_merge::StreamingMergeBuilder; +use crate::stream::RecordBatchStreamAdapter; +use crate::{DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, Statistics}; + +use arrow::compute::take_arrays; +use arrow::datatypes::{SchemaRef, UInt32Type}; +use arrow::record_batch::RecordBatch; +use arrow_array::{PrimitiveArray, RecordBatchOptions}; +use datafusion_common::utils::transpose; +use datafusion_common::{not_impl_err, DataFusionError, Result}; +use datafusion_common_runtime::SpawnedTask; +use datafusion_execution::memory_pool::MemoryConsumer; +use datafusion_execution::TaskContext; +use datafusion_physical_expr::{EquivalenceProperties, PhysicalExpr}; +use datafusion_physical_expr_common::sort_expr::LexOrdering; + +use datafusion_common::HashMap; +use futures::stream::Stream; +use futures::{FutureExt, StreamExt, TryStreamExt}; +use log::trace; +use parking_lot::Mutex; + +/// Inner state of [`RepartitionExec`]. +#[derive(Debug)] +struct OnDemandRepartitionExecState { + /// Channels for sending batches from input partitions to output partitions. + /// Key is the partition number. + channels: HashMap< + usize, + ( + InputPartitionsToCurrentPartitionSender, + InputPartitionsToCurrentPartitionReceiver, + SharedMemoryReservation, + ), + >, + + /// Helper that ensures that that background job is killed once it is no longer needed. + abort_helper: Arc>>, +} + +impl OnDemandRepartitionExecState { + fn new( + input: Arc, + partitioning: Partitioning, + metrics: ExecutionPlanMetricsSet, + preserve_order: bool, + name: String, + context: Arc, + ) -> Self { + let num_input_partitions = input.output_partitioning().partition_count(); + let num_output_partitions = partitioning.partition_count(); + + let (txs, rxs) = if preserve_order { + let (txs, rxs) = + partition_aware_channels(num_input_partitions, num_output_partitions); + // Take transpose of senders and receivers. `state.channels` keeps track of entries per output partition + let txs = transpose(txs); + let rxs = transpose(rxs); + (txs, rxs) + } else { + // create one channel per *output* partition + // note we use a custom channel that ensures there is always data for each receiver + // but limits the amount of buffering if required. + let (txs, rxs) = channels(num_output_partitions); + // Clone sender for each input partitions + let txs = txs + .into_iter() + .map(|item| vec![item; num_input_partitions]) + .collect::>(); + let rxs = rxs.into_iter().map(|item| vec![item]).collect::>(); + (txs, rxs) + }; + + let mut channels = HashMap::with_capacity(txs.len()); + for (partition, (tx, rx)) in txs.into_iter().zip(rxs).enumerate() { + let reservation = Arc::new(Mutex::new( + MemoryConsumer::new(format!("{}[{partition}]", name)) + .register(context.memory_pool()), + )); + channels.insert(partition, (tx, rx, reservation)); + } + + // launch one async task per *input* partition + let mut spawned_tasks = Vec::with_capacity(num_input_partitions); + for i in 0..num_input_partitions { + let txs: HashMap<_, _> = channels + .iter() + .map(|(partition, (tx, _rx, reservation))| { + (*partition, (tx[i].clone(), Arc::clone(reservation))) + }) + .collect(); + + let r_metrics = RepartitionMetrics::new(i, num_output_partitions, &metrics); + + let input_task = + SpawnedTask::spawn(OnDemandRepartitionExec::pull_from_input( + Arc::clone(&input), + i, + txs.clone(), + partitioning.clone(), + r_metrics, + Arc::clone(&context), + )); + + // In a separate task, wait for each input to be done + // (and pass along any errors, including panic!s) + let wait_for_task = + SpawnedTask::spawn(OnDemandRepartitionExec::wait_for_task( + input_task, + txs.into_iter() + .map(|(partition, (tx, _reservation))| (partition, tx)) + .collect(), + )); + spawned_tasks.push(wait_for_task); + } + + Self { + channels, + abort_helper: Arc::new(spawned_tasks), + } + } +} + +/// Lazily initialized state +/// +/// Note that the state is initialized ONCE for all partitions by a single task(thread). +/// This may take a short while. It is also like that multiple threads +/// call execute at the same time, because we have just started "target partitions" tasks +/// which is commonly set to the number of CPU cores and all call execute at the same time. +/// +/// Thus, use a **tokio** `OnceCell` for this initialization so as not to waste CPU cycles +/// in a futex lock but instead allow other threads to do something useful. +/// +/// Uses a parking_lot `Mutex` to control other accesses as they are very short duration +/// (e.g. removing channels on completion) where the overhead of `await` is not warranted. +type LazyState = Arc>>; + +#[derive(Debug, Clone)] +pub struct OnDemandRepartitionExec { + /// Input execution plan + input: Arc, + /// Inner state that is initialized when the first output stream is created. + state: LazyState, + /// Execution metrics + metrics: ExecutionPlanMetricsSet, + /// Boolean flag to decide whether to preserve ordering. If true means + /// `SortPreservingRepartitionExec`, false means `RepartitionExec`. + preserve_order: bool, + /// Cache holding plan properties like equivalences, output partitioning etc. + cache: PlanProperties, +} + +impl OnDemandRepartitionExec { + /// Input execution plan + pub fn input(&self) -> &Arc { + &self.input + } + + /// Partitioning scheme to use + pub fn partitioning(&self) -> &Partitioning { + &self.cache.partitioning + } + + /// Get preserve_order flag of the RepartitionExecutor + /// `true` means `SortPreservingRepartitionExec`, `false` means `RepartitionExec` + pub fn preserve_order(&self) -> bool { + self.preserve_order + } + + /// Get name used to display this Exec + pub fn name(&self) -> &str { + "OnDemandRepartitionExec" + } +} + +impl DisplayAs for OnDemandRepartitionExec { + fn fmt_as( + &self, + t: DisplayFormatType, + f: &mut std::fmt::Formatter, + ) -> std::fmt::Result { + match t { + DisplayFormatType::Default | DisplayFormatType::Verbose => { + write!( + f, + "{}: partitioning={}, input_partitions={}", + self.name(), + self.partitioning(), + self.input.output_partitioning().partition_count() + )?; + + if self.preserve_order { + write!(f, ", preserve_order=true")?; + } + + if let Some(sort_exprs) = self.sort_exprs() { + write!(f, ", sort_exprs={}", sort_exprs.clone())?; + } + Ok(()) + } + } + } +} + +impl ExecutionPlan for OnDemandRepartitionExec { + fn name(&self) -> &'static str { + "OnDemandRepartitionExec" + } + + /// Return a reference to Any that can be used for downcasting + fn as_any(&self) -> &dyn Any { + self + } + + fn properties(&self) -> &PlanProperties { + &self.cache + } + + fn children(&self) -> Vec<&Arc> { + vec![&self.input] + } + + fn with_new_children( + self: Arc, + mut children: Vec>, + ) -> Result> { + let mut repartition = OnDemandRepartitionExec::try_new( + children.swap_remove(0), + self.partitioning().clone(), + )?; + if self.preserve_order { + repartition = repartition.with_preserve_order(); + } + Ok(Arc::new(repartition)) + } + + fn benefits_from_input_partitioning(&self) -> Vec { + vec![matches!(self.partitioning(), Partitioning::Hash(_, _))] + } + + fn maintains_input_order(&self) -> Vec { + Self::maintains_input_order_helper(self.input(), self.preserve_order) + } + + fn execute( + &self, + partition: usize, + context: Arc, + ) -> Result { + trace!( + "Start {}::execute for partition: {}", + self.name(), + partition + ); + + let lazy_state = Arc::clone(&self.state); + let input = Arc::clone(&self.input); + let partitioning = self.partitioning().clone(); + let metrics = self.metrics.clone(); + let preserve_order = self.preserve_order; + let name = self.name().to_owned(); + let schema = self.schema(); + let schema_captured = Arc::clone(&schema); + + // Get existing ordering to use for merging + let sort_exprs = self.sort_exprs().cloned().unwrap_or_default(); + + let stream = futures::stream::once(async move { + let num_input_partitions = input.output_partitioning().partition_count(); + + let input_captured = Arc::clone(&input); + let metrics_captured = metrics.clone(); + let name_captured = name.clone(); + let context_captured = Arc::clone(&context); + let state = lazy_state + .get_or_init(|| async move { + Mutex::new(OnDemandRepartitionExecState::new( + input_captured, + partitioning, + metrics_captured, + preserve_order, + name_captured, + context_captured, + )) + }) + .await; + + // lock scope + let (mut rx, reservation, abort_helper) = { + // lock mutexes + let mut state = state.lock(); + + // now return stream for the specified *output* partition which will + // read from the channel + let (_tx, rx, reservation) = state + .channels + .remove(&partition) + .expect("partition not used yet"); + + (rx, reservation, Arc::clone(&state.abort_helper)) + }; + + trace!( + "Before returning stream in {}::execute for partition: {}", + name, + partition + ); + + if preserve_order { + // Store streams from all the input partitions: + let input_streams = rx + .into_iter() + .map(|receiver| { + Box::pin(PerPartitionStream { + schema: Arc::clone(&schema_captured), + receiver, + _drop_helper: Arc::clone(&abort_helper), + reservation: Arc::clone(&reservation), + }) as SendableRecordBatchStream + }) + .collect::>(); + // Note that receiver size (`rx.len()`) and `num_input_partitions` are same. + + // Merge streams (while preserving ordering) coming from + // input partitions to this partition: + let fetch = None; + let merge_reservation = + MemoryConsumer::new(format!("{}[Merge {partition}]", name)) + .register(context.memory_pool()); + StreamingMergeBuilder::new() + .with_streams(input_streams) + .with_schema(schema_captured) + .with_expressions(&sort_exprs) + .with_metrics(BaselineMetrics::new(&metrics, partition)) + .with_batch_size(context.session_config().batch_size()) + .with_fetch(fetch) + .with_reservation(merge_reservation) + .build() + } else { + Ok(Box::pin(OnDemandRepartitionStream { + num_input_partitions, + num_input_partitions_processed: 0, + schema: input.schema(), + input: rx.swap_remove(0), + _drop_helper: abort_helper, + reservation, + }) as SendableRecordBatchStream) + } + }) + .try_flatten(); + let stream = RecordBatchStreamAdapter::new(schema, stream); + Ok(Box::pin(stream)) + } + + fn metrics(&self) -> Option { + Some(self.metrics.clone_inner()) + } + + fn statistics(&self) -> Result { + self.input.statistics() + } + + fn cardinality_effect(&self) -> CardinalityEffect { + CardinalityEffect::Equal + } +} + +impl OnDemandRepartitionExec { + /// Create a new RepartitionExec, that produces output `partitioning`, and + /// does not preserve the order of the input (see [`Self::with_preserve_order`] + /// for more details) + pub fn try_new( + input: Arc, + partitioning: Partitioning, + ) -> Result { + let preserve_order = false; + let cache = + Self::compute_properties(&input, partitioning.clone(), preserve_order); + Ok(OnDemandRepartitionExec { + input, + state: Default::default(), + metrics: ExecutionPlanMetricsSet::new(), + preserve_order, + cache, + }) + } + + fn maintains_input_order_helper( + input: &Arc, + preserve_order: bool, + ) -> Vec { + // We preserve ordering when repartition is order preserving variant or input partitioning is 1 + vec![preserve_order || input.output_partitioning().partition_count() <= 1] + } + + fn eq_properties_helper( + input: &Arc, + preserve_order: bool, + ) -> EquivalenceProperties { + // Equivalence Properties + let mut eq_properties = input.equivalence_properties().clone(); + // If the ordering is lost, reset the ordering equivalence class: + if !Self::maintains_input_order_helper(input, preserve_order)[0] { + eq_properties.clear_orderings(); + } + // When there are more than one input partitions, they will be fused at the output. + // Therefore, remove per partition constants. + if input.output_partitioning().partition_count() > 1 { + eq_properties.clear_per_partition_constants(); + } + eq_properties + } + + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. + fn compute_properties( + input: &Arc, + partitioning: Partitioning, + preserve_order: bool, + ) -> PlanProperties { + // Equivalence Properties + let eq_properties = Self::eq_properties_helper(input, preserve_order); + + PlanProperties::new( + eq_properties, // Equivalence Properties + partitioning, // Output Partitioning + input.execution_mode(), // Execution Mode + ) + } + + /// Specify if this reparititoning operation should preserve the order of + /// rows from its input when producing output. Preserving order is more + /// expensive at runtime, so should only be set if the output of this + /// operator can take advantage of it. + /// + /// If the input is not ordered, or has only one partition, this is a no op, + /// and the node remains a `RepartitionExec`. + pub fn with_preserve_order(mut self) -> Self { + self.preserve_order = + // If the input isn't ordered, there is no ordering to preserve + self.input.output_ordering().is_some() && + // if there is only one input partition, merging is not required + // to maintain order + self.input.output_partitioning().partition_count() > 1; + let eq_properties = Self::eq_properties_helper(&self.input, self.preserve_order); + self.cache = self.cache.with_eq_properties(eq_properties); + self + } + + /// Return the sort expressions that are used to merge + fn sort_exprs(&self) -> Option<&LexOrdering> { + if self.preserve_order { + self.input.output_ordering() + } else { + None + } + } + + /// Pulls data from the specified input plan, feeding it to the + /// output partitions based on the desired partitioning + /// + /// txs hold the output sending channels for each output partition + async fn pull_from_input( + input: Arc, + partition: usize, + mut output_channels: HashMap< + usize, + (DistributionSender, SharedMemoryReservation), + >, + partitioning: Partitioning, + metrics: RepartitionMetrics, + context: Arc, + ) -> Result<()> { + let mut partitioner = + BatchPartitioner::try_new(partitioning, metrics.repartition_time.clone())?; + + // execute the child operator + let timer = metrics.fetch_time.timer(); + let mut stream = input.execute(partition, context)?; + timer.done(); + + // While there are still outputs to send to, keep pulling inputs + let mut batches_until_yield = partitioner.num_partitions(); + while !output_channels.is_empty() { + // fetch the next batch + let timer = metrics.fetch_time.timer(); + let result = stream.next().await; + timer.done(); + + // Input is done + let batch = match result { + Some(result) => result?, + None => break, + }; + + for res in partitioner.partition_iter(batch)? { + let (partition, batch) = res?; + let size = batch.get_array_memory_size(); + + let timer = metrics.send_time[partition].timer(); + // if there is still a receiver, send to it + if let Some((tx, reservation)) = output_channels.get_mut(&partition) { + reservation.lock().try_grow(size)?; + + if tx.send(Some(Ok(batch))).await.is_err() { + // If the other end has hung up, it was an early shutdown (e.g. LIMIT) + reservation.lock().shrink(size); + output_channels.remove(&partition); + } + } + timer.done(); + } + + // If the input stream is endless, we may spin forever and + // never yield back to tokio. See + // https://github.com/apache/datafusion/issues/5278. + // + // However, yielding on every batch causes a bottleneck + // when running with multiple cores. See + // https://github.com/apache/datafusion/issues/6290 + // + // Thus, heuristically yield after producing num_partition + // batches + // + // In round robin this is ideal as each input will get a + // new batch. In hash partitioning it may yield too often + // on uneven distributions even if some partition can not + // make progress, but parallelism is going to be limited + // in that case anyways + if batches_until_yield == 0 { + tokio::task::yield_now().await; + batches_until_yield = partitioner.num_partitions(); + } else { + batches_until_yield -= 1; + } + } + + Ok(()) + } + + /// Waits for `input_task` which is consuming one of the inputs to + /// complete. Upon each successful completion, sends a `None` to + /// each of the output tx channels to signal one of the inputs is + /// complete. Upon error, propagates the errors to all output tx + /// channels. + async fn wait_for_task( + input_task: SpawnedTask>, + txs: HashMap>, + ) { + // wait for completion, and propagate error + // note we ignore errors on send (.ok) as that means the receiver has already shutdown. + + match input_task.join().await { + // Error in joining task + Err(e) => { + let e = Arc::new(e); + + for (_, tx) in txs { + let err = Err(DataFusionError::Context( + "Join Error".to_string(), + Box::new(DataFusionError::External(Box::new(Arc::clone(&e)))), + )); + tx.send(Some(err)).await.ok(); + } + } + // Error from running input task + Ok(Err(e)) => { + let e = Arc::new(e); + + for (_, tx) in txs { + // wrap it because need to send error to all output partitions + let err = Err(DataFusionError::External(Box::new(Arc::clone(&e)))); + tx.send(Some(err)).await.ok(); + } + } + // Input task completed successfully + Ok(Ok(())) => { + // notify each output partition that this input partition has no more data + for (_, tx) in txs { + tx.send(None).await.ok(); + } + } + } + } +} + +struct OnDemandRepartitionStream { + /// Number of input partitions that will be sending batches to this output channel + num_input_partitions: usize, + + /// Number of input partitions that have finished sending batches to this output channel + num_input_partitions_processed: usize, + + /// Schema wrapped by Arc + schema: SchemaRef, + + /// channel containing the repartitioned batches + input: DistributionReceiver, + + /// Handle to ensure background tasks are killed when no longer needed. + _drop_helper: Arc>>, + + /// Memory reservation. + reservation: SharedMemoryReservation, +} + +impl Stream for OnDemandRepartitionStream { + type Item = Result; + + fn poll_next( + mut self: Pin<&mut Self>, + cx: &mut Context<'_>, + ) -> Poll> { + loop { + match self.input.recv().poll_unpin(cx) { + Poll::Ready(Some(Some(v))) => { + if let Ok(batch) = &v { + self.reservation + .lock() + .shrink(batch.get_array_memory_size()); + } + + return Poll::Ready(Some(v)); + } + Poll::Ready(Some(None)) => { + self.num_input_partitions_processed += 1; + + if self.num_input_partitions == self.num_input_partitions_processed { + // all input partitions have finished sending batches + return Poll::Ready(None); + } else { + // other partitions still have data to send + continue; + } + } + Poll::Ready(None) => { + return Poll::Ready(None); + } + Poll::Pending => { + return Poll::Pending; + } + } + } + } +} + +impl RecordBatchStream for OnDemandRepartitionStream { + /// Get the schema + fn schema(&self) -> SchemaRef { + Arc::clone(&self.schema) + } +} + +#[cfg(test)] +mod tests { + + use super::*; + use crate::{ + test::{ + assert_is_pending, + exec::{ + assert_strong_count_converges_to_zero, BarrierExec, BlockingExec, + ErrorExec, MockExec, + }, + }, + {collect, expressions::col, memory::MemoryExec}, + }; + + use arrow::array::{ArrayRef, StringArray, UInt32Array}; + use arrow::datatypes::{DataType, Field, Schema}; + use datafusion_common::cast::as_string_array; + use datafusion_common::{arrow_datafusion_err, assert_batches_sorted_eq, exec_err}; + use datafusion_execution::runtime_env::RuntimeEnvBuilder; + + #[tokio::test] + async fn one_to_many_round_robin() -> Result<()> { + // define input partitions + let schema = test_schema(); + let partition = create_vec_batches(50); + let partitions = vec![partition]; + + // repartition from 1 input to 4 output + let output_partitions = + repartition(&schema, partitions, Partitioning::RoundRobinBatch(4)).await?; + + assert_eq!(4, output_partitions.len()); + assert_eq!(13, output_partitions[0].len()); + assert_eq!(13, output_partitions[1].len()); + assert_eq!(12, output_partitions[2].len()); + assert_eq!(12, output_partitions[3].len()); + + Ok(()) + } + + #[tokio::test] + async fn many_to_one_round_robin() -> Result<()> { + // define input partitions + let schema = test_schema(); + let partition = create_vec_batches(50); + let partitions = vec![partition.clone(), partition.clone(), partition.clone()]; + + // repartition from 3 input to 1 output + let output_partitions = + repartition(&schema, partitions, Partitioning::RoundRobinBatch(1)).await?; + + assert_eq!(1, output_partitions.len()); + assert_eq!(150, output_partitions[0].len()); + + Ok(()) + } + + #[tokio::test] + async fn many_to_many_round_robin() -> Result<()> { + // define input partitions + let schema = test_schema(); + let partition = create_vec_batches(50); + let partitions = vec![partition.clone(), partition.clone(), partition.clone()]; + + // repartition from 3 input to 5 output + let output_partitions = + repartition(&schema, partitions, Partitioning::RoundRobinBatch(5)).await?; + + assert_eq!(5, output_partitions.len()); + assert_eq!(30, output_partitions[0].len()); + assert_eq!(30, output_partitions[1].len()); + assert_eq!(30, output_partitions[2].len()); + assert_eq!(30, output_partitions[3].len()); + assert_eq!(30, output_partitions[4].len()); + + Ok(()) + } + + #[tokio::test] + async fn many_to_many_hash_partition() -> Result<()> { + // define input partitions + let schema = test_schema(); + let partition = create_vec_batches(50); + let partitions = vec![partition.clone(), partition.clone(), partition.clone()]; + + let output_partitions = repartition( + &schema, + partitions, + Partitioning::Hash(vec![col("c0", &schema)?], 8), + ) + .await?; + + let total_rows: usize = output_partitions + .iter() + .map(|x| x.iter().map(|x| x.num_rows()).sum::()) + .sum(); + + assert_eq!(8, output_partitions.len()); + assert_eq!(total_rows, 8 * 50 * 3); + + Ok(()) + } + + fn test_schema() -> Arc { + Arc::new(Schema::new(vec![Field::new("c0", DataType::UInt32, false)])) + } + + async fn repartition( + schema: &SchemaRef, + input_partitions: Vec>, + partitioning: Partitioning, + ) -> Result>> { + let task_ctx = Arc::new(TaskContext::default()); + // create physical plan + let exec = MemoryExec::try_new(&input_partitions, Arc::clone(schema), None)?; + let exec = OnDemandRepartitionExec::try_new(Arc::new(exec), partitioning)?; + + // execute and collect results + let mut output_partitions = vec![]; + for i in 0..exec.partitioning().partition_count() { + // execute this *output* partition and collect all batches + let mut stream = exec.execute(i, Arc::clone(&task_ctx))?; + let mut batches = vec![]; + while let Some(result) = stream.next().await { + batches.push(result?); + } + output_partitions.push(batches); + } + Ok(output_partitions) + } + + #[tokio::test] + async fn unsupported_partitioning() { + let task_ctx = Arc::new(TaskContext::default()); + // have to send at least one batch through to provoke error + let batch = RecordBatch::try_from_iter(vec![( + "my_awesome_field", + Arc::new(StringArray::from(vec!["foo", "bar"])) as ArrayRef, + )]) + .unwrap(); + + let schema = batch.schema(); + let input = MockExec::new(vec![Ok(batch)], schema); + // This generates an error (partitioning type not supported) + // but only after the plan is executed. The error should be + // returned and no results produced + let partitioning = Partitioning::UnknownPartitioning(1); + let exec = + OnDemandRepartitionExec::try_new(Arc::new(input), partitioning).unwrap(); + let output_stream = exec.execute(0, task_ctx).unwrap(); + + // Expect that an error is returned + let result_string = crate::common::collect(output_stream) + .await + .unwrap_err() + .to_string(); + assert!( + result_string + .contains("Unsupported repartitioning scheme UnknownPartitioning(1)"), + "actual: {result_string}" + ); + } + + #[tokio::test] + async fn error_for_input_exec() { + // This generates an error on a call to execute. The error + // should be returned and no results produced. + + let task_ctx = Arc::new(TaskContext::default()); + let input = ErrorExec::new(); + let partitioning = Partitioning::RoundRobinBatch(1); + let exec = + OnDemandRepartitionExec::try_new(Arc::new(input), partitioning).unwrap(); + + // Note: this should pass (the stream can be created) but the + // error when the input is executed should get passed back + let output_stream = exec.execute(0, task_ctx).unwrap(); + + // Expect that an error is returned + let result_string = crate::common::collect(output_stream) + .await + .unwrap_err() + .to_string(); + assert!( + result_string.contains("ErrorExec, unsurprisingly, errored in partition 0"), + "actual: {result_string}" + ); + } + + #[tokio::test] + async fn repartition_with_error_in_stream() { + let task_ctx = Arc::new(TaskContext::default()); + let batch = RecordBatch::try_from_iter(vec![( + "my_awesome_field", + Arc::new(StringArray::from(vec!["foo", "bar"])) as ArrayRef, + )]) + .unwrap(); + + // input stream returns one good batch and then one error. The + // error should be returned. + let err = exec_err!("bad data error"); + + let schema = batch.schema(); + let input = MockExec::new(vec![Ok(batch), err], schema); + let partitioning = Partitioning::RoundRobinBatch(1); + let exec = + OnDemandRepartitionExec::try_new(Arc::new(input), partitioning).unwrap(); + + // Note: this should pass (the stream can be created) but the + // error when the input is executed should get passed back + let output_stream = exec.execute(0, task_ctx).unwrap(); + + // Expect that an error is returned + let result_string = crate::common::collect(output_stream) + .await + .unwrap_err() + .to_string(); + assert!( + result_string.contains("bad data error"), + "actual: {result_string}" + ); + } + + #[tokio::test] + async fn repartition_with_delayed_stream() { + let task_ctx = Arc::new(TaskContext::default()); + let batch1 = RecordBatch::try_from_iter(vec![( + "my_awesome_field", + Arc::new(StringArray::from(vec!["foo", "bar"])) as ArrayRef, + )]) + .unwrap(); + + let batch2 = RecordBatch::try_from_iter(vec![( + "my_awesome_field", + Arc::new(StringArray::from(vec!["frob", "baz"])) as ArrayRef, + )]) + .unwrap(); + + // The mock exec doesn't return immediately (instead it + // requires the input to wait at least once) + let schema = batch1.schema(); + let expected_batches = vec![batch1.clone(), batch2.clone()]; + let input = MockExec::new(vec![Ok(batch1), Ok(batch2)], schema); + let partitioning = Partitioning::RoundRobinBatch(1); + + let exec = + OnDemandRepartitionExec::try_new(Arc::new(input), partitioning).unwrap(); + + let expected = vec![ + "+------------------+", + "| my_awesome_field |", + "+------------------+", + "| foo |", + "| bar |", + "| frob |", + "| baz |", + "+------------------+", + ]; + + assert_batches_sorted_eq!(&expected, &expected_batches); + + let output_stream = exec.execute(0, task_ctx).unwrap(); + let batches = crate::common::collect(output_stream).await.unwrap(); + + assert_batches_sorted_eq!(&expected, &batches); + } + + fn str_batches_to_vec(batches: &[RecordBatch]) -> Vec<&str> { + batches + .iter() + .flat_map(|batch| { + assert_eq!(batch.columns().len(), 1); + let string_array = as_string_array(batch.column(0)) + .expect("Unexpected type for repartitoned batch"); + + string_array + .iter() + .map(|v| v.expect("Unexpected null")) + .collect::>() + }) + .collect::>() + } + + /// Create a BarrierExec that returns two partitions of two batches each + fn make_barrier_exec() -> BarrierExec { + let batch1 = RecordBatch::try_from_iter(vec![( + "my_awesome_field", + Arc::new(StringArray::from(vec!["foo", "bar"])) as ArrayRef, + )]) + .unwrap(); + + let batch2 = RecordBatch::try_from_iter(vec![( + "my_awesome_field", + Arc::new(StringArray::from(vec!["frob", "baz"])) as ArrayRef, + )]) + .unwrap(); + + let batch3 = RecordBatch::try_from_iter(vec![( + "my_awesome_field", + Arc::new(StringArray::from(vec!["goo", "gar"])) as ArrayRef, + )]) + .unwrap(); + + let batch4 = RecordBatch::try_from_iter(vec![( + "my_awesome_field", + Arc::new(StringArray::from(vec!["grob", "gaz"])) as ArrayRef, + )]) + .unwrap(); + + // The barrier exec waits to be pinged + // requires the input to wait at least once) + let schema = batch1.schema(); + BarrierExec::new(vec![vec![batch1, batch2], vec![batch3, batch4]], schema) + } + + #[tokio::test] + async fn test_drop_cancel() -> Result<()> { + let task_ctx = Arc::new(TaskContext::default()); + let schema = + Arc::new(Schema::new(vec![Field::new("a", DataType::Float32, true)])); + + let blocking_exec = Arc::new(BlockingExec::new(Arc::clone(&schema), 2)); + let refs = blocking_exec.refs(); + let repartition_exec = Arc::new(OnDemandRepartitionExec::try_new( + blocking_exec, + Partitioning::UnknownPartitioning(1), + )?); + + let fut = collect(repartition_exec, task_ctx); + let mut fut = fut.boxed(); + + assert_is_pending(&mut fut); + drop(fut); + assert_strong_count_converges_to_zero(refs).await; + + Ok(()) + } + + #[tokio::test] + async fn oom() -> Result<()> { + // define input partitions + let schema = test_schema(); + let partition = create_vec_batches(50); + let input_partitions = vec![partition]; + let partitioning = Partitioning::RoundRobinBatch(4); + + // setup up context + let runtime = RuntimeEnvBuilder::default() + .with_memory_limit(1, 1.0) + .build_arc()?; + + let task_ctx = TaskContext::default().with_runtime(runtime); + let task_ctx = Arc::new(task_ctx); + + // create physical plan + let exec = MemoryExec::try_new(&input_partitions, Arc::clone(&schema), None)?; + let exec = OnDemandRepartitionExec::try_new(Arc::new(exec), partitioning)?; + + // pull partitions + for i in 0..exec.partitioning().partition_count() { + let mut stream = exec.execute(i, Arc::clone(&task_ctx))?; + let err = + arrow_datafusion_err!(stream.next().await.unwrap().unwrap_err().into()); + let err = err.find_root(); + assert!( + matches!(err, DataFusionError::ResourcesExhausted(_)), + "Wrong error type: {err}", + ); + } + + Ok(()) + } + + /// Create vector batches + fn create_vec_batches(n: usize) -> Vec { + let batch = create_batch(); + (0..n).map(|_| batch.clone()).collect() + } + + /// Create batch + fn create_batch() -> RecordBatch { + let schema = test_schema(); + RecordBatch::try_new( + schema, + vec![Arc::new(UInt32Array::from(vec![1, 2, 3, 4, 5, 6, 7, 8]))], + ) + .unwrap() + } +} + +#[cfg(test)] +mod test { + use arrow_schema::{DataType, Field, Schema, SortOptions}; + + use crate::memory::MemoryExec; + use crate::union::UnionExec; + use datafusion_physical_expr::expressions::col; + use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; + + use super::*; + + /// Asserts that the plan is as expected + /// + /// `$EXPECTED_PLAN_LINES`: input plan + /// `$PLAN`: the plan to optimized + /// + macro_rules! assert_plan { + ($EXPECTED_PLAN_LINES: expr, $PLAN: expr) => { + let physical_plan = $PLAN; + let formatted = crate::displayable(&physical_plan).indent(true).to_string(); + let actual: Vec<&str> = formatted.trim().lines().collect(); + + let expected_plan_lines: Vec<&str> = $EXPECTED_PLAN_LINES + .iter().map(|s| *s).collect(); + + assert_eq!( + expected_plan_lines, actual, + "\n**Original Plan Mismatch\n\nexpected:\n\n{expected_plan_lines:#?}\nactual:\n\n{actual:#?}\n\n" + ); + }; + } + + #[tokio::test] + async fn test_preserve_order() -> Result<()> { + let schema = test_schema(); + let sort_exprs = sort_exprs(&schema); + let source1 = sorted_memory_exec(&schema, sort_exprs.clone()); + let source2 = sorted_memory_exec(&schema, sort_exprs); + // output has multiple partitions, and is sorted + let union = UnionExec::new(vec![source1, source2]); + let exec = OnDemandRepartitionExec::try_new( + Arc::new(union), + Partitioning::RoundRobinBatch(10), + ) + .unwrap() + .with_preserve_order(); + + // Repartition should preserve order + let expected_plan = [ + "OnDemandRepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2, preserve_order=true, sort_exprs=c0@0 ASC", + " UnionExec", + " MemoryExec: partitions=1, partition_sizes=[0], output_ordering=c0@0 ASC", + " MemoryExec: partitions=1, partition_sizes=[0], output_ordering=c0@0 ASC", + ]; + assert_plan!(expected_plan, exec); + Ok(()) + } + + #[tokio::test] + async fn test_preserve_order_one_partition() -> Result<()> { + let schema = test_schema(); + let sort_exprs = sort_exprs(&schema); + let source = sorted_memory_exec(&schema, sort_exprs); + // output is sorted, but has only a single partition, so no need to sort + let exec = + OnDemandRepartitionExec::try_new(source, Partitioning::RoundRobinBatch(10)) + .unwrap() + .with_preserve_order(); + + // Repartition should not preserve order + let expected_plan = [ + "OnDemandRepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " MemoryExec: partitions=1, partition_sizes=[0], output_ordering=c0@0 ASC", + ]; + assert_plan!(expected_plan, exec); + Ok(()) + } + + #[tokio::test] + async fn test_preserve_order_input_not_sorted() -> Result<()> { + let schema = test_schema(); + let source1 = memory_exec(&schema); + let source2 = memory_exec(&schema); + // output has multiple partitions, but is not sorted + let union = UnionExec::new(vec![source1, source2]); + let exec = OnDemandRepartitionExec::try_new( + Arc::new(union), + Partitioning::RoundRobinBatch(10), + ) + .unwrap() + .with_preserve_order(); + + // Repartition should not preserve order, as there is no order to preserve + let expected_plan = [ + "OnDemandRepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", + " UnionExec", + " MemoryExec: partitions=1, partition_sizes=[0]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + assert_plan!(expected_plan, exec); + Ok(()) + } + + fn test_schema() -> Arc { + Arc::new(Schema::new(vec![Field::new("c0", DataType::UInt32, false)])) + } + + fn sort_exprs(schema: &Schema) -> LexOrdering { + let options = SortOptions::default(); + LexOrdering::new(vec![PhysicalSortExpr { + expr: col("c0", schema).unwrap(), + options, + }]) + } + + fn memory_exec(schema: &SchemaRef) -> Arc { + Arc::new(MemoryExec::try_new(&[vec![]], Arc::clone(schema), None).unwrap()) + } + + fn sorted_memory_exec( + schema: &SchemaRef, + sort_exprs: LexOrdering, + ) -> Arc { + Arc::new( + MemoryExec::try_new(&[vec![]], Arc::clone(schema), None) + .unwrap() + .try_with_sort_information(vec![sort_exprs]) + .unwrap(), + ) + } +} From 69865e312715d88f8090b5a78172dafc12075915 Mon Sep 17 00:00:00 2001 From: Weijun-H Date: Thu, 26 Dec 2024 21:40:31 +0200 Subject: [PATCH 05/25] feat: Support OnDemandRepartitionExec --- datafusion/physical-expr/src/partitioning.rs | 5 +- datafusion/physical-plan/Cargo.toml | 1 + .../physical-plan/src/repartition/mod.rs | 11 +- .../src/repartition/on_demand_repartition.rs | 290 +++++++++++++----- 4 files changed, 221 insertions(+), 86 deletions(-) diff --git a/datafusion/physical-expr/src/partitioning.rs b/datafusion/physical-expr/src/partitioning.rs index eb7e1ea6282bb..2814470485a16 100644 --- a/datafusion/physical-expr/src/partitioning.rs +++ b/datafusion/physical-expr/src/partitioning.rs @@ -119,6 +119,8 @@ pub enum Partitioning { Hash(Vec>, usize), /// Unknown partitioning scheme with a known number of partitions UnknownPartitioning(usize), + /// On Demand partitioning, where the partitioning is determined at runtime + OnDemand(usize), } impl Display for Partitioning { @@ -136,6 +138,7 @@ impl Display for Partitioning { Partitioning::UnknownPartitioning(size) => { write!(f, "UnknownPartitioning({size})") } + Partitioning::OnDemand(size) => write!(f, "OnDemand({size})"), } } } @@ -144,7 +147,7 @@ impl Partitioning { pub fn partition_count(&self) -> usize { use Partitioning::*; match self { - RoundRobinBatch(n) | Hash(_, n) | UnknownPartitioning(n) => *n, + RoundRobinBatch(n) | Hash(_, n) | UnknownPartitioning(n) | OnDemand(n) => *n, } } diff --git a/datafusion/physical-plan/Cargo.toml b/datafusion/physical-plan/Cargo.toml index 4a10398e5a9ef..8192f3625e010 100644 --- a/datafusion/physical-plan/Cargo.toml +++ b/datafusion/physical-plan/Cargo.toml @@ -45,6 +45,7 @@ ahash = { workspace = true } arrow = { workspace = true } arrow-ord = { workspace = true } arrow-schema = { workspace = true } +async-channel = "2.3.1" async-trait = { workspace = true } chrono = { workspace = true } datafusion-common = { workspace = true, default-features = true } diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 546527beee446..b7d07d4f8e695 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -60,7 +60,6 @@ use parking_lot::Mutex; mod distributor_channels; mod on_demand_repartition; - type MaybeBatch = Option>; type InputPartitionsToCurrentPartitionSender = Vec>; type InputPartitionsToCurrentPartitionReceiver = Vec>; @@ -195,6 +194,9 @@ enum BatchPartitionerState { num_partitions: usize, next_idx: usize, }, + OnDemand { + num_partitions: usize, + }, } impl BatchPartitioner { @@ -216,6 +218,9 @@ impl BatchPartitioner { random_state: ahash::RandomState::with_seeds(0, 0, 0, 0), hash_buffer: vec![], }, + Partitioning::OnDemand(num_partitions) => { + BatchPartitionerState::OnDemand { num_partitions } + } other => return not_impl_err!("Unsupported repartitioning scheme {other:?}"), }; @@ -260,6 +265,9 @@ impl BatchPartitioner { *next_idx = (*next_idx + 1) % *num_partitions; Box::new(std::iter::once(Ok((idx, batch)))) } + BatchPartitionerState::OnDemand { .. } => { + Box::new(std::iter::once(Ok((0, batch)))) + } BatchPartitionerState::Hash { random_state, exprs, @@ -330,6 +338,7 @@ impl BatchPartitioner { match self.state { BatchPartitionerState::RoundRobin { num_partitions, .. } => num_partitions, BatchPartitionerState::Hash { num_partitions, .. } => num_partitions, + BatchPartitionerState::OnDemand { num_partitions } => num_partitions, } } } diff --git a/datafusion/physical-plan/src/repartition/on_demand_repartition.rs b/datafusion/physical-plan/src/repartition/on_demand_repartition.rs index 354e82d926017..7595facdb36f9 100644 --- a/datafusion/physical-plan/src/repartition/on_demand_repartition.rs +++ b/datafusion/physical-plan/src/repartition/on_demand_repartition.rs @@ -24,7 +24,7 @@ use std::sync::Arc; use std::task::{Context, Poll}; use std::{any::Any, vec}; -use super::metrics::{self, ExecutionPlanMetricsSet, MetricBuilder, MetricsSet}; +use super::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use super::{ BatchPartitioner, DisplayAs, ExecutionPlanProperties, InputPartitionsToCurrentPartitionReceiver, InputPartitionsToCurrentPartitionSender, @@ -32,26 +32,23 @@ use super::{ }; use crate::common::SharedMemoryReservation; use crate::execution_plan::CardinalityEffect; -use crate::hash_utils::create_hashes; use crate::metrics::BaselineMetrics; use crate::repartition::distributor_channels::{ channels, partition_aware_channels, DistributionReceiver, DistributionSender, }; -use crate::repartition::PerPartitionStream; use crate::sorts::streaming_merge::StreamingMergeBuilder; use crate::stream::RecordBatchStreamAdapter; use crate::{DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, Statistics}; -use arrow::compute::take_arrays; -use arrow::datatypes::{SchemaRef, UInt32Type}; +use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; -use arrow_array::{PrimitiveArray, RecordBatchOptions}; +use async_channel::{Receiver, Sender}; use datafusion_common::utils::transpose; -use datafusion_common::{not_impl_err, DataFusionError, Result}; +use datafusion_common::{internal_datafusion_err, DataFusionError, Result}; use datafusion_common_runtime::SpawnedTask; use datafusion_execution::memory_pool::MemoryConsumer; use datafusion_execution::TaskContext; -use datafusion_physical_expr::{EquivalenceProperties, PhysicalExpr}; +use datafusion_physical_expr::EquivalenceProperties; use datafusion_physical_expr_common::sort_expr::LexOrdering; use datafusion_common::HashMap; @@ -86,6 +83,7 @@ impl OnDemandRepartitionExecState { preserve_order: bool, name: String, context: Arc, + partition_receiver: Receiver, ) -> Self { let num_input_partitions = input.output_partitioning().partition_count(); let num_output_partitions = partitioning.partition_count(); @@ -138,6 +136,7 @@ impl OnDemandRepartitionExecState { i, txs.clone(), partitioning.clone(), + partition_receiver.clone(), r_metrics, Arc::clone(&context), )); @@ -312,6 +311,7 @@ impl ExecutionPlan for OnDemandRepartitionExec { let metrics_captured = metrics.clone(); let name_captured = name.clone(); let context_captured = Arc::clone(&context); + let (partition_tx, patition_rx) = async_channel::unbounded(); let state = lazy_state .get_or_init(|| async move { Mutex::new(OnDemandRepartitionExecState::new( @@ -321,6 +321,7 @@ impl ExecutionPlan for OnDemandRepartitionExec { preserve_order, name_captured, context_captured, + patition_rx.clone(), )) }) .await; @@ -351,11 +352,13 @@ impl ExecutionPlan for OnDemandRepartitionExec { let input_streams = rx .into_iter() .map(|receiver| { - Box::pin(PerPartitionStream { + Box::pin(OnDemandPerPartitionStream { schema: Arc::clone(&schema_captured), receiver, _drop_helper: Arc::clone(&abort_helper), reservation: Arc::clone(&reservation), + sender: partition_tx.clone(), + partition, }) as SendableRecordBatchStream }) .collect::>(); @@ -384,6 +387,8 @@ impl ExecutionPlan for OnDemandRepartitionExec { input: rx.swap_remove(0), _drop_helper: abort_helper, reservation, + sender: partition_tx.clone(), + partition, }) as SendableRecordBatchStream) } }) @@ -507,6 +512,7 @@ impl OnDemandRepartitionExec { (DistributionSender, SharedMemoryReservation), >, partitioning: Partitioning, + output_partition_rx: Receiver, metrics: RepartitionMetrics, context: Arc, ) -> Result<()> { @@ -533,9 +539,16 @@ impl OnDemandRepartitionExec { }; for res in partitioner.partition_iter(batch)? { - let (partition, batch) = res?; + let (_, batch) = res?; let size = batch.get_array_memory_size(); + let partition = output_partition_rx.recv().await.map_err(|e| { + internal_datafusion_err!( + "Error receiving partition number from output partition: {}", + e + ) + })?; + let timer = metrics.send_time[partition].timer(); // if there is still a receiver, send to it if let Some((tx, reservation)) = output_channels.get_mut(&partition) { @@ -623,6 +636,70 @@ impl OnDemandRepartitionExec { } } +/// This struct converts a receiver to a stream. +/// Receiver receives data on an SPSC channel. +struct OnDemandPerPartitionStream { + /// Schema wrapped by Arc + schema: SchemaRef, + + /// channel containing the repartitioned batches + receiver: DistributionReceiver, + + /// Handle to ensure background tasks are killed when no longer needed. + _drop_helper: Arc>>, + + /// Memory reservation. + reservation: SharedMemoryReservation, + + /// Sender to send partititon number to the receiver + sender: Sender, + + /// Partition number + partition: usize, +} + +impl Stream for OnDemandPerPartitionStream { + type Item = Result; + + fn poll_next( + mut self: Pin<&mut Self>, + cx: &mut Context<'_>, + ) -> Poll> { + if !self.sender.is_closed() { + self.sender.send_blocking(self.partition).map_err(|e| { + internal_datafusion_err!( + "Error sending partition number to input partitions: {}", + e + ) + })?; + } + + match self.receiver.recv().poll_unpin(cx) { + Poll::Ready(Some(Some(v))) => { + if let Ok(batch) = &v { + self.reservation + .lock() + .shrink(batch.get_array_memory_size()); + } + Poll::Ready(Some(v)) + } + Poll::Ready(Some(None)) => { + // Input partition has finished sending batches + Poll::Ready(None) + } + Poll::Ready(None) => Poll::Ready(None), + Poll::Pending => Poll::Pending, + } + } +} + +impl RecordBatchStream for OnDemandPerPartitionStream { + /// Get the schema + fn schema(&self) -> SchemaRef { + Arc::clone(&self.schema) + } +} + struct OnDemandRepartitionStream { /// Number of input partitions that will be sending batches to this output channel num_input_partitions: usize, @@ -641,6 +718,12 @@ struct OnDemandRepartitionStream { /// Memory reservation. reservation: SharedMemoryReservation, + + /// Sender for the output partition + sender: Sender, + + /// Partition number + partition: usize, } impl Stream for OnDemandRepartitionStream { @@ -651,6 +734,16 @@ impl Stream for OnDemandRepartitionStream { cx: &mut Context<'_>, ) -> Poll> { loop { + // Send partition number to input partitions + if !self.sender.is_closed() { + self.sender.send_blocking(self.partition).map_err(|e| { + internal_datafusion_err!( + "Error sending partition number to input partitions: {}", + e + ) + })?; + } + match self.input.recv().poll_unpin(cx) { Poll::Ready(Some(Some(v))) => { if let Ok(batch) = &v { @@ -692,6 +785,7 @@ impl RecordBatchStream for OnDemandRepartitionStream { #[cfg(test)] mod tests { + use std::collections::HashSet; use super::*; use crate::{ @@ -702,7 +796,7 @@ mod tests { ErrorExec, MockExec, }, }, - {collect, expressions::col, memory::MemoryExec}, + {collect, memory::MemoryExec}, }; use arrow::array::{ArrayRef, StringArray, UInt32Array}; @@ -710,29 +804,38 @@ mod tests { use datafusion_common::cast::as_string_array; use datafusion_common::{arrow_datafusion_err, assert_batches_sorted_eq, exec_err}; use datafusion_execution::runtime_env::RuntimeEnvBuilder; + use tokio::task::JoinSet; - #[tokio::test] - async fn one_to_many_round_robin() -> Result<()> { - // define input partitions - let schema = test_schema(); - let partition = create_vec_batches(50); - let partitions = vec![partition]; - - // repartition from 1 input to 4 output - let output_partitions = - repartition(&schema, partitions, Partitioning::RoundRobinBatch(4)).await?; + fn test_schema() -> Arc { + Arc::new(Schema::new(vec![Field::new("c0", DataType::UInt32, false)])) + } - assert_eq!(4, output_partitions.len()); - assert_eq!(13, output_partitions[0].len()); - assert_eq!(13, output_partitions[1].len()); - assert_eq!(12, output_partitions[2].len()); - assert_eq!(12, output_partitions[3].len()); + async fn repartition( + schema: &SchemaRef, + input_partitions: Vec>, + partitioning: Partitioning, + ) -> Result>> { + let task_ctx = Arc::new(TaskContext::default()); + // create physical plan + let exec = MemoryExec::try_new(&input_partitions, Arc::clone(schema), None)?; + let exec = OnDemandRepartitionExec::try_new(Arc::new(exec), partitioning)?; - Ok(()) + // execute and collect results + let mut output_partitions = vec![]; + for i in 0..exec.partitioning().partition_count() { + // execute this *output* partition and collect all batches + let mut stream = exec.execute(i, Arc::clone(&task_ctx))?; + let mut batches = vec![]; + while let Some(result) = stream.next().await { + batches.push(result?); + } + output_partitions.push(batches); + } + Ok(output_partitions) } #[tokio::test] - async fn many_to_one_round_robin() -> Result<()> { + async fn many_to_one_on_demand() -> Result<()> { // define input partitions let schema = test_schema(); let partition = create_vec_batches(50); @@ -740,7 +843,7 @@ mod tests { // repartition from 3 input to 1 output let output_partitions = - repartition(&schema, partitions, Partitioning::RoundRobinBatch(1)).await?; + repartition(&schema, partitions, Partitioning::OnDemand(1)).await?; assert_eq!(1, output_partitions.len()); assert_eq!(150, output_partitions[0].len()); @@ -749,39 +852,14 @@ mod tests { } #[tokio::test] - async fn many_to_many_round_robin() -> Result<()> { + async fn many_to_many_on_demand() -> Result<()> { // define input partitions let schema = test_schema(); let partition = create_vec_batches(50); let partitions = vec![partition.clone(), partition.clone(), partition.clone()]; - // repartition from 3 input to 5 output let output_partitions = - repartition(&schema, partitions, Partitioning::RoundRobinBatch(5)).await?; - - assert_eq!(5, output_partitions.len()); - assert_eq!(30, output_partitions[0].len()); - assert_eq!(30, output_partitions[1].len()); - assert_eq!(30, output_partitions[2].len()); - assert_eq!(30, output_partitions[3].len()); - assert_eq!(30, output_partitions[4].len()); - - Ok(()) - } - - #[tokio::test] - async fn many_to_many_hash_partition() -> Result<()> { - // define input partitions - let schema = test_schema(); - let partition = create_vec_batches(50); - let partitions = vec![partition.clone(), partition.clone(), partition.clone()]; - - let output_partitions = repartition( - &schema, - partitions, - Partitioning::Hash(vec![col("c0", &schema)?], 8), - ) - .await?; + repartition(&schema, partitions, Partitioning::OnDemand(8)).await?; let total_rows: usize = output_partitions .iter() @@ -794,34 +872,6 @@ mod tests { Ok(()) } - fn test_schema() -> Arc { - Arc::new(Schema::new(vec![Field::new("c0", DataType::UInt32, false)])) - } - - async fn repartition( - schema: &SchemaRef, - input_partitions: Vec>, - partitioning: Partitioning, - ) -> Result>> { - let task_ctx = Arc::new(TaskContext::default()); - // create physical plan - let exec = MemoryExec::try_new(&input_partitions, Arc::clone(schema), None)?; - let exec = OnDemandRepartitionExec::try_new(Arc::new(exec), partitioning)?; - - // execute and collect results - let mut output_partitions = vec![]; - for i in 0..exec.partitioning().partition_count() { - // execute this *output* partition and collect all batches - let mut stream = exec.execute(i, Arc::clone(&task_ctx))?; - let mut batches = vec![]; - while let Some(result) = stream.next().await { - batches.push(result?); - } - output_partitions.push(batches); - } - Ok(output_partitions) - } - #[tokio::test] async fn unsupported_partitioning() { let task_ctx = Arc::new(TaskContext::default()); @@ -958,6 +1008,78 @@ mod tests { assert_batches_sorted_eq!(&expected, &batches); } + #[tokio::test] + async fn hash_repartition_avoid_empty_batch() -> Result<()> { + let task_ctx = Arc::new(TaskContext::default()); + let batch = RecordBatch::try_from_iter(vec![( + "a", + Arc::new(StringArray::from(vec!["foo"])) as ArrayRef, + )]) + .unwrap(); + let partitioning = Partitioning::OnDemand(2); + let schema = batch.schema(); + let input = MockExec::new(vec![Ok(batch)], schema); + let exec = + OnDemandRepartitionExec::try_new(Arc::new(input), partitioning).unwrap(); + let output_stream0 = exec.execute(0, Arc::clone(&task_ctx)).unwrap(); + let batch0 = crate::common::collect(output_stream0).await.unwrap(); + let output_stream1 = exec.execute(1, Arc::clone(&task_ctx)).unwrap(); + let batch1 = crate::common::collect(output_stream1).await.unwrap(); + assert!(batch0.is_empty() || batch1.is_empty()); + Ok(()) + } + + #[tokio::test] + async fn on_demand_repartition_with_dropping_output_stream() { + let task_ctx = Arc::new(TaskContext::default()); + let partitioning = Partitioning::OnDemand(2); + + // We first collect the results without dropping the output stream. + let input = Arc::new(make_barrier_exec()); + let exec = OnDemandRepartitionExec::try_new( + Arc::clone(&input) as Arc, + partitioning.clone(), + ) + .unwrap(); + let output_stream1 = exec.execute(1, Arc::clone(&task_ctx)).unwrap(); + let mut background_task = JoinSet::new(); + background_task.spawn(async move { + input.wait().await; + }); + let batches_without_drop = crate::common::collect(output_stream1).await.unwrap(); + + // run some checks on the result + let items_vec = str_batches_to_vec(&batches_without_drop); + let items_set: HashSet<&str> = items_vec.iter().copied().collect(); + assert_eq!(items_vec.len(), items_set.len()); + let source_str_set: HashSet<&str> = + ["foo", "bar", "frob", "baz", "goo", "gar", "grob", "gaz"] + .iter() + .copied() + .collect(); + assert_eq!(items_set.difference(&source_str_set).count(), 0); + + // Now do the same but dropping the stream before waiting for the barrier + let input = Arc::new(make_barrier_exec()); + let exec = OnDemandRepartitionExec::try_new( + Arc::clone(&input) as Arc, + partitioning, + ) + .unwrap(); + let output_stream0 = exec.execute(0, Arc::clone(&task_ctx)).unwrap(); + let output_stream1 = exec.execute(1, Arc::clone(&task_ctx)).unwrap(); + // now, purposely drop output stream 0 + // *before* any outputs are produced + drop(output_stream0); + let mut background_task = JoinSet::new(); + background_task.spawn(async move { + input.wait().await; + }); + let batches_with_drop = crate::common::collect(output_stream1).await.unwrap(); + + assert_eq!(batches_without_drop, batches_with_drop); + } + fn str_batches_to_vec(batches: &[RecordBatch]) -> Vec<&str> { batches .iter() From 22f6d401be1f9c6c1be239f7da3b177f1ba5fd4c Mon Sep 17 00:00:00 2001 From: Weijun-H Date: Sun, 29 Dec 2024 16:03:05 +0100 Subject: [PATCH 06/25] bench: add config to avoid OnDemandRepartitionExec instead of Round-Robin RepartitionExec --- Cargo.lock | 49 +++++++++++++++++++ datafusion/common/src/config.rs | 2 + .../physical-plan/src/repartition/mod.rs | 2 +- 3 files changed, 52 insertions(+), 1 deletion(-) diff --git a/Cargo.lock b/Cargo.lock index 716e0cf10386a..78e6ece14330c 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -514,6 +514,18 @@ dependencies = [ "wait-timeout", ] +[[package]] +name = "async-channel" +version = "2.3.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "89b47800b0be77592da0afd425cc03468052844aff33b84e33cc696f64e77b6a" +dependencies = [ + "concurrent-queue", + "event-listener-strategy", + "futures-core", + "pin-project-lite", +] + [[package]] name = "async-compression" version = "0.4.18" @@ -1418,6 +1430,15 @@ dependencies = [ "unicode-width 0.2.0", ] +[[package]] +name = "concurrent-queue" +version = "2.5.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4ca0197aee26d1ae37445ee532fefce43251d24cc7c166799f4d46817f1d3973" +dependencies = [ + "crossbeam-utils", +] + [[package]] name = "console" version = "0.15.10" @@ -2253,6 +2274,7 @@ dependencies = [ "arrow", "arrow-ord", "arrow-schema", + "async-channel", "async-trait", "chrono", "criterion", @@ -2603,6 +2625,27 @@ dependencies = [ "windows-sys 0.48.0", ] +[[package]] +name = "event-listener" +version = "5.4.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3492acde4c3fc54c845eaab3eed8bd00c7a7d881f78bfc801e43a93dec1331ae" +dependencies = [ + "concurrent-queue", + "parking", + "pin-project-lite", +] + +[[package]] +name = "event-listener-strategy" +version = "0.5.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3c3e4e0dd3673c1139bf041f3008816d9cf2946bbfac2945c09e523b8d7b05b2" +dependencies = [ + "event-listener", + "pin-project-lite", +] + [[package]] name = "fallible-iterator" version = "0.2.0" @@ -4031,6 +4074,12 @@ version = "4.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "fb37767f6569cd834a413442455e0f066d0d522de8630436e2a1761d9726ba56" +[[package]] +name = "parking" +version = "2.2.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f38d5652c16fde515bb1ecef450ab0f6a219d619a7274976324d5e377f7dceba" + [[package]] name = "parking_lot" version = "0.12.3" diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index 5e8317c081d97..09ef298c437cd 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -574,6 +574,8 @@ config_namespace! { /// repartitioning to increase parallelism to leverage more CPU cores pub enable_round_robin_repartition: bool, default = true + pub enable_on_demand_repartition: bool, default = true + /// When set to true, the optimizer will attempt to perform limit operations /// during aggregations, if possible pub enable_topk_aggregation: bool, default = true diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index b7d07d4f8e695..e9c1e3dadff00 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -58,7 +58,7 @@ use log::trace; use parking_lot::Mutex; mod distributor_channels; -mod on_demand_repartition; +pub mod on_demand_repartition; type MaybeBatch = Option>; type InputPartitionsToCurrentPartitionSender = Vec>; From cca6a69ffce055281397e26325007044d5528d63 Mon Sep 17 00:00:00 2001 From: Weijun-H Date: Mon, 30 Dec 2024 23:58:04 +0100 Subject: [PATCH 07/25] refactor: Use common structure to avoid duplication --- .../physical-plan/src/repartition/mod.rs | 430 ++++++++++++------ .../src/repartition/on_demand_repartition.rs | 321 +++---------- 2 files changed, 355 insertions(+), 396 deletions(-) diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index e9c1e3dadff00..e41f7f2d18620 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -43,6 +43,7 @@ use crate::{DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, Stat use arrow::array::{PrimitiveArray, RecordBatch, RecordBatchOptions}; use arrow::compute::take_arrays; use arrow::datatypes::{SchemaRef, UInt32Type}; +use async_channel::Receiver; use datafusion_common::utils::transpose; use datafusion_common::HashMap; use datafusion_common::{not_impl_err, DataFusionError, Result}; @@ -55,6 +56,7 @@ use datafusion_physical_expr_common::sort_expr::LexOrdering; use futures::stream::Stream; use futures::{FutureExt, StreamExt, TryStreamExt}; use log::trace; +use on_demand_repartition::OnDemandRepartitionExec; use parking_lot::Mutex; mod distributor_channels; @@ -64,6 +66,50 @@ type MaybeBatch = Option>; type InputPartitionsToCurrentPartitionSender = Vec>; type InputPartitionsToCurrentPartitionReceiver = Vec>; +struct RepartitionExecStateBuilder { + /// Whether to enable pull based execution. + enable_pull_based: bool, + partition_receiver: Option>, +} + +impl RepartitionExecStateBuilder { + fn new() -> Self { + Self { + enable_pull_based: false, + partition_receiver: None, + } + } + fn enable_pull_based(mut self, enable_pull_based: bool) -> Self { + self.enable_pull_based = enable_pull_based; + self + } + fn partition_receiver(mut self, partition_receiver: Receiver) -> Self { + self.partition_receiver = Some(partition_receiver); + self + } + + fn build( + &self, + input: Arc, + partitioning: Partitioning, + metrics: ExecutionPlanMetricsSet, + preserve_order: bool, + name: String, + context: Arc, + ) -> RepartitionExecState { + RepartitionExecState::new( + input, + partitioning, + metrics, + preserve_order, + name, + context, + self.enable_pull_based, + self.partition_receiver.clone(), + ) + } +} + /// Inner state of [`RepartitionExec`]. #[derive(Debug)] struct RepartitionExecState { @@ -82,6 +128,63 @@ struct RepartitionExecState { abort_helper: Arc>>, } +/// create channels for sending batches from input partitions to output partitions. +fn create_repartition_channels( + preserve_order: bool, + num_input_partitions: usize, + num_output_partitions: usize, +) -> ( + Vec, + Vec, +) { + if preserve_order { + let (txs, rxs) = + partition_aware_channels(num_input_partitions, num_output_partitions); + // Take transpose of senders and receivers. `state.channels` keeps track of entries per output partition + let txs = transpose(txs); + let rxs = transpose(rxs); + (txs, rxs) + } else { + // create one channel per *output* partition + // note we use a custom channel that ensures there is always data for each receiver + // but limits the amount of buffering if required. + let (txs, rxs) = channels(num_output_partitions); + // Clone sender for each input partitions + let txs = txs + .into_iter() + .map(|item| vec![item; num_input_partitions]) + .collect::>(); + let rxs = rxs.into_iter().map(|item| vec![item]).collect::>(); + (txs, rxs) + } +} + +/// Create a hashmap of channels for sending batches from input partitions to output partitions. +fn create_partition_channels_hashmap( + txs: Vec, + rxs: Vec, + name: String, + context: Arc, +) -> HashMap< + usize, + ( + InputPartitionsToCurrentPartitionSender, + InputPartitionsToCurrentPartitionReceiver, + SharedMemoryReservation, + ), +> { + let mut channels = HashMap::with_capacity(txs.len()); + + for (partition, (tx, rx)) in txs.into_iter().zip(rxs).enumerate() { + let reservation = Arc::new(Mutex::new( + MemoryConsumer::new(format!("{}[{partition}]", name)) + .register(context.memory_pool()), + )); + channels.insert(partition, (tx, rx, reservation)); + } + + channels +} impl RepartitionExecState { fn new( input: Arc, @@ -90,39 +193,20 @@ impl RepartitionExecState { preserve_order: bool, name: String, context: Arc, + enable_pull_based: bool, + partition_receiver: Option>, ) -> Self { let num_input_partitions = input.output_partitioning().partition_count(); let num_output_partitions = partitioning.partition_count(); - let (txs, rxs) = if preserve_order { - let (txs, rxs) = - partition_aware_channels(num_input_partitions, num_output_partitions); - // Take transpose of senders and receivers. `state.channels` keeps track of entries per output partition - let txs = transpose(txs); - let rxs = transpose(rxs); - (txs, rxs) - } else { - // create one channel per *output* partition - // note we use a custom channel that ensures there is always data for each receiver - // but limits the amount of buffering if required. - let (txs, rxs) = channels(num_output_partitions); - // Clone sender for each input partitions - let txs = txs - .into_iter() - .map(|item| vec![item; num_input_partitions]) - .collect::>(); - let rxs = rxs.into_iter().map(|item| vec![item]).collect::>(); - (txs, rxs) - }; + let (txs, rxs) = create_repartition_channels( + preserve_order, + num_input_partitions, + num_output_partitions, + ); - let mut channels = HashMap::with_capacity(txs.len()); - for (partition, (tx, rx)) in txs.into_iter().zip(rxs).enumerate() { - let reservation = Arc::new(Mutex::new( - MemoryConsumer::new(format!("{}[{partition}]", name)) - .register(context.memory_pool()), - )); - channels.insert(partition, (tx, rx, reservation)); - } + let channels = + create_partition_channels_hashmap(txs, rxs, name, Arc::clone(&context)); // launch one async task per *input* partition let mut spawned_tasks = Vec::with_capacity(num_input_partitions); @@ -136,23 +220,45 @@ impl RepartitionExecState { let r_metrics = RepartitionMetrics::new(i, num_output_partitions, &metrics); - let input_task = SpawnedTask::spawn(RepartitionExec::pull_from_input( - Arc::clone(&input), - i, - txs.clone(), - partitioning.clone(), - r_metrics, - Arc::clone(&context), - )); + let input_task = if enable_pull_based { + SpawnedTask::spawn(OnDemandRepartitionExec::pull_from_input( + Arc::clone(&input), + i, + txs.clone(), + partitioning.clone(), + partition_receiver.clone().unwrap(), + r_metrics, + Arc::clone(&context), + )) + } else { + SpawnedTask::spawn(RepartitionExec::pull_from_input( + Arc::clone(&input), + i, + txs.clone(), + partitioning.clone(), + r_metrics, + Arc::clone(&context), + )) + }; // In a separate task, wait for each input to be done // (and pass along any errors, including panic!s) - let wait_for_task = SpawnedTask::spawn(RepartitionExec::wait_for_task( - input_task, - txs.into_iter() - .map(|(partition, (tx, _reservation))| (partition, tx)) - .collect(), - )); + + let wait_for_task = if enable_pull_based { + SpawnedTask::spawn(OnDemandRepartitionExec::wait_for_task( + input_task, + txs.into_iter() + .map(|(partition, (tx, _reservation))| (partition, tx)) + .collect(), + )) + } else { + SpawnedTask::spawn(RepartitionExec::wait_for_task( + input_task, + txs.into_iter() + .map(|(partition, (tx, _reservation))| (partition, tx)) + .collect(), + )) + }; spawned_tasks.push(wait_for_task); } @@ -343,6 +449,91 @@ impl BatchPartitioner { } } +#[derive(Debug, Clone)] +pub struct RepartitionExecBase { + /// Input execution plan + input: Arc, + /// Execution metrics + metrics: ExecutionPlanMetricsSet, + /// Boolean flag to decide whether to preserve ordering. If true means + /// `SortPreservingRepartitionExec`, false means `RepartitionExec`. + preserve_order: bool, + /// Cache holding plan properties like equivalences, output partitioning etc. + cache: PlanProperties, + /// Inner state that is initialized when the first output stream is created. + state: LazyState, +} + +impl RepartitionExecBase { + fn maintains_input_order_helper( + input: &Arc, + preserve_order: bool, + ) -> Vec { + // We preserve ordering when repartition is order preserving variant or input partitioning is 1 + vec![preserve_order || input.output_partitioning().partition_count() <= 1] + } + + fn eq_properties_helper( + input: &Arc, + preserve_order: bool, + ) -> EquivalenceProperties { + // Equivalence Properties + let mut eq_properties = input.equivalence_properties().clone(); + // If the ordering is lost, reset the ordering equivalence class: + if !Self::maintains_input_order_helper(input, preserve_order)[0] { + eq_properties.clear_orderings(); + } + // When there are more than one input partitions, they will be fused at the output. + // Therefore, remove per partition constants. + if input.output_partitioning().partition_count() > 1 { + eq_properties.clear_per_partition_constants(); + } + eq_properties + } + + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. + fn compute_properties( + input: &Arc, + partitioning: Partitioning, + preserve_order: bool, + ) -> PlanProperties { + PlanProperties::new( + Self::eq_properties_helper(input, preserve_order), + partitioning, + input.pipeline_behavior(), + input.boundedness(), + ) + } + + /// Specify if this repartitioning operation should preserve the order of + /// rows from its input when producing output. Preserving order is more + /// expensive at runtime, so should only be set if the output of this + /// operator can take advantage of it. + /// + /// If the input is not ordered, or has only one partition, this is a no op, + /// and the node remains a `RepartitionExec`. + pub fn with_preserve_order(mut self) -> Self { + self.preserve_order = + // If the input isn't ordered, there is no ordering to preserve + self.input.output_ordering().is_some() && + // if there is only one input partition, merging is not required + // to maintain order + self.input.output_partitioning().partition_count() > 1; + let eq_properties = Self::eq_properties_helper(&self.input, self.preserve_order); + self.cache = self.cache.with_eq_properties(eq_properties); + self + } + + /// Return the sort expressions that are used to merge + fn sort_exprs(&self) -> Option<&LexOrdering> { + if self.preserve_order { + self.input.output_ordering() + } else { + None + } + } +} + /// Maps `N` input partitions to `M` output partitions based on a /// [`Partitioning`] scheme. /// @@ -411,21 +602,12 @@ impl BatchPartitioner { /// data across threads. #[derive(Debug, Clone)] pub struct RepartitionExec { - /// Input execution plan - input: Arc, - /// Inner state that is initialized when the first output stream is created. - state: LazyState, - /// Execution metrics - metrics: ExecutionPlanMetricsSet, - /// Boolean flag to decide whether to preserve ordering. If true means - /// `SortPreservingRepartitionExec`, false means `RepartitionExec`. - preserve_order: bool, - /// Cache holding plan properties like equivalences, output partitioning etc. - cache: PlanProperties, + /// Common fields for all repartitioning executors + base: RepartitionExecBase, } #[derive(Debug, Clone)] -struct RepartitionMetrics { +pub(crate) struct RepartitionMetrics { /// Time in nanos to execute child operator and fetch batches fetch_time: metrics::Time, /// Repartitioning elapsed time in nanos @@ -472,18 +654,30 @@ impl RepartitionMetrics { impl RepartitionExec { /// Input execution plan pub fn input(&self) -> &Arc { - &self.input + &self.base.input } /// Partitioning scheme to use pub fn partitioning(&self) -> &Partitioning { - &self.cache.partitioning + &self.base.cache.partitioning } /// Get preserve_order flag of the RepartitionExecutor /// `true` means `SortPreservingRepartitionExec`, `false` means `RepartitionExec` pub fn preserve_order(&self) -> bool { - self.preserve_order + self.base.preserve_order + } + + /// Specify if this reparititoning operation should preserve the order of + /// rows from its input when producing output. Preserving order is more + /// expensive at runtime, so should only be set if the output of this + /// operator can take advantage of it. + /// + /// If the input is not ordered, or has only one partition, this is a no op, + /// and the node remains a `RepartitionExec`. + pub fn with_preserve_order(mut self) -> Self { + self.base = self.base.with_preserve_order(); + self } /// Get name used to display this Exec @@ -505,14 +699,14 @@ impl DisplayAs for RepartitionExec { "{}: partitioning={}, input_partitions={}", self.name(), self.partitioning(), - self.input.output_partitioning().partition_count() + self.base.input.output_partitioning().partition_count() )?; - if self.preserve_order { + if self.base.preserve_order { write!(f, ", preserve_order=true")?; } - if let Some(sort_exprs) = self.sort_exprs() { + if let Some(sort_exprs) = self.base.sort_exprs() { write!(f, ", sort_exprs={}", sort_exprs.clone())?; } Ok(()) @@ -532,11 +726,11 @@ impl ExecutionPlan for RepartitionExec { } fn properties(&self) -> &PlanProperties { - &self.cache + &self.base.cache } fn children(&self) -> Vec<&Arc> { - vec![&self.input] + vec![&self.base.input] } fn with_new_children( @@ -547,7 +741,7 @@ impl ExecutionPlan for RepartitionExec { children.swap_remove(0), self.partitioning().clone(), )?; - if self.preserve_order { + if self.base.preserve_order { repartition = repartition.with_preserve_order(); } Ok(Arc::new(repartition)) @@ -558,7 +752,10 @@ impl ExecutionPlan for RepartitionExec { } fn maintains_input_order(&self) -> Vec { - Self::maintains_input_order_helper(self.input(), self.preserve_order) + RepartitionExecBase::maintains_input_order_helper( + self.input(), + self.base.preserve_order, + ) } fn execute( @@ -572,17 +769,17 @@ impl ExecutionPlan for RepartitionExec { partition ); - let lazy_state = Arc::clone(&self.state); - let input = Arc::clone(&self.input); + let lazy_state = Arc::clone(&self.base.state); + let input = Arc::clone(&self.base.input); let partitioning = self.partitioning().clone(); - let metrics = self.metrics.clone(); - let preserve_order = self.preserve_order; + let metrics = self.base.metrics.clone(); + let preserve_order = self.base.preserve_order; let name = self.name().to_owned(); let schema = self.schema(); let schema_captured = Arc::clone(&schema); // Get existing ordering to use for merging - let sort_exprs = self.sort_exprs().cloned().unwrap_or_default(); + let sort_exprs = self.base.sort_exprs().cloned().unwrap_or_default(); let stream = futures::stream::once(async move { let num_input_partitions = input.output_partitioning().partition_count(); @@ -593,9 +790,9 @@ impl ExecutionPlan for RepartitionExec { let context_captured = Arc::clone(&context); let state = lazy_state .get_or_init(|| async move { - Mutex::new(RepartitionExecState::new( + Mutex::new(RepartitionExecStateBuilder::new().build( input_captured, - partitioning, + partitioning.clone(), metrics_captured, preserve_order, name_captured, @@ -672,11 +869,11 @@ impl ExecutionPlan for RepartitionExec { } fn metrics(&self) -> Option { - Some(self.metrics.clone_inner()) + Some(self.base.metrics.clone_inner()) } fn statistics(&self) -> Result { - self.input.statistics() + self.base.input.statistics() } fn cardinality_effect(&self) -> CardinalityEffect { @@ -733,85 +930,22 @@ impl RepartitionExec { partitioning: Partitioning, ) -> Result { let preserve_order = false; - let cache = - Self::compute_properties(&input, partitioning.clone(), preserve_order); - Ok(RepartitionExec { - input, - state: Default::default(), - metrics: ExecutionPlanMetricsSet::new(), + let cache = RepartitionExecBase::compute_properties( + &input, + partitioning.clone(), preserve_order, - cache, + ); + Ok(RepartitionExec { + base: RepartitionExecBase { + input, + state: Default::default(), + metrics: ExecutionPlanMetricsSet::new(), + preserve_order, + cache, + }, }) } - fn maintains_input_order_helper( - input: &Arc, - preserve_order: bool, - ) -> Vec { - // We preserve ordering when repartition is order preserving variant or input partitioning is 1 - vec![preserve_order || input.output_partitioning().partition_count() <= 1] - } - - fn eq_properties_helper( - input: &Arc, - preserve_order: bool, - ) -> EquivalenceProperties { - // Equivalence Properties - let mut eq_properties = input.equivalence_properties().clone(); - // If the ordering is lost, reset the ordering equivalence class: - if !Self::maintains_input_order_helper(input, preserve_order)[0] { - eq_properties.clear_orderings(); - } - // When there are more than one input partitions, they will be fused at the output. - // Therefore, remove per partition constants. - if input.output_partitioning().partition_count() > 1 { - eq_properties.clear_per_partition_constants(); - } - eq_properties - } - - /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. - fn compute_properties( - input: &Arc, - partitioning: Partitioning, - preserve_order: bool, - ) -> PlanProperties { - PlanProperties::new( - Self::eq_properties_helper(input, preserve_order), - partitioning, - input.pipeline_behavior(), - input.boundedness(), - ) - } - - /// Specify if this repartitioning operation should preserve the order of - /// rows from its input when producing output. Preserving order is more - /// expensive at runtime, so should only be set if the output of this - /// operator can take advantage of it. - /// - /// If the input is not ordered, or has only one partition, this is a no op, - /// and the node remains a `RepartitionExec`. - pub fn with_preserve_order(mut self) -> Self { - self.preserve_order = - // If the input isn't ordered, there is no ordering to preserve - self.input.output_ordering().is_some() && - // if there is only one input partition, merging is not required - // to maintain order - self.input.output_partitioning().partition_count() > 1; - let eq_properties = Self::eq_properties_helper(&self.input, self.preserve_order); - self.cache = self.cache.with_eq_properties(eq_properties); - self - } - - /// Return the sort expressions that are used to merge - fn sort_exprs(&self) -> Option<&LexOrdering> { - if self.preserve_order { - self.input.output_ordering() - } else { - None - } - } - /// Pulls data from the specified input plan, feeding it to the /// output partitions based on the desired partitioning /// diff --git a/datafusion/physical-plan/src/repartition/on_demand_repartition.rs b/datafusion/physical-plan/src/repartition/on_demand_repartition.rs index 7595facdb36f9..a2637a035ad21 100644 --- a/datafusion/physical-plan/src/repartition/on_demand_repartition.rs +++ b/datafusion/physical-plan/src/repartition/on_demand_repartition.rs @@ -26,16 +26,16 @@ use std::{any::Any, vec}; use super::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use super::{ - BatchPartitioner, DisplayAs, ExecutionPlanProperties, - InputPartitionsToCurrentPartitionReceiver, InputPartitionsToCurrentPartitionSender, - MaybeBatch, RecordBatchStream, RepartitionMetrics, SendableRecordBatchStream, + BatchPartitioner, DisplayAs, ExecutionPlanProperties, MaybeBatch, RecordBatchStream, + RepartitionExecBase, RepartitionMetrics, SendableRecordBatchStream, }; use crate::common::SharedMemoryReservation; use crate::execution_plan::CardinalityEffect; use crate::metrics::BaselineMetrics; use crate::repartition::distributor_channels::{ - channels, partition_aware_channels, DistributionReceiver, DistributionSender, + DistributionReceiver, DistributionSender, }; +use crate::repartition::RepartitionExecStateBuilder; use crate::sorts::streaming_merge::StreamingMergeBuilder; use crate::stream::RecordBatchStreamAdapter; use crate::{DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, Statistics}; @@ -43,13 +43,11 @@ use crate::{DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, Stat use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; use async_channel::{Receiver, Sender}; -use datafusion_common::utils::transpose; + use datafusion_common::{internal_datafusion_err, DataFusionError, Result}; use datafusion_common_runtime::SpawnedTask; use datafusion_execution::memory_pool::MemoryConsumer; use datafusion_execution::TaskContext; -use datafusion_physical_expr::EquivalenceProperties; -use datafusion_physical_expr_common::sort_expr::LexOrdering; use datafusion_common::HashMap; use futures::stream::Stream; @@ -57,153 +55,38 @@ use futures::{FutureExt, StreamExt, TryStreamExt}; use log::trace; use parking_lot::Mutex; -/// Inner state of [`RepartitionExec`]. -#[derive(Debug)] -struct OnDemandRepartitionExecState { - /// Channels for sending batches from input partitions to output partitions. - /// Key is the partition number. - channels: HashMap< - usize, - ( - InputPartitionsToCurrentPartitionSender, - InputPartitionsToCurrentPartitionReceiver, - SharedMemoryReservation, - ), - >, - - /// Helper that ensures that that background job is killed once it is no longer needed. - abort_helper: Arc>>, -} - -impl OnDemandRepartitionExecState { - fn new( - input: Arc, - partitioning: Partitioning, - metrics: ExecutionPlanMetricsSet, - preserve_order: bool, - name: String, - context: Arc, - partition_receiver: Receiver, - ) -> Self { - let num_input_partitions = input.output_partitioning().partition_count(); - let num_output_partitions = partitioning.partition_count(); - - let (txs, rxs) = if preserve_order { - let (txs, rxs) = - partition_aware_channels(num_input_partitions, num_output_partitions); - // Take transpose of senders and receivers. `state.channels` keeps track of entries per output partition - let txs = transpose(txs); - let rxs = transpose(rxs); - (txs, rxs) - } else { - // create one channel per *output* partition - // note we use a custom channel that ensures there is always data for each receiver - // but limits the amount of buffering if required. - let (txs, rxs) = channels(num_output_partitions); - // Clone sender for each input partitions - let txs = txs - .into_iter() - .map(|item| vec![item; num_input_partitions]) - .collect::>(); - let rxs = rxs.into_iter().map(|item| vec![item]).collect::>(); - (txs, rxs) - }; - - let mut channels = HashMap::with_capacity(txs.len()); - for (partition, (tx, rx)) in txs.into_iter().zip(rxs).enumerate() { - let reservation = Arc::new(Mutex::new( - MemoryConsumer::new(format!("{}[{partition}]", name)) - .register(context.memory_pool()), - )); - channels.insert(partition, (tx, rx, reservation)); - } - - // launch one async task per *input* partition - let mut spawned_tasks = Vec::with_capacity(num_input_partitions); - for i in 0..num_input_partitions { - let txs: HashMap<_, _> = channels - .iter() - .map(|(partition, (tx, _rx, reservation))| { - (*partition, (tx[i].clone(), Arc::clone(reservation))) - }) - .collect(); - - let r_metrics = RepartitionMetrics::new(i, num_output_partitions, &metrics); - - let input_task = - SpawnedTask::spawn(OnDemandRepartitionExec::pull_from_input( - Arc::clone(&input), - i, - txs.clone(), - partitioning.clone(), - partition_receiver.clone(), - r_metrics, - Arc::clone(&context), - )); - - // In a separate task, wait for each input to be done - // (and pass along any errors, including panic!s) - let wait_for_task = - SpawnedTask::spawn(OnDemandRepartitionExec::wait_for_task( - input_task, - txs.into_iter() - .map(|(partition, (tx, _reservation))| (partition, tx)) - .collect(), - )); - spawned_tasks.push(wait_for_task); - } - - Self { - channels, - abort_helper: Arc::new(spawned_tasks), - } - } -} - -/// Lazily initialized state -/// -/// Note that the state is initialized ONCE for all partitions by a single task(thread). -/// This may take a short while. It is also like that multiple threads -/// call execute at the same time, because we have just started "target partitions" tasks -/// which is commonly set to the number of CPU cores and all call execute at the same time. -/// -/// Thus, use a **tokio** `OnceCell` for this initialization so as not to waste CPU cycles -/// in a futex lock but instead allow other threads to do something useful. -/// -/// Uses a parking_lot `Mutex` to control other accesses as they are very short duration -/// (e.g. removing channels on completion) where the overhead of `await` is not warranted. -type LazyState = Arc>>; - #[derive(Debug, Clone)] pub struct OnDemandRepartitionExec { - /// Input execution plan - input: Arc, - /// Inner state that is initialized when the first output stream is created. - state: LazyState, - /// Execution metrics - metrics: ExecutionPlanMetricsSet, - /// Boolean flag to decide whether to preserve ordering. If true means - /// `SortPreservingRepartitionExec`, false means `RepartitionExec`. - preserve_order: bool, - /// Cache holding plan properties like equivalences, output partitioning etc. - cache: PlanProperties, + base: RepartitionExecBase, } impl OnDemandRepartitionExec { /// Input execution plan pub fn input(&self) -> &Arc { - &self.input + &self.base.input } /// Partitioning scheme to use pub fn partitioning(&self) -> &Partitioning { - &self.cache.partitioning + &self.base.cache.partitioning } /// Get preserve_order flag of the RepartitionExecutor /// `true` means `SortPreservingRepartitionExec`, `false` means `RepartitionExec` pub fn preserve_order(&self) -> bool { - self.preserve_order + self.base.preserve_order + } + + /// Specify if this reparititoning operation should preserve the order of + /// rows from its input when producing output. Preserving order is more + /// expensive at runtime, so should only be set if the output of this + /// operator can take advantage of it. + /// + /// If the input is not ordered, or has only one partition, this is a no op, + /// and the node remains a `RepartitionExec`. + pub fn with_preserve_order(mut self) -> Self { + self.base = self.base.with_preserve_order(); + self } /// Get name used to display this Exec @@ -225,14 +108,14 @@ impl DisplayAs for OnDemandRepartitionExec { "{}: partitioning={}, input_partitions={}", self.name(), self.partitioning(), - self.input.output_partitioning().partition_count() + self.base.input.output_partitioning().partition_count() )?; - if self.preserve_order { + if self.base.preserve_order { write!(f, ", preserve_order=true")?; } - if let Some(sort_exprs) = self.sort_exprs() { + if let Some(sort_exprs) = self.base.sort_exprs() { write!(f, ", sort_exprs={}", sort_exprs.clone())?; } Ok(()) @@ -252,11 +135,11 @@ impl ExecutionPlan for OnDemandRepartitionExec { } fn properties(&self) -> &PlanProperties { - &self.cache + &self.base.cache } fn children(&self) -> Vec<&Arc> { - vec![&self.input] + vec![&self.base.input] } fn with_new_children( @@ -267,7 +150,7 @@ impl ExecutionPlan for OnDemandRepartitionExec { children.swap_remove(0), self.partitioning().clone(), )?; - if self.preserve_order { + if self.base.preserve_order { repartition = repartition.with_preserve_order(); } Ok(Arc::new(repartition)) @@ -278,7 +161,10 @@ impl ExecutionPlan for OnDemandRepartitionExec { } fn maintains_input_order(&self) -> Vec { - Self::maintains_input_order_helper(self.input(), self.preserve_order) + RepartitionExecBase::maintains_input_order_helper( + self.input(), + self.base.preserve_order, + ) } fn execute( @@ -292,17 +178,17 @@ impl ExecutionPlan for OnDemandRepartitionExec { partition ); - let lazy_state = Arc::clone(&self.state); - let input = Arc::clone(&self.input); + let lazy_state = Arc::clone(&self.base.state); + let input = Arc::clone(&self.base.input); let partitioning = self.partitioning().clone(); - let metrics = self.metrics.clone(); - let preserve_order = self.preserve_order; + let metrics = self.base.metrics.clone(); + let preserve_order = self.base.preserve_order; let name = self.name().to_owned(); let schema = self.schema(); let schema_captured = Arc::clone(&schema); // Get existing ordering to use for merging - let sort_exprs = self.sort_exprs().cloned().unwrap_or_default(); + let sort_exprs = self.base.sort_exprs().cloned().unwrap_or_default(); let stream = futures::stream::once(async move { let num_input_partitions = input.output_partitioning().partition_count(); @@ -311,18 +197,22 @@ impl ExecutionPlan for OnDemandRepartitionExec { let metrics_captured = metrics.clone(); let name_captured = name.clone(); let context_captured = Arc::clone(&context); - let (partition_tx, patition_rx) = async_channel::unbounded(); + let (partition_tx, partition_rx) = async_channel::unbounded(); let state = lazy_state .get_or_init(|| async move { - Mutex::new(OnDemandRepartitionExecState::new( - input_captured, - partitioning, - metrics_captured, - preserve_order, - name_captured, - context_captured, - patition_rx.clone(), - )) + Mutex::new( + RepartitionExecStateBuilder::new() + .enable_pull_based(true) + .partition_receiver(partition_rx.clone()) + .build( + input_captured, + partitioning.clone(), + metrics_captured, + preserve_order, + name_captured, + context_captured, + ), + ) }) .await; @@ -398,11 +288,11 @@ impl ExecutionPlan for OnDemandRepartitionExec { } fn metrics(&self) -> Option { - Some(self.metrics.clone_inner()) + Some(self.base.metrics.clone_inner()) } fn statistics(&self) -> Result { - self.input.statistics() + self.base.input.statistics() } fn cardinality_effect(&self) -> CardinalityEffect { @@ -419,92 +309,27 @@ impl OnDemandRepartitionExec { partitioning: Partitioning, ) -> Result { let preserve_order = false; - let cache = - Self::compute_properties(&input, partitioning.clone(), preserve_order); - Ok(OnDemandRepartitionExec { - input, - state: Default::default(), - metrics: ExecutionPlanMetricsSet::new(), + let cache = RepartitionExecBase::compute_properties( + &input, + partitioning.clone(), preserve_order, - cache, + ); + Ok(OnDemandRepartitionExec { + base: RepartitionExecBase { + input, + state: Default::default(), + metrics: ExecutionPlanMetricsSet::new(), + preserve_order, + cache, + }, }) } - fn maintains_input_order_helper( - input: &Arc, - preserve_order: bool, - ) -> Vec { - // We preserve ordering when repartition is order preserving variant or input partitioning is 1 - vec![preserve_order || input.output_partitioning().partition_count() <= 1] - } - - fn eq_properties_helper( - input: &Arc, - preserve_order: bool, - ) -> EquivalenceProperties { - // Equivalence Properties - let mut eq_properties = input.equivalence_properties().clone(); - // If the ordering is lost, reset the ordering equivalence class: - if !Self::maintains_input_order_helper(input, preserve_order)[0] { - eq_properties.clear_orderings(); - } - // When there are more than one input partitions, they will be fused at the output. - // Therefore, remove per partition constants. - if input.output_partitioning().partition_count() > 1 { - eq_properties.clear_per_partition_constants(); - } - eq_properties - } - - /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. - fn compute_properties( - input: &Arc, - partitioning: Partitioning, - preserve_order: bool, - ) -> PlanProperties { - // Equivalence Properties - let eq_properties = Self::eq_properties_helper(input, preserve_order); - - PlanProperties::new( - eq_properties, // Equivalence Properties - partitioning, // Output Partitioning - input.execution_mode(), // Execution Mode - ) - } - - /// Specify if this reparititoning operation should preserve the order of - /// rows from its input when producing output. Preserving order is more - /// expensive at runtime, so should only be set if the output of this - /// operator can take advantage of it. - /// - /// If the input is not ordered, or has only one partition, this is a no op, - /// and the node remains a `RepartitionExec`. - pub fn with_preserve_order(mut self) -> Self { - self.preserve_order = - // If the input isn't ordered, there is no ordering to preserve - self.input.output_ordering().is_some() && - // if there is only one input partition, merging is not required - // to maintain order - self.input.output_partitioning().partition_count() > 1; - let eq_properties = Self::eq_properties_helper(&self.input, self.preserve_order); - self.cache = self.cache.with_eq_properties(eq_properties); - self - } - - /// Return the sort expressions that are used to merge - fn sort_exprs(&self) -> Option<&LexOrdering> { - if self.preserve_order { - self.input.output_ordering() - } else { - None - } - } - /// Pulls data from the specified input plan, feeding it to the /// output partitions based on the desired partitioning /// /// txs hold the output sending channels for each output partition - async fn pull_from_input( + pub(crate) async fn pull_from_input( input: Arc, partition: usize, mut output_channels: HashMap< @@ -527,6 +352,13 @@ impl OnDemandRepartitionExec { // While there are still outputs to send to, keep pulling inputs let mut batches_until_yield = partitioner.num_partitions(); while !output_channels.is_empty() { + let partition = output_partition_rx.recv().await.map_err(|e| { + internal_datafusion_err!( + "Error receiving partition number from output partition: {}", + e + ) + })?; + // fetch the next batch let timer = metrics.fetch_time.timer(); let result = stream.next().await; @@ -542,13 +374,6 @@ impl OnDemandRepartitionExec { let (_, batch) = res?; let size = batch.get_array_memory_size(); - let partition = output_partition_rx.recv().await.map_err(|e| { - internal_datafusion_err!( - "Error receiving partition number from output partition: {}", - e - ) - })?; - let timer = metrics.send_time[partition].timer(); // if there is still a receiver, send to it if let Some((tx, reservation)) = output_channels.get_mut(&partition) { @@ -595,7 +420,7 @@ impl OnDemandRepartitionExec { /// each of the output tx channels to signal one of the inputs is /// complete. Upon error, propagates the errors to all output tx /// channels. - async fn wait_for_task( + pub(crate) async fn wait_for_task( input_task: SpawnedTask>, txs: HashMap>, ) { From eda8a7fc0480e1f430ba5ec41e00a6e59f1d5c03 Mon Sep 17 00:00:00 2001 From: Weijun-H Date: Tue, 31 Dec 2024 00:09:57 +0100 Subject: [PATCH 08/25] refactor: Use macro ready --- .../physical-plan/src/repartition/mod.rs | 23 ++++++++----------- .../src/repartition/on_demand_repartition.rs | 23 ++++++++----------- 2 files changed, 20 insertions(+), 26 deletions(-) diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index e41f7f2d18620..92157819e93aa 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -54,7 +54,7 @@ use datafusion_physical_expr::{EquivalenceProperties, PhysicalExpr}; use datafusion_physical_expr_common::sort_expr::LexOrdering; use futures::stream::Stream; -use futures::{FutureExt, StreamExt, TryStreamExt}; +use futures::{ready, FutureExt, StreamExt, TryStreamExt}; use log::trace; use on_demand_repartition::OnDemandRepartitionExec; use parking_lot::Mutex; @@ -1103,8 +1103,8 @@ impl Stream for RepartitionStream { cx: &mut Context<'_>, ) -> Poll> { loop { - match self.input.recv().poll_unpin(cx) { - Poll::Ready(Some(Some(v))) => { + match ready!(self.input.recv().poll_unpin(cx)) { + Some(Some(v)) => { if let Ok(batch) = &v { self.reservation .lock() @@ -1113,7 +1113,7 @@ impl Stream for RepartitionStream { return Poll::Ready(Some(v)); } - Poll::Ready(Some(None)) => { + Some(None) => { self.num_input_partitions_processed += 1; if self.num_input_partitions == self.num_input_partitions_processed { @@ -1124,12 +1124,9 @@ impl Stream for RepartitionStream { continue; } } - Poll::Ready(None) => { + None => { return Poll::Ready(None); } - Poll::Pending => { - return Poll::Pending; - } } } } @@ -1165,21 +1162,21 @@ impl Stream for PerPartitionStream { mut self: Pin<&mut Self>, cx: &mut Context<'_>, ) -> Poll> { - match self.receiver.recv().poll_unpin(cx) { - Poll::Ready(Some(Some(v))) => { + match ready!(self.receiver.recv().poll_unpin(cx)) { + Some(Some(v)) => { if let Ok(batch) = &v { self.reservation .lock() .shrink(batch.get_array_memory_size()); } + Poll::Ready(Some(v)) } - Poll::Ready(Some(None)) => { + Some(None) => { // Input partition has finished sending batches Poll::Ready(None) } - Poll::Ready(None) => Poll::Ready(None), - Poll::Pending => Poll::Pending, + None => Poll::Ready(None), } } } diff --git a/datafusion/physical-plan/src/repartition/on_demand_repartition.rs b/datafusion/physical-plan/src/repartition/on_demand_repartition.rs index a2637a035ad21..10540a93d5acb 100644 --- a/datafusion/physical-plan/src/repartition/on_demand_repartition.rs +++ b/datafusion/physical-plan/src/repartition/on_demand_repartition.rs @@ -51,7 +51,7 @@ use datafusion_execution::TaskContext; use datafusion_common::HashMap; use futures::stream::Stream; -use futures::{FutureExt, StreamExt, TryStreamExt}; +use futures::{ready, FutureExt, StreamExt, TryStreamExt}; use log::trace; use parking_lot::Mutex; @@ -499,21 +499,21 @@ impl Stream for OnDemandPerPartitionStream { })?; } - match self.receiver.recv().poll_unpin(cx) { - Poll::Ready(Some(Some(v))) => { + match ready!(self.receiver.recv().poll_unpin(cx)) { + Some(Some(v)) => { if let Ok(batch) = &v { self.reservation .lock() .shrink(batch.get_array_memory_size()); } + Poll::Ready(Some(v)) } - Poll::Ready(Some(None)) => { + Some(None) => { // Input partition has finished sending batches Poll::Ready(None) } - Poll::Ready(None) => Poll::Ready(None), - Poll::Pending => Poll::Pending, + None => Poll::Ready(None), } } } @@ -569,8 +569,8 @@ impl Stream for OnDemandRepartitionStream { })?; } - match self.input.recv().poll_unpin(cx) { - Poll::Ready(Some(Some(v))) => { + match ready!(self.input.recv().poll_unpin(cx)) { + Some(Some(v)) => { if let Ok(batch) = &v { self.reservation .lock() @@ -579,7 +579,7 @@ impl Stream for OnDemandRepartitionStream { return Poll::Ready(Some(v)); } - Poll::Ready(Some(None)) => { + Some(None) => { self.num_input_partitions_processed += 1; if self.num_input_partitions == self.num_input_partitions_processed { @@ -590,12 +590,9 @@ impl Stream for OnDemandRepartitionStream { continue; } } - Poll::Ready(None) => { + None => { return Poll::Ready(None); } - Poll::Pending => { - return Poll::Pending; - } } } } From a91fdd889232ce828342d2ebb47d77cb05599903 Mon Sep 17 00:00:00 2001 From: Weijun-H Date: Mon, 13 Jan 2025 22:10:44 +0800 Subject: [PATCH 09/25] chore --- benchmarks/bench.sh | 13 +- benchmarks/src/tpch/run.rs | 7 + datafusion/common/src/config.rs | 4 +- datafusion/core/src/physical_planner.rs | 5 +- .../enforce_distribution.rs | 386 +++++----- .../physical_optimizer/enforce_sorting.rs | 336 +++++---- datafusion/physical-expr/src/partitioning.rs | 5 + .../src/coalesce_batches.rs | 3 +- .../src/enforce_distribution.rs | 45 ++ .../src/enforce_sorting/mod.rs | 17 + .../src/enforce_sorting/sort_pushdown.rs | 2 + datafusion/physical-optimizer/src/utils.rs | 6 + .../physical-plan/src/repartition/mod.rs | 70 +- .../src/repartition/on_demand_repartition.rs | 692 ++++++++++++------ .../src/sorts/sort_preserving_merge.rs | 39 +- .../src/sorts/streaming_merge.rs | 3 +- datafusion/proto/proto/datafusion.proto | 1 + datafusion/proto/src/generated/pbjson.rs | 15 + datafusion/proto/src/generated/prost.rs | 4 +- datafusion/proto/src/logical_plan/mod.rs | 1 + .../proto/src/physical_plan/from_proto.rs | 3 + .../proto/src/physical_plan/to_proto.rs | 5 + .../test_files/information_schema.slt | 2 + .../sqllogictest/test_files/join.slt.part | 2 +- .../test_files/on_demand_repartition.slt | 445 +++++++++++ docs/source/user-guide/configs.md | 1 + 26 files changed, 1461 insertions(+), 651 deletions(-) create mode 100644 datafusion/sqllogictest/test_files/on_demand_repartition.slt diff --git a/benchmarks/bench.sh b/benchmarks/bench.sh index 96c90aa1f60d8..12e4d8002edec 100755 --- a/benchmarks/bench.sh +++ b/benchmarks/bench.sh @@ -36,6 +36,7 @@ DATAFUSION_DIR=${DATAFUSION_DIR:-$SCRIPT_DIR/..} DATA_DIR=${DATA_DIR:-$SCRIPT_DIR/data} CARGO_COMMAND=${CARGO_COMMAND:-"cargo run --release"} PREFER_HASH_JOIN=${PREFER_HASH_JOIN:-true} +PREFER_ROUND_ROBIN=${PREFER_ROUND_ROBIN:-true} VIRTUAL_ENV=${VIRTUAL_ENV:-$SCRIPT_DIR/venv} usage() { @@ -93,6 +94,7 @@ CARGO_COMMAND command that runs the benchmark binary DATAFUSION_DIR directory to use (default $DATAFUSION_DIR) RESULTS_NAME folder where the benchmark files are stored PREFER_HASH_JOIN Prefer hash join algorithm (default true) +PREFER_ROUND_ROBIN Prefer round robin partitioning (default true) VENV_PATH Python venv to use for compare and venv commands (default ./venv, override by /bin/activate) " exit 1 @@ -163,6 +165,9 @@ main() { tpch10) data_tpch "10" ;; + tpch50) + data_tpch "50" + ;; tpch_mem10) # same data as for tpch10 data_tpch "10" @@ -220,6 +225,7 @@ main() { echo "RESULTS_DIR: ${RESULTS_DIR}" echo "CARGO_COMMAND: ${CARGO_COMMAND}" echo "PREFER_HASH_JOIN: ${PREFER_HASH_JOIN}" + echo "PREFER_ROUND_ROBIN: ${PREFER_ROUND_ROBIN}" echo "***************************" # navigate to the appropriate directory @@ -252,6 +258,9 @@ main() { tpch10) run_tpch "10" ;; + tpch50) + run_tpch "50" + ;; tpch_mem10) run_tpch_mem "10" ;; @@ -378,7 +387,7 @@ run_tpch() { RESULTS_FILE="${RESULTS_DIR}/tpch_sf${SCALE_FACTOR}.json" echo "RESULTS_FILE: ${RESULTS_FILE}" echo "Running tpch benchmark..." - $CARGO_COMMAND --bin tpch -- benchmark datafusion --iterations 5 --path "${TPCH_DIR}" --prefer_hash_join "${PREFER_HASH_JOIN}" --format parquet -o "${RESULTS_FILE}" + $CARGO_COMMAND --bin tpch -- benchmark datafusion --iterations 5 --path "${TPCH_DIR}" --prefer_hash_join "${PREFER_HASH_JOIN}" --prefer_round_robin "${PREFER_ROUND_ROBIN}" --format parquet -o "${RESULTS_FILE}" } # Runs the tpch in memory @@ -394,7 +403,7 @@ run_tpch_mem() { echo "RESULTS_FILE: ${RESULTS_FILE}" echo "Running tpch_mem benchmark..." # -m means in memory - $CARGO_COMMAND --bin tpch -- benchmark datafusion --iterations 5 --path "${TPCH_DIR}" --prefer_hash_join "${PREFER_HASH_JOIN}" -m --format parquet -o "${RESULTS_FILE}" + $CARGO_COMMAND --bin tpch -- benchmark datafusion --iterations 5 --path "${TPCH_DIR}" --prefer_hash_join "${PREFER_HASH_JOIN}" -m --prefer_round_robin "${PREFER_ROUND_ROBIN}" --format parquet -o "${RESULTS_FILE}" } # Runs the parquet filter benchmark diff --git a/benchmarks/src/tpch/run.rs b/benchmarks/src/tpch/run.rs index eb9db821db02f..48a54fb93618b 100644 --- a/benchmarks/src/tpch/run.rs +++ b/benchmarks/src/tpch/run.rs @@ -90,6 +90,11 @@ pub struct RunOpt { /// True by default. #[structopt(short = "j", long = "prefer_hash_join", default_value = "true")] prefer_hash_join: BoolDefaultTrue, + + /// If true then round robin repartitioning is used, if false then on demand repartitioning + /// True by default. + #[structopt(short = "r", long = "prefer_round_robin", default_value = "true")] + prefer_round_robin: BoolDefaultTrue, } const TPCH_QUERY_START_ID: usize = 1; @@ -357,6 +362,7 @@ mod tests { output_path: None, disable_statistics: false, prefer_hash_join: true, + prefer_round_robin: true, }; opt.register_tables(&ctx).await?; let queries = get_query_sql(query)?; @@ -393,6 +399,7 @@ mod tests { output_path: None, disable_statistics: false, prefer_hash_join: true, + prefer_round_robin: true, }; opt.register_tables(&ctx).await?; let queries = get_query_sql(query)?; diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index 09ef298c437cd..da00c7515ea11 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -574,7 +574,9 @@ config_namespace! { /// repartitioning to increase parallelism to leverage more CPU cores pub enable_round_robin_repartition: bool, default = true - pub enable_on_demand_repartition: bool, default = true + /// When set to false, the physical plan optimizer will replace the round robin + /// repartitioning with on demand repartitioning + pub prefer_round_robin_repartition: bool, default = true /// When set to true, the optimizer will attempt to perform limit operations /// during aggregations, if possible diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index a74cdcc5920bf..392bd89db73a7 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -787,7 +787,10 @@ impl DefaultPhysicalPlanner { let input_dfschema = input.as_ref().schema(); let physical_partitioning = match partitioning_scheme { LogicalPartitioning::RoundRobinBatch(n) => { - Partitioning::RoundRobinBatch(*n) + return Ok(Arc::new(RepartitionExec::try_new( + physical_input, + Partitioning::RoundRobinBatch(*n), + )?)); } LogicalPartitioning::Hash(expr, n) => { let runtime_expr = expr diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index 66d1380e09c38..1def872129a02 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -161,22 +161,23 @@ impl ExecutionPlan for SortRequiredExec { } } -fn parquet_exec() -> Arc { +fn parquet_exec() -> Arc { parquet_exec_with_sort(vec![]) } -fn parquet_exec_multiple() -> Arc { +fn parquet_exec_multiple() -> Arc { parquet_exec_multiple_sorted(vec![]) } /// Created a sorted parquet exec with multiple files -fn parquet_exec_multiple_sorted( - output_ordering: Vec, -) -> Arc { - FileScanConfig::new( - ObjectStoreUrl::parse("test:///").unwrap(), - schema(), - Arc::new(ParquetSource::default()), +fn parquet_exec_multiple_sorted(output_ordering: Vec) -> Arc { + ParquetExec::builder( + FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema()) + .with_file_groups(vec![ + vec![PartitionedFile::new("x".to_string(), 100)], + vec![PartitionedFile::new("y".to_string(), 100)], + ]) + .with_output_ordering(output_ordering), ) .with_file_groups(vec![ vec![PartitionedFile::new("x".to_string(), 100)], @@ -186,31 +187,54 @@ fn parquet_exec_multiple_sorted( .build() } -fn csv_exec() -> Arc { +fn csv_exec() -> Arc { csv_exec_with_sort(vec![]) } -fn csv_exec_with_sort(output_ordering: Vec) -> Arc { - FileScanConfig::new( - ObjectStoreUrl::parse("test:///").unwrap(), - schema(), - Arc::new(CsvSource::new(false, b',', b'"')), +fn csv_exec_with_sort(output_ordering: Vec) -> Arc { + Arc::new( + CsvExec::builder( + FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema()) + .with_file(PartitionedFile::new("x".to_string(), 100)) + .with_output_ordering(output_ordering), + ) + .with_has_header(false) + .with_delimeter(b',') + .with_quote(b'"') + .with_escape(None) + .with_comment(None) + .with_newlines_in_values(false) + .with_file_compression_type(FileCompressionType::UNCOMPRESSED) + .build(), ) .with_file(PartitionedFile::new("x".to_string(), 100)) .with_output_ordering(output_ordering) .build() } -fn csv_exec_multiple() -> Arc { +fn csv_exec_multiple() -> Arc { csv_exec_multiple_sorted(vec![]) } // Created a sorted parquet exec with multiple files -fn csv_exec_multiple_sorted(output_ordering: Vec) -> Arc { - FileScanConfig::new( - ObjectStoreUrl::parse("test:///").unwrap(), - schema(), - Arc::new(CsvSource::new(false, b',', b'"')), +fn csv_exec_multiple_sorted(output_ordering: Vec) -> Arc { + Arc::new( + CsvExec::builder( + FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema()) + .with_file_groups(vec![ + vec![PartitionedFile::new("x".to_string(), 100)], + vec![PartitionedFile::new("y".to_string(), 100)], + ]) + .with_output_ordering(output_ordering), + ) + .with_has_header(false) + .with_delimeter(b',') + .with_quote(b'"') + .with_escape(None) + .with_comment(None) + .with_newlines_in_values(false) + .with_file_compression_type(FileCompressionType::UNCOMPRESSED) + .build(), ) .with_file_groups(vec![ vec![PartitionedFile::new("x".to_string(), 100)], @@ -567,14 +591,14 @@ fn multi_hash_joins() -> Result<()> { join_plan.as_str(), "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ], // Should include 4 RepartitionExecs _ => vec![ @@ -583,14 +607,14 @@ fn multi_hash_joins() -> Result<()> { join_plan.as_str(), "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ], }; assert_optimized!(expected, top_join.clone(), true); @@ -630,14 +654,14 @@ fn multi_hash_joins() -> Result<()> { join_plan.as_str(), "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ], // Should include 4 RepartitionExecs _ => @@ -647,14 +671,14 @@ fn multi_hash_joins() -> Result<()> { join_plan.as_str(), "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ], }; assert_optimized!(expected, top_join.clone(), true); @@ -706,13 +730,13 @@ fn multi_joins_after_alias() -> Result<()> { "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, b@1)]", "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", "RepartitionExec: partitioning=Hash([b@1], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ]; assert_optimized!(expected, top_join.clone(), true); assert_optimized!(expected, top_join, false); @@ -732,13 +756,13 @@ fn multi_joins_after_alias() -> Result<()> { "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, b@1)]", "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", "RepartitionExec: partitioning=Hash([b@1], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ]; assert_optimized!(expected, top_join.clone(), true); assert_optimized!(expected, top_join, false); @@ -785,13 +809,13 @@ fn multi_joins_after_multi_alias() -> Result<()> { "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, b@1)]", "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", "RepartitionExec: partitioning=Hash([b@1], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ]; assert_optimized!(expected, top_join.clone(), true); @@ -827,12 +851,12 @@ fn join_after_agg_alias() -> Result<()> { "RepartitionExec: partitioning=Hash([a1@0], 10), input_partitions=10", "AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[]", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", "AggregateExec: mode=FinalPartitioned, gby=[a2@0 as a2], aggr=[]", "RepartitionExec: partitioning=Hash([a2@0], 10), input_partitions=10", "AggregateExec: mode=Partial, gby=[a@0 as a2], aggr=[]", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ]; assert_optimized!(expected, join.clone(), true); assert_optimized!(expected, join, false); @@ -880,12 +904,12 @@ fn hash_join_key_ordering() -> Result<()> { "RepartitionExec: partitioning=Hash([b1@0, a1@1], 10), input_partitions=10", "AggregateExec: mode=Partial, gby=[b@1 as b1, a@0 as a1], aggr=[]", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", "AggregateExec: mode=FinalPartitioned, gby=[b@0 as b, a@1 as a], aggr=[]", "RepartitionExec: partitioning=Hash([b@0, a@1], 10), input_partitions=10", "AggregateExec: mode=Partial, gby=[b@1 as b, a@0 as a], aggr=[]", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ]; assert_optimized!(expected, join.clone(), true); assert_optimized!(expected, join, false); @@ -999,19 +1023,19 @@ fn multi_hash_join_key_ordering() -> Result<()> { "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(b@1, b1@1), (c@2, c1@2), (a@0, a1@0)]", "RepartitionExec: partitioning=Hash([b@1, c@2, a@0], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", "RepartitionExec: partitioning=Hash([b1@1, c1@2, a1@0], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1]", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(b@1, b1@1), (c@2, c1@2), (a@0, a1@0)]", "RepartitionExec: partitioning=Hash([b@1, c@2, a@0], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", "RepartitionExec: partitioning=Hash([b1@1, c1@2, a1@0], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1]", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ]; assert_optimized!(expected, filter_top_join.clone(), true); assert_optimized!(expected, filter_top_join, false); @@ -1137,19 +1161,19 @@ fn reorder_join_keys_to_left_input() -> Result<()> { "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a1@0), (b@1, b1@1), (c@2, c1@2)]", "RepartitionExec: partitioning=Hash([a@0, b@1, c@2], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", "RepartitionExec: partitioning=Hash([a1@0, b1@1, c1@2], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1]", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@2, c1@2), (b@1, b1@1), (a@0, a1@0)]", "RepartitionExec: partitioning=Hash([c@2, b@1, a@0], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", "RepartitionExec: partitioning=Hash([c1@2, b1@1, a1@0], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1]", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ]; assert_plan_txt!(expected, reordered); @@ -1271,19 +1295,19 @@ fn reorder_join_keys_to_right_input() -> Result<()> { "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a1@0), (b@1, b1@1)]", "RepartitionExec: partitioning=Hash([a@0, b@1], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", "RepartitionExec: partitioning=Hash([a1@0, b1@1], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1]", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@2, c1@2), (b@1, b1@1), (a@0, a1@0)]", "RepartitionExec: partitioning=Hash([c@2, b@1, a@0], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", "RepartitionExec: partitioning=Hash([c1@2, b1@1, a1@0], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1]", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ]; assert_plan_txt!(expected, reordered); @@ -1344,16 +1368,16 @@ fn multi_smj_joins() -> Result<()> { "SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", "SortExec: expr=[b1@1 ASC], preserve_partitioning=[true]", "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", "SortExec: expr=[c@2 ASC], preserve_partitioning=[true]", "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ], // Should include 7 RepartitionExecs (4 hash, 3 round-robin), 4 SortExecs // Since ordering of the left child is not preserved after SortMergeJoin @@ -1373,16 +1397,16 @@ fn multi_smj_joins() -> Result<()> { "SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", "SortExec: expr=[b1@1 ASC], preserve_partitioning=[true]", "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", "SortExec: expr=[c@2 ASC], preserve_partitioning=[true]", "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ], }; assert_optimized!(expected, top_join.clone(), true, true); @@ -1396,16 +1420,16 @@ fn multi_smj_joins() -> Result<()> { "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10, preserve_order=true, sort_exprs=b1@1 ASC", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[b1@1 ASC], preserve_partitioning=[false]", "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10, preserve_order=true, sort_exprs=c@2 ASC", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ], // Should include 8 RepartitionExecs (4 hash, 8 round-robin), 4 SortExecs // Since ordering of the left child is not preserved after SortMergeJoin @@ -1427,16 +1451,16 @@ fn multi_smj_joins() -> Result<()> { "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10, preserve_order=true, sort_exprs=b1@1 ASC", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[b1@1 ASC], preserve_partitioning=[false]", "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10, preserve_order=true, sort_exprs=c@2 ASC", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ], }; assert_optimized!(expected_first_sort_enforcement, top_join, false, true); @@ -1462,16 +1486,16 @@ fn multi_smj_joins() -> Result<()> { "SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", "SortExec: expr=[b1@1 ASC], preserve_partitioning=[true]", "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", "SortExec: expr=[c@2 ASC], preserve_partitioning=[true]", "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ], // Should include 7 RepartitionExecs (4 hash, 3 round-robin) and 4 SortExecs JoinType::Left | JoinType::Full => vec![ @@ -1482,16 +1506,16 @@ fn multi_smj_joins() -> Result<()> { "SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", "SortExec: expr=[b1@1 ASC], preserve_partitioning=[true]", "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", "SortExec: expr=[c@2 ASC], preserve_partitioning=[true]", "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ], // this match arm cannot be reached _ => unreachable!() @@ -1506,16 +1530,16 @@ fn multi_smj_joins() -> Result<()> { "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10, preserve_order=true, sort_exprs=b1@1 ASC", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[b1@1 ASC], preserve_partitioning=[false]", "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10, preserve_order=true, sort_exprs=c@2 ASC", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ], // Should include 8 RepartitionExecs (4 of them preserves order) and 4 SortExecs JoinType::Left | JoinType::Full => vec![ @@ -1528,16 +1552,16 @@ fn multi_smj_joins() -> Result<()> { "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10, preserve_order=true, sort_exprs=b1@1 ASC", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[b1@1 ASC], preserve_partitioning=[false]", "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10, preserve_order=true, sort_exprs=c@2 ASC", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ], // this match arm cannot be reached _ => unreachable!() @@ -1607,14 +1631,14 @@ fn smj_join_key_ordering() -> Result<()> { "RepartitionExec: partitioning=Hash([b1@0, a1@1], 10), input_partitions=10", "AggregateExec: mode=Partial, gby=[b@1 as b1, a@0 as a1], aggr=[]", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", "SortExec: expr=[b2@1 ASC, a2@0 ASC], preserve_partitioning=[true]", "ProjectionExec: expr=[a@1 as a2, b@0 as b2]", "AggregateExec: mode=FinalPartitioned, gby=[b@0 as b, a@1 as a], aggr=[]", "RepartitionExec: partitioning=Hash([b@0, a@1], 10), input_partitions=10", "AggregateExec: mode=Partial, gby=[b@1 as b, a@0 as a], aggr=[]", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ]; assert_optimized!(expected, join.clone(), true, true); @@ -1630,7 +1654,7 @@ fn smj_join_key_ordering() -> Result<()> { "RepartitionExec: partitioning=Hash([b1@0, a1@1], 10), input_partitions=10", "AggregateExec: mode=Partial, gby=[b@1 as b1, a@0 as a1], aggr=[]", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", "RepartitionExec: partitioning=Hash([b2@1, a2@0], 10), input_partitions=10, preserve_order=true, sort_exprs=b2@1 ASC, a2@0 ASC", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[b2@1 ASC, a2@0 ASC], preserve_partitioning=[false]", @@ -1640,7 +1664,7 @@ fn smj_join_key_ordering() -> Result<()> { "RepartitionExec: partitioning=Hash([b@0, a@1], 10), input_partitions=10", "AggregateExec: mode=Partial, gby=[b@1 as b, a@0 as a], aggr=[]", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ]; assert_optimized!(expected_first_sort_enforcement, join, false, true); @@ -1671,7 +1695,7 @@ fn merge_does_not_need_sort() -> Result<()> { let expected = &[ "SortPreservingMergeExec: [a@0 ASC]", "CoalesceBatchesExec: target_batch_size=4096", - "DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", + "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", ]; assert_optimized!(expected, exec, true); @@ -1683,7 +1707,7 @@ fn merge_does_not_need_sort() -> Result<()> { "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", "CoalescePartitionsExec", "CoalesceBatchesExec: target_batch_size=4096", - "DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", + "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", ]; assert_optimized!(expected, exec, false); @@ -1719,12 +1743,12 @@ fn union_to_interleave() -> Result<()> { "RepartitionExec: partitioning=Hash([a1@0], 10), input_partitions=10", "AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[]", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", "AggregateExec: mode=FinalPartitioned, gby=[a1@0 as a1], aggr=[]", "RepartitionExec: partitioning=Hash([a1@0], 10), input_partitions=10", "AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[]", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ]; assert_optimized!(expected, plan.clone(), true); assert_optimized!(expected, plan.clone(), false); @@ -1762,12 +1786,12 @@ fn union_not_to_interleave() -> Result<()> { "RepartitionExec: partitioning=Hash([a1@0], 10), input_partitions=10", "AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[]", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", "AggregateExec: mode=FinalPartitioned, gby=[a1@0 as a1], aggr=[]", "RepartitionExec: partitioning=Hash([a1@0], 10), input_partitions=10", "AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[]", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ]; // no sort in the plan but since we need it as a parameter, make it default false let prefer_existing_sort = false; @@ -1802,7 +1826,7 @@ fn added_repartition_to_single_partition() -> Result<()> { "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ]; assert_optimized!(expected, plan.clone(), true); assert_optimized!(expected, plan, false); @@ -1821,7 +1845,7 @@ fn repartition_deepest_node() -> Result<()> { "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", "FilterExec: c@2 = 0", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ]; assert_optimized!(expected, plan.clone(), true); assert_optimized!(expected, plan, false); @@ -1841,7 +1865,7 @@ fn repartition_unsorted_limit() -> Result<()> { "FilterExec: c@2 = 0", // nothing sorts the data, so the local limit doesn't require sorted data either "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ]; assert_optimized!(expected, plan.clone(), true); @@ -1864,7 +1888,7 @@ fn repartition_sorted_limit() -> Result<()> { "LocalLimitExec: fetch=100", // data is sorted so can't repartition here "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ]; assert_optimized!(expected, plan.clone(), true); assert_optimized!(expected, plan, false); @@ -1891,7 +1915,7 @@ fn repartition_sorted_limit_with_filter() -> Result<()> { // is still satisfied. "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ]; assert_optimized!(expected, plan.clone(), true); @@ -1922,7 +1946,7 @@ fn repartition_ignores_limit() -> Result<()> { "GlobalLimitExec: skip=0, fetch=100", "LocalLimitExec: fetch=100", // Expect no repartition to happen for local limit - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ]; assert_optimized!(expected, plan.clone(), true); assert_optimized!(expected, plan, false); @@ -1936,12 +1960,12 @@ fn repartition_ignores_union() -> Result<()> { let expected = &[ "UnionExec", - // Expect no repartition of DataSourceExec - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + // Expect no repartition of ParquetExec + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ]; assert_optimized!(expected, plan.clone(), true); @@ -1963,7 +1987,7 @@ fn repartition_through_sort_preserving_merge() -> Result<()> { // need resort as the data was not sorted correctly let expected = &[ "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ]; assert_optimized!(expected, plan.clone(), true); assert_optimized!(expected, plan, false); @@ -1988,7 +2012,7 @@ fn repartition_ignores_sort_preserving_merge() -> Result<()> { // should not repartition, since increased parallelism is not beneficial for SortPReservingMerge let expected = &[ "SortPreservingMergeExec: [c@2 ASC]", - "DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", + "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", ]; assert_optimized!(expected, plan.clone(), true); @@ -1996,7 +2020,7 @@ fn repartition_ignores_sort_preserving_merge() -> Result<()> { let expected = &[ "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", "CoalescePartitionsExec", - "DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", + "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", ]; assert_optimized!(expected, plan, false); @@ -2018,8 +2042,8 @@ fn repartition_ignores_sort_preserving_merge_with_union() -> Result<()> { let expected = &[ "SortPreservingMergeExec: [c@2 ASC]", "UnionExec", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", ]; assert_optimized!(expected, plan.clone(), true); @@ -2028,8 +2052,8 @@ fn repartition_ignores_sort_preserving_merge_with_union() -> Result<()> { "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", "CoalescePartitionsExec", "UnionExec", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", ]; assert_optimized!(expected, plan, false); @@ -2055,7 +2079,7 @@ fn repartition_does_not_destroy_sort() -> Result<()> { "SortRequiredExec: [d@3 ASC]", "FilterExec: c@2 = 0", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[d@3 ASC], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[d@3 ASC]", ]; assert_optimized!(expected, plan.clone(), true, true); @@ -2093,11 +2117,11 @@ fn repartition_does_not_destroy_sort_more_complex() -> Result<()> { "UnionExec", // union input 1: no repartitioning "SortRequiredExec: [c@2 ASC]", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", // union input 2: should repartition "FilterExec: c@2 = 0", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ]; assert_optimized!(expected, plan.clone(), true); assert_optimized!(expected, plan, false); @@ -2130,7 +2154,7 @@ fn repartition_transitively_with_projection() -> Result<()> { // Since this projection is not trivial, increasing parallelism is beneficial "ProjectionExec: expr=[a@0 + b@1 as sum]", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ]; assert_optimized!(expected, plan.clone(), true); @@ -2141,7 +2165,7 @@ fn repartition_transitively_with_projection() -> Result<()> { // Since this projection is not trivial, increasing parallelism is beneficial "ProjectionExec: expr=[a@0 + b@1 as sum]", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ]; assert_optimized!(expected_first_sort_enforcement, plan, false); @@ -2173,7 +2197,7 @@ fn repartition_ignores_transitively_with_projection() -> Result<()> { "SortRequiredExec: [c@2 ASC]", // Since this projection is trivial, increasing parallelism is not beneficial "ProjectionExec: expr=[a@0 as a, b@1 as b, c@2 as c]", - "DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", + "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", ]; assert_optimized!(expected, plan.clone(), true); assert_optimized!(expected, plan, false); @@ -2206,7 +2230,7 @@ fn repartition_transitively_past_sort_with_projection() -> Result<()> { "SortExec: expr=[c@2 ASC], preserve_partitioning=[true]", // Since this projection is trivial, increasing parallelism is not beneficial "ProjectionExec: expr=[a@0 as a, b@1 as b, c@2 as c]", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ]; assert_optimized!(expected, plan.clone(), true); assert_optimized!(expected, plan, false); @@ -2229,7 +2253,7 @@ fn repartition_transitively_past_sort_with_filter() -> Result<()> { // Expect repartition on the input to the sort (as it can benefit from additional parallelism) "FilterExec: c@2 = 0", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ]; assert_optimized!(expected, plan.clone(), true); @@ -2240,7 +2264,7 @@ fn repartition_transitively_past_sort_with_filter() -> Result<()> { "FilterExec: c@2 = 0", // Expect repartition on the input of the filter (as it can benefit from additional parallelism) "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ]; assert_optimized!(expected_first_sort_enforcement, plan, false); @@ -2276,7 +2300,7 @@ fn repartition_transitively_past_sort_with_projection_and_filter() -> Result<()> "FilterExec: c@2 = 0", // repartition is lowest down "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ]; assert_optimized!(expected, plan.clone(), true); @@ -2287,7 +2311,7 @@ fn repartition_transitively_past_sort_with_projection_and_filter() -> Result<()> "ProjectionExec: expr=[a@0 as a, b@1 as b, c@2 as c]", "FilterExec: c@2 = 0", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ]; assert_optimized!(expected_first_sort_enforcement, plan, false); @@ -2304,13 +2328,13 @@ fn parallelization_single_partition() -> Result<()> { "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", "RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2", "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", - "DataSourceExec: file_groups={2 groups: [[x:0..50], [x:50..100]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={2 groups: [[x:0..50], [x:50..100]]}, projection=[a, b, c, d, e]", ]; let expected_csv = [ "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", "RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2", "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", - "DataSourceExec: file_groups={2 groups: [[x:0..50], [x:50..100]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", + "CsvExec: file_groups={2 groups: [[x:0..50], [x:50..100]]}, projection=[a, b, c, d, e], has_header=false", ]; assert_optimized!(expected_parquet, plan_parquet, true, false, 2, true, 10); assert_optimized!(expected_csv, plan_csv, true, false, 2, true, 10); @@ -2335,7 +2359,7 @@ fn parallelization_multiple_files() -> Result<()> { let expected = [ "SortRequiredExec: [a@0 ASC]", "FilterExec: c@2 = 0", - "DataSourceExec: file_groups={3 groups: [[x:0..50], [y:0..100], [x:50..100]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", ]; + "ParquetExec: file_groups={3 groups: [[x:0..50], [y:0..100], [x:50..100]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", ]; let target_partitions = 3; let repartition_size = 1; assert_optimized!( @@ -2352,7 +2376,7 @@ fn parallelization_multiple_files() -> Result<()> { let expected = [ "SortRequiredExec: [a@0 ASC]", "FilterExec: c@2 = 0", - "DataSourceExec: file_groups={8 groups: [[x:0..25], [y:0..25], [x:25..50], [y:25..50], [x:50..75], [y:50..75], [x:75..100], [y:75..100]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", + "ParquetExec: file_groups={8 groups: [[x:0..25], [y:0..25], [x:25..50], [y:25..50], [x:50..75], [y:50..75], [x:75..100], [y:75..100]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", ]; let target_partitions = 8; let repartition_size = 1; @@ -2371,7 +2395,7 @@ fn parallelization_multiple_files() -> Result<()> { } #[test] -/// DataSourceExec on compressed csv file will not be partitioned +/// CsvExec on compressed csv file will not be partitioned /// (Not able to decompress chunked csv file) fn parallelization_compressed_csv() -> Result<()> { let compression_types = [ @@ -2387,14 +2411,14 @@ fn parallelization_compressed_csv() -> Result<()> { "RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2", "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", "RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", + "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", ]; let expected_partitioned = [ "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", "RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2", "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", - "DataSourceExec: file_groups={2 groups: [[x:0..50], [x:50..100]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", + "CsvExec: file_groups={2 groups: [[x:0..50], [x:50..100]]}, projection=[a, b, c, d, e], has_header=false", ]; for compression_type in compression_types { @@ -2431,14 +2455,14 @@ fn parallelization_two_partitions() -> Result<()> { "RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2", "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", // Plan already has two partitions - "DataSourceExec: file_groups={2 groups: [[x:0..100], [y:0..100]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={2 groups: [[x:0..100], [y:0..100]]}, projection=[a, b, c, d, e]", ]; let expected_csv = [ "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", "RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2", "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", // Plan already has two partitions - "DataSourceExec: file_groups={2 groups: [[x:0..100], [y:0..100]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", + "CsvExec: file_groups={2 groups: [[x:0..100], [y:0..100]]}, projection=[a, b, c, d, e], has_header=false", ]; assert_optimized!(expected_parquet, plan_parquet, true, false, 2, true, 10); assert_optimized!(expected_csv, plan_csv, true, false, 2, true, 10); @@ -2456,14 +2480,14 @@ fn parallelization_two_partitions_into_four() -> Result<()> { "RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4", "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", // Multiple source files splitted across partitions - "DataSourceExec: file_groups={4 groups: [[x:0..50], [x:50..100], [y:0..50], [y:50..100]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={4 groups: [[x:0..50], [x:50..100], [y:0..50], [y:50..100]]}, projection=[a, b, c, d, e]", ]; let expected_csv = [ "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", "RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4", "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", // Multiple source files splitted across partitions - "DataSourceExec: file_groups={4 groups: [[x:0..50], [x:50..100], [y:0..50], [y:50..100]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", + "CsvExec: file_groups={4 groups: [[x:0..50], [x:50..100], [y:0..50], [y:50..100]]}, projection=[a, b, c, d, e], has_header=false", ]; assert_optimized!(expected_parquet, plan_parquet, true, false, 4, true, 10); assert_optimized!(expected_csv, plan_csv, true, false, 4, true, 10); @@ -2487,7 +2511,7 @@ fn parallelization_sorted_limit() -> Result<()> { // data is sorted so can't repartition here "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", // Doesn't parallelize for SortExec without preserve_partitioning - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ]; let expected_csv = &[ "GlobalLimitExec: skip=0, fetch=100", @@ -2495,7 +2519,7 @@ fn parallelization_sorted_limit() -> Result<()> { // data is sorted so can't repartition here "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", // Doesn't parallelize for SortExec without preserve_partitioning - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", + "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", ]; assert_optimized!(expected_parquet, plan_parquet, true); assert_optimized!(expected_csv, plan_csv, true); @@ -2527,7 +2551,7 @@ fn parallelization_limit_with_filter() -> Result<()> { "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", // SortExec doesn't benefit from input partitioning - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ]; let expected_csv = &[ "GlobalLimitExec: skip=0, fetch=100", @@ -2539,7 +2563,7 @@ fn parallelization_limit_with_filter() -> Result<()> { "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", // SortExec doesn't benefit from input partitioning - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", + "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", ]; assert_optimized!(expected_parquet, plan_parquet, true); assert_optimized!(expected_csv, plan_csv, true); @@ -2571,7 +2595,7 @@ fn parallelization_ignores_limit() -> Result<()> { "GlobalLimitExec: skip=0, fetch=100", // Limit doesn't benefit from input partitioning - no parallelism "LocalLimitExec: fetch=100", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ]; let expected_csv = &[ "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", @@ -2587,7 +2611,7 @@ fn parallelization_ignores_limit() -> Result<()> { "GlobalLimitExec: skip=0, fetch=100", // Limit doesn't benefit from input partitioning - no parallelism "LocalLimitExec: fetch=100", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", + "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", ]; assert_optimized!(expected_parquet, plan_parquet, true); assert_optimized!(expected_csv, plan_csv, true); @@ -2603,20 +2627,20 @@ fn parallelization_union_inputs() -> Result<()> { let expected_parquet = &[ "UnionExec", // Union doesn't benefit from input partitioning - no parallelism - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ]; let expected_csv = &[ "UnionExec", // Union doesn't benefit from input partitioning - no parallelism - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", + "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", ]; assert_optimized!(expected_parquet, plan_parquet, true); assert_optimized!(expected_csv, plan_csv, true); @@ -2641,10 +2665,10 @@ fn parallelization_prior_to_sort_preserving_merge() -> Result<()> { // parallelization is not beneficial for SortPreservingMerge let expected_parquet = &[ - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", ]; let expected_csv = &[ - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=csv, has_header=false", + "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], has_header=false", ]; assert_optimized!(expected_parquet, plan_parquet, true); assert_optimized!(expected_csv, plan_csv, true); @@ -2671,14 +2695,14 @@ fn parallelization_sort_preserving_merge_with_union() -> Result<()> { let expected_parquet = &[ "SortPreservingMergeExec: [c@2 ASC]", "UnionExec", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", ]; let expected_csv = &[ "SortPreservingMergeExec: [c@2 ASC]", "UnionExec", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=csv, has_header=false", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=csv, has_header=false", + "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], has_header=false", + "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], has_header=false", ]; assert_optimized!(expected_parquet, plan_parquet, true); assert_optimized!(expected_csv, plan_csv, true); @@ -2705,11 +2729,11 @@ fn parallelization_does_not_benefit() -> Result<()> { // no parallelization, because SortRequiredExec doesn't benefit from increased parallelism let expected_parquet = &[ "SortRequiredExec: [c@2 ASC]", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", ]; let expected_csv = &[ "SortRequiredExec: [c@2 ASC]", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=csv, has_header=false", + "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], has_header=false", ]; assert_optimized!(expected_parquet, plan_parquet, true); assert_optimized!(expected_csv, plan_csv, true); @@ -2742,14 +2766,14 @@ fn parallelization_ignores_transitively_with_projection_parquet() -> Result<()> let expected = &[ "SortPreservingMergeExec: [c2@1 ASC]", " ProjectionExec: expr=[a@0 as a2, c@2 as c2]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", ]; plans_matches_expected!(expected, &plan_parquet); // data should not be repartitioned / resorted let expected_parquet = &[ "ProjectionExec: expr=[a@0 as a2, c@2 as c2]", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", ]; assert_optimized!(expected_parquet, plan_parquet, true); @@ -2781,14 +2805,14 @@ fn parallelization_ignores_transitively_with_projection_csv() -> Result<()> { let expected = &[ "SortPreservingMergeExec: [c2@1 ASC]", " ProjectionExec: expr=[a@0 as a2, c@2 as c2]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=csv, has_header=false", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], has_header=false", ]; plans_matches_expected!(expected, &plan_csv); // data should not be repartitioned / resorted let expected_csv = &[ "ProjectionExec: expr=[a@0 as a2, c@2 as c2]", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=csv, has_header=false", + "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], has_header=false", ]; assert_optimized!(expected_csv, plan_csv, true); @@ -2805,14 +2829,14 @@ fn remove_redundant_roundrobins() -> Result<()> { " FilterExec: c@2 = 0", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ]; plans_matches_expected!(expected, &physical_plan); let expected = &[ "FilterExec: c@2 = 0", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ]; assert_optimized!(expected, physical_plan.clone(), true); assert_optimized!(expected, physical_plan, false); @@ -2836,7 +2860,7 @@ fn remove_unnecessary_spm_after_filter() -> Result<()> { "CoalescePartitionsExec", "FilterExec: c@2 = 0", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2, preserve_order=true, sort_exprs=c@2 ASC", - "DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", + "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", ]; // last flag sets config.optimizer.PREFER_EXISTING_SORT assert_optimized!(expected, physical_plan.clone(), true, true); @@ -2859,7 +2883,7 @@ fn preserve_ordering_through_repartition() -> Result<()> { "SortPreservingMergeExec: [d@3 ASC]", "FilterExec: c@2 = 0", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2, preserve_order=true, sort_exprs=d@3 ASC", - "DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[d@3 ASC], file_type=parquet", + "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[d@3 ASC]", ]; // last flag sets config.optimizer.PREFER_EXISTING_SORT assert_optimized!(expected, physical_plan.clone(), true, true); @@ -2883,7 +2907,7 @@ fn do_not_preserve_ordering_through_repartition() -> Result<()> { "SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", "FilterExec: c@2 = 0", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", - "DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", + "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", ]; assert_optimized!(expected, physical_plan.clone(), true); @@ -2893,7 +2917,7 @@ fn do_not_preserve_ordering_through_repartition() -> Result<()> { "CoalescePartitionsExec", "FilterExec: c@2 = 0", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", - "DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", + "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", ]; assert_optimized!(expected, physical_plan, false); @@ -2916,7 +2940,7 @@ fn no_need_for_sort_after_filter() -> Result<()> { // Since after this stage c is constant. c@2 ASC ordering is already satisfied. "FilterExec: c@2 = 0", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", - "DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", + "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", ]; assert_optimized!(expected, physical_plan.clone(), true); assert_optimized!(expected, physical_plan, false); @@ -2944,7 +2968,7 @@ fn do_not_preserve_ordering_through_repartition2() -> Result<()> { "SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", "FilterExec: c@2 = 0", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", - "DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", + "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", ]; assert_optimized!(expected, physical_plan.clone(), true); @@ -2955,7 +2979,7 @@ fn do_not_preserve_ordering_through_repartition2() -> Result<()> { "SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", "FilterExec: c@2 = 0", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", - "DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", + "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", ]; assert_optimized!(expected, physical_plan, false); @@ -2975,7 +2999,7 @@ fn do_not_preserve_ordering_through_repartition3() -> Result<()> { let expected = &[ "FilterExec: c@2 = 0", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", - "DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", + "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", ]; assert_optimized!(expected, physical_plan.clone(), true); assert_optimized!(expected, physical_plan, false); @@ -2997,7 +3021,7 @@ fn do_not_put_sort_when_input_is_invalid() -> Result<()> { // by existing ordering at the source. "SortRequiredExec: [a@0 ASC]", "FilterExec: c@2 = 0", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ]; assert_plan_txt!(expected, physical_plan); @@ -3007,7 +3031,7 @@ fn do_not_put_sort_when_input_is_invalid() -> Result<()> { // EnforceDistribution rule doesn't satisfy this requirement either. "FilterExec: c@2 = 0", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ]; let mut config = ConfigOptions::new(); @@ -3035,7 +3059,7 @@ fn put_sort_when_input_is_valid() -> Result<()> { // by existing ordering at the source. "SortRequiredExec: [a@0 ASC]", "FilterExec: c@2 = 0", - "DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", + "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", ]; assert_plan_txt!(expected, physical_plan); @@ -3044,7 +3068,7 @@ fn put_sort_when_input_is_valid() -> Result<()> { // EnforceDistribution rule satisfy this requirement also. "SortRequiredExec: [a@0 ASC]", "FilterExec: c@2 = 0", - "DataSourceExec: file_groups={10 groups: [[x:0..20], [y:0..20], [x:20..40], [y:20..40], [x:40..60], [y:40..60], [x:60..80], [y:60..80], [x:80..100], [y:80..100]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", + "ParquetExec: file_groups={10 groups: [[x:0..20], [y:0..20], [x:20..40], [y:20..40], [x:40..60], [y:40..60], [x:60..80], [y:60..80], [x:80..100], [y:80..100]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", ]; let mut config = ConfigOptions::new(); @@ -3071,7 +3095,7 @@ fn do_not_add_unnecessary_hash() -> Result<()> { let expected = &[ "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", ]; // Make sure target partition number is 1. In this case hash repartition is unnecessary assert_optimized!(expected, physical_plan.clone(), true, false, 1, false, 1024); @@ -3101,7 +3125,7 @@ fn do_not_add_unnecessary_hash2() -> Result<()> { "RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4", "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", "RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2", - "DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", + "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", ]; // Make sure target partition number is larger than 2 (e.g partition number at the source). assert_optimized!(expected, physical_plan.clone(), true, false, 4, false, 1024); @@ -3116,12 +3140,12 @@ fn optimize_away_unnecessary_repartition() -> Result<()> { let expected = &[ "CoalescePartitionsExec", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ]; plans_matches_expected!(expected, physical_plan.clone()); let expected = - &["DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet"]; + &["ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]"]; assert_optimized!(expected, physical_plan.clone(), true); assert_optimized!(expected, physical_plan, false); @@ -3139,7 +3163,7 @@ fn optimize_away_unnecessary_repartition2() -> Result<()> { " CoalescePartitionsExec", " FilterExec: c@2 = 0", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ]; plans_matches_expected!(expected, physical_plan.clone()); @@ -3147,7 +3171,7 @@ fn optimize_away_unnecessary_repartition2() -> Result<()> { "FilterExec: c@2 = 0", "FilterExec: c@2 = 0", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ]; assert_optimized!(expected, physical_plan.clone(), true); assert_optimized!(expected, physical_plan, false); diff --git a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs index 3412b962d8598..64bd72579436d 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs @@ -46,9 +46,10 @@ use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeE use datafusion_physical_plan::{get_plan_string, ExecutionPlan}; use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::{TreeNode, TransformedResult}; -use datafusion::datasource::physical_plan::{CsvSource, FileScanConfig, ParquetSource}; +use datafusion::datasource::physical_plan::{CsvExec, FileScanConfig, ParquetExec}; use datafusion_execution::object_store::ObjectStoreUrl; use datafusion::datasource::listing::PartitionedFile; +use datafusion::datasource::file_format::file_compression_type::FileCompressionType; use datafusion_physical_optimizer::enforce_distribution::EnforceDistribution; use datafusion_physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; use datafusion_physical_plan::sorts::sort::SortExec; @@ -62,10 +63,23 @@ fn csv_exec_ordered( ) -> Arc { let sort_exprs = sort_exprs.into_iter().collect(); - FileScanConfig::new( - ObjectStoreUrl::parse("test:///").unwrap(), - schema.clone(), - Arc::new(CsvSource::new(true, 0, b'"')), + Arc::new( + CsvExec::builder( + FileScanConfig::new( + ObjectStoreUrl::parse("test:///").unwrap(), + schema.clone(), + ) + .with_file(PartitionedFile::new("file_path".to_string(), 100)) + .with_output_ordering(vec![sort_exprs]), + ) + .with_has_header(true) + .with_delimeter(0) + .with_quote(b'"') + .with_escape(None) + .with_comment(None) + .with_newlines_in_values(false) + .with_file_compression_type(FileCompressionType::UNCOMPRESSED) + .build(), ) .with_file(PartitionedFile::new("file_path".to_string(), 100)) .with_output_ordering(vec![sort_exprs]) @@ -79,11 +93,10 @@ pub fn parquet_exec_sorted( ) -> Arc { let sort_exprs = sort_exprs.into_iter().collect(); - let source = Arc::new(ParquetSource::default()); - FileScanConfig::new( - ObjectStoreUrl::parse("test:///").unwrap(), - schema.clone(), - source, + ParquetExec::builder( + FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema.clone()) + .with_file(PartitionedFile::new("x".to_string(), 100)) + .with_output_ordering(vec![sort_exprs]), ) .with_file(PartitionedFile::new("x".to_string(), 100)) .with_output_ordering(vec![sort_exprs]) @@ -97,10 +110,23 @@ fn csv_exec_sorted( ) -> Arc { let sort_exprs = sort_exprs.into_iter().collect(); - FileScanConfig::new( - ObjectStoreUrl::parse("test:///").unwrap(), - schema.clone(), - Arc::new(CsvSource::new(false, 0, 0)), + Arc::new( + CsvExec::builder( + FileScanConfig::new( + ObjectStoreUrl::parse("test:///").unwrap(), + schema.clone(), + ) + .with_file(PartitionedFile::new("x".to_string(), 100)) + .with_output_ordering(vec![sort_exprs]), + ) + .with_has_header(false) + .with_delimeter(0) + .with_quote(0) + .with_escape(None) + .with_comment(None) + .with_newlines_in_values(false) + .with_file_compression_type(FileCompressionType::UNCOMPRESSED) + .build(), ) .with_file(PartitionedFile::new("x".to_string(), 100)) .with_output_ordering(vec![sort_exprs]) @@ -211,12 +237,12 @@ async fn test_remove_unnecessary_sort5() -> Result<()> { let expected_input = ["SortExec: expr=[a@2 ASC], preserve_partitioning=[false]", " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(col_a@0, c@2)]", - " DataSourceExec: partitions=1, partition_sizes=[0]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet"]; + " MemoryExec: partitions=1, partition_sizes=[0]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]"]; let expected_optimized = ["HashJoinExec: mode=Partitioned, join_type=Inner, on=[(col_a@0, c@2)]", - " DataSourceExec: partitions=1, partition_sizes=[0]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet"]; + " MemoryExec: partitions=1, partition_sizes=[0]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -255,11 +281,11 @@ async fn test_bounded_window_set_monotonic_no_partition() -> Result<()> { "SortExec: expr=[count@2 ASC NULLS LAST], preserve_partitioning=[false]", " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", " SortExec: expr=[nullable_col@0 DESC NULLS LAST], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", ]; let expected_optimized = [ "BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -306,13 +332,13 @@ async fn test_bounded_plain_window_set_monotonic_with_partitions() -> Result<()> "SortExec: expr=[count@2 ASC NULLS LAST], preserve_partitioning=[false]", " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", " SortExec: expr=[nullable_col@0 DESC NULLS LAST], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", ]; let expected_optimized = [ "SortExec: expr=[count@2 ASC NULLS LAST], preserve_partitioning=[false]", " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", " SortExec: expr=[nullable_col@0 ASC NULLS LAST], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -369,12 +395,12 @@ async fn test_bounded_plain_window_set_monotonic_with_partitions_partial() -> Re "SortExec: expr=[nullable_col@0 DESC NULLS LAST, count@2 ASC NULLS LAST], preserve_partitioning=[false]", " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", " SortExec: expr=[nullable_col@0 DESC NULLS LAST], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", ]; let expected_optimized = [ "BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", " SortExec: expr=[nullable_col@0 DESC NULLS LAST, count@2 ASC NULLS LAST], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -412,12 +438,12 @@ async fn test_bounded_window_non_set_monotonic_sort() -> Result<()> { "SortExec: expr=[avg@5 ASC NULLS LAST], preserve_partitioning=[false]", " BoundedWindowAggExec: wdw=[avg: Ok(Field { name: \"avg\", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", " SortExec: expr=[a@0 DESC NULLS LAST], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 DESC NULLS LAST], file_type=parquet", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 DESC NULLS LAST]", ]; let expected_optimized = [ "SortExec: expr=[avg@5 ASC NULLS LAST], preserve_partitioning=[false]", " BoundedWindowAggExec: wdw=[avg: Ok(Field { name: \"avg\", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 DESC NULLS LAST], file_type=parquet", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 DESC NULLS LAST]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -446,22 +472,22 @@ async fn test_do_not_remove_sort_with_limit() -> Result<()> { let expected_input = ["SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", " UnionExec", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC], file_type=parquet", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", " GlobalLimitExec: skip=0, fetch=100", " LocalLimitExec: fetch=100", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; // We should keep the bottom `SortExec`. let expected_optimized = ["SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[true]", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", " UnionExec", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC], file_type=parquet", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", " GlobalLimitExec: skip=0, fetch=100", " LocalLimitExec: fetch=100", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -484,9 +510,9 @@ async fn test_union_inputs_sorted() -> Result<()> { let expected_input = vec![ "SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC], file_type=parquet", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", ]; // should not add a sort at the output of the union, input plan should not be changed let expected_optimized = expected_input.clone(); @@ -516,9 +542,9 @@ async fn test_union_inputs_different_sorted() -> Result<()> { let expected_input = vec![ "SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC, non_nullable_col@1 ASC], file_type=parquet", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC, non_nullable_col@1 ASC]", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", ]; // should not add a sort at the output of the union, input plan should not be changed let expected_optimized = expected_input.clone(); @@ -545,20 +571,20 @@ async fn test_union_inputs_different_sorted2() -> Result<()> { let physical_plan = sort_preserving_merge_exec(sort_exprs, union); // Input is an invalid plan. In this case rule should add required sorting in appropriate places. - // First DataSourceExec has output ordering(nullable_col@0 ASC). However, it doesn't satisfy the + // First ParquetExec has output ordering(nullable_col@0 ASC). However, it doesn't satisfy the // required ordering of SortPreservingMergeExec. let expected_input = ["SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", " UnionExec", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC], file_type=parquet", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; let expected_optimized = ["SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", " UnionExec", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC], file_type=parquet", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -589,18 +615,18 @@ async fn test_union_inputs_different_sorted3() -> Result<()> { let expected_input = ["SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC], file_type=parquet", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; // should adjust sorting in the first input of the union such that it is not unnecessarily fine let expected_optimized = ["SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC], file_type=parquet", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -631,18 +657,18 @@ async fn test_union_inputs_different_sorted4() -> Result<()> { let expected_input = ["SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", " UnionExec", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC], file_type=parquet", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; let expected_optimized = ["SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", " UnionExec", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC], file_type=parquet", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -681,15 +707,15 @@ async fn test_union_inputs_different_sorted5() -> Result<()> { let expected_input = ["SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 DESC NULLS LAST], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; let expected_optimized = ["SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -724,21 +750,21 @@ async fn test_union_inputs_different_sorted6() -> Result<()> { let expected_input = ["SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC], file_type=parquet", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", " SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; // Should adjust the requirement in the third input of the union so // that it is not unnecessarily fine. let expected_optimized = ["SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC], file_type=parquet", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[true]", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -764,16 +790,16 @@ async fn test_union_inputs_different_sorted7() -> Result<()> { let expected_input = ["SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; // Union preserves the inputs ordering and we should not change any of the SortExecs under UnionExec let expected_output = ["SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; assert_optimized!(expected_input, expected_output, physical_plan, true); Ok(()) @@ -815,14 +841,14 @@ async fn test_union_inputs_different_sorted8() -> Result<()> { // example below. let expected_input = ["UnionExec", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", " SortExec: expr=[nullable_col@0 DESC NULLS LAST, non_nullable_col@1 DESC NULLS LAST], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; // Since `UnionExec` doesn't preserve ordering in the plan above. // We shouldn't keep SortExecs in the plan. let expected_optimized = ["UnionExec", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -864,15 +890,15 @@ async fn test_window_multi_path_sort() -> Result<()> { " SortPreservingMergeExec: [nullable_col@0 DESC NULLS LAST]", " UnionExec", " SortExec: expr=[nullable_col@0 DESC NULLS LAST], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC, non_nullable_col@1 ASC], file_type=parquet", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC, non_nullable_col@1 ASC]", " SortExec: expr=[nullable_col@0 DESC NULLS LAST], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC], file_type=parquet"]; + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]"]; let expected_optimized = [ "WindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(NULL), is_causal: false }]", " SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC, non_nullable_col@1 ASC], file_type=parquet", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC], file_type=parquet"]; + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC, non_nullable_col@1 ASC]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -902,14 +928,14 @@ async fn test_window_multi_path_sort2() -> Result<()> { " SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", " UnionExec", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC], file_type=parquet", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC], file_type=parquet"]; + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]"]; let expected_optimized = ["BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", " SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC], file_type=parquet", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC], file_type=parquet"]; + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -949,19 +975,19 @@ async fn test_union_inputs_different_sorted_with_limit() -> Result<()> { let expected_input = ["SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", " GlobalLimitExec: skip=0, fetch=100", " LocalLimitExec: fetch=100", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 DESC NULLS LAST], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; let expected_optimized = ["SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", " GlobalLimitExec: skip=0, fetch=100", " LocalLimitExec: fetch=100", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 DESC NULLS LAST], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -1006,8 +1032,8 @@ async fn test_sort_merge_join_order_by_left() -> Result<()> { ); let expected_input = ["SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", join_plan2.as_str(), - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b], file_type=parquet"]; + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]"]; let expected_optimized = match join_type { JoinType::Inner | JoinType::Left @@ -1017,9 +1043,9 @@ async fn test_sort_merge_join_order_by_left() -> Result<()> { vec![ join_plan.as_str(), " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", " SortExec: expr=[col_a@0 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b], file_type=parquet", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]", ] } _ => { @@ -1028,9 +1054,9 @@ async fn test_sort_merge_join_order_by_left() -> Result<()> { "SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", join_plan2.as_str(), " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", " SortExec: expr=[col_a@0 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b], file_type=parquet", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]", ] } }; @@ -1081,17 +1107,17 @@ async fn test_sort_merge_join_order_by_right() -> Result<()> { ); let expected_input = [spm_plan, join_plan2.as_str(), - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b], file_type=parquet"]; + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]"]; let expected_optimized = match join_type { JoinType::Inner | JoinType::Right | JoinType::RightAnti => { // can push down the sort requirements and save 1 SortExec vec![ join_plan.as_str(), " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", " SortExec: expr=[col_a@0 ASC, col_b@1 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b], file_type=parquet", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]", ] } _ => { @@ -1100,9 +1126,9 @@ async fn test_sort_merge_join_order_by_right() -> Result<()> { "SortExec: expr=[col_a@2 ASC, col_b@3 ASC], preserve_partitioning=[false]", join_plan2.as_str(), " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", " SortExec: expr=[col_a@0 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b], file_type=parquet", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]", ] } }; @@ -1136,16 +1162,16 @@ async fn test_sort_merge_join_complex_order_by() -> Result<()> { let expected_input = ["SortPreservingMergeExec: [col_b@3 ASC, col_a@2 ASC]", " SortMergeJoin: join_type=Inner, on=[(nullable_col@0, col_a@0)]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b], file_type=parquet"]; + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]"]; // can not push down the sort requirements, need to add SortExec let expected_optimized = ["SortExec: expr=[col_b@3 ASC, col_a@2 ASC], preserve_partitioning=[false]", " SortMergeJoin: join_type=Inner, on=[(nullable_col@0, col_a@0)]", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", " SortExec: expr=[col_a@0 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b], file_type=parquet"]; + " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); // order by (nullable_col, col_b, col_a) @@ -1158,16 +1184,16 @@ async fn test_sort_merge_join_complex_order_by() -> Result<()> { let expected_input = ["SortPreservingMergeExec: [nullable_col@0 ASC, col_b@3 ASC, col_a@2 ASC]", " SortMergeJoin: join_type=Inner, on=[(nullable_col@0, col_a@0)]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b], file_type=parquet"]; + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]"]; // can not push down the sort requirements, need to add SortExec let expected_optimized = ["SortExec: expr=[nullable_col@0 ASC, col_b@3 ASC, col_a@2 ASC], preserve_partitioning=[false]", " SortMergeJoin: join_type=Inner, on=[(nullable_col@0, col_a@0)]", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", " SortExec: expr=[col_a@0 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b], file_type=parquet"]; + " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -1197,12 +1223,12 @@ async fn test_multilayer_coalesce_partitions() -> Result<()> { " FilterExec: NOT non_nullable_col@1", " CoalescePartitionsExec", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; let expected_optimized = ["SortPreservingMergeExec: [nullable_col@0 ASC]", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[true]", " FilterExec: NOT non_nullable_col@1", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -1225,12 +1251,12 @@ async fn test_with_lost_ordering_bounded() -> Result<()> { " CoalescePartitionsExec", " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=csv, has_header=false"]; + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], has_header=false"]; let expected_optimized = ["SortPreservingMergeExec: [a@0 ASC]", " SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=csv, has_header=false"]; + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], has_header=false"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -1270,7 +1296,7 @@ async fn test_with_lost_ordering_unbounded_bounded( " CoalescePartitionsExec", " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=csv, has_header=true", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], has_header=true", ]; // Expected unbounded result (same for with and without flag) @@ -1287,14 +1313,14 @@ async fn test_with_lost_ordering_unbounded_bounded( " CoalescePartitionsExec", " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=csv, has_header=true", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], has_header=true", ]; let expected_optimized_bounded_parallelize_sort = vec![ "SortPreservingMergeExec: [a@0 ASC]", " SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=csv, has_header=true", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], has_header=true", ]; let (expected_input, expected_optimized, expected_optimized_sort_parallelize) = if source_unbounded { @@ -1338,11 +1364,11 @@ async fn test_do_not_pushdown_through_spm() -> Result<()> { let expected_input = ["SortExec: expr=[b@1 ASC], preserve_partitioning=[false]", " SortPreservingMergeExec: [a@0 ASC, b@1 ASC]", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC, b@1 ASC], file_type=csv, has_header=false",]; + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC, b@1 ASC], has_header=false",]; let expected_optimized = ["SortExec: expr=[b@1 ASC], preserve_partitioning=[false]", " SortPreservingMergeExec: [a@0 ASC, b@1 ASC]", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC, b@1 ASC], file_type=csv, has_header=false",]; + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC, b@1 ASC], has_header=false",]; assert_optimized!(expected_input, expected_optimized, physical_plan, false); Ok(()) @@ -1367,11 +1393,11 @@ async fn test_pushdown_through_spm() -> Result<()> { let expected_input = ["SortExec: expr=[a@0 ASC, b@1 ASC, c@2 ASC], preserve_partitioning=[false]", " SortPreservingMergeExec: [a@0 ASC, b@1 ASC]", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC, b@1 ASC], file_type=csv, has_header=false",]; + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC, b@1 ASC], has_header=false",]; let expected_optimized = ["SortPreservingMergeExec: [a@0 ASC, b@1 ASC]", " SortExec: expr=[a@0 ASC, b@1 ASC, c@2 ASC], preserve_partitioning=[true]", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC, b@1 ASC], file_type=csv, has_header=false",]; + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC, b@1 ASC], has_header=false",]; assert_optimized!(expected_input, expected_optimized, physical_plan, false); Ok(()) @@ -1395,7 +1421,7 @@ async fn test_window_multi_layer_requirement() -> Result<()> { " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC, b@1 ASC", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", " SortExec: expr=[a@0 ASC, b@1 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", ]; let expected_optimized = [ "BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", @@ -1403,7 +1429,7 @@ async fn test_window_multi_layer_requirement() -> Result<()> { " CoalescePartitionsExec", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, false); @@ -1425,7 +1451,7 @@ async fn test_not_replaced_with_partial_sort_for_bounded_input() -> Result<()> { ); let expected_input = [ "SortExec: expr=[a@0 ASC, b@1 ASC, c@2 ASC], preserve_partitioning=[false]", - " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[b@1 ASC, c@2 ASC], file_type=parquet" + " ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[b@1 ASC, c@2 ASC]" ]; let expected_no_change = expected_input; assert_optimized!(expected_input, expected_no_change, physical_plan, false); @@ -1528,11 +1554,11 @@ async fn test_remove_unnecessary_sort() -> Result<()> { let expected_input = [ "SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", " SortExec: expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", - " DataSourceExec: partitions=1, partition_sizes=[0]", + " MemoryExec: partitions=1, partition_sizes=[0]", ]; let expected_optimized = [ "SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " DataSourceExec: partitions=1, partition_sizes=[0]", + " MemoryExec: partitions=1, partition_sizes=[0]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -1586,14 +1612,14 @@ async fn test_remove_unnecessary_sort_window_multilayer() -> Result<()> { " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", " CoalesceBatchesExec: target_batch_size=128", " SortExec: expr=[non_nullable_col@1 DESC], preserve_partitioning=[false]", - " DataSourceExec: partitions=1, partition_sizes=[0]"]; + " MemoryExec: partitions=1, partition_sizes=[0]"]; let expected_optimized = ["WindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(NULL), is_causal: false }]", " FilterExec: NOT non_nullable_col@1", " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", " CoalesceBatchesExec: target_batch_size=128", " SortExec: expr=[non_nullable_col@1 DESC], preserve_partitioning=[false]", - " DataSourceExec: partitions=1, partition_sizes=[0]"]; + " MemoryExec: partitions=1, partition_sizes=[0]"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -1610,11 +1636,11 @@ async fn test_add_required_sort() -> Result<()> { let expected_input = [ "SortPreservingMergeExec: [nullable_col@0 ASC]", - " DataSourceExec: partitions=1, partition_sizes=[0]", + " MemoryExec: partitions=1, partition_sizes=[0]", ]; let expected_optimized = [ "SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " DataSourceExec: partitions=1, partition_sizes=[0]", + " MemoryExec: partitions=1, partition_sizes=[0]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -1637,11 +1663,11 @@ async fn test_remove_unnecessary_sort1() -> Result<()> { " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", " SortPreservingMergeExec: [nullable_col@0 ASC]", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " DataSourceExec: partitions=1, partition_sizes=[0]", + " MemoryExec: partitions=1, partition_sizes=[0]", ]; let expected_optimized = [ "SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " DataSourceExec: partitions=1, partition_sizes=[0]", + " MemoryExec: partitions=1, partition_sizes=[0]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -1675,13 +1701,13 @@ async fn test_remove_unnecessary_sort2() -> Result<()> { " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", " SortPreservingMergeExec: [non_nullable_col@1 ASC]", " SortExec: expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", - " DataSourceExec: partitions=1, partition_sizes=[0]", + " MemoryExec: partitions=1, partition_sizes=[0]", ]; let expected_optimized = [ "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: partitions=1, partition_sizes=[0]", + " MemoryExec: partitions=1, partition_sizes=[0]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -1719,14 +1745,14 @@ async fn test_remove_unnecessary_sort3() -> Result<()> { " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", " SortPreservingMergeExec: [non_nullable_col@1 ASC]", " SortExec: expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", - " DataSourceExec: partitions=1, partition_sizes=[0]", + " MemoryExec: partitions=1, partition_sizes=[0]", ]; let expected_optimized = [ "AggregateExec: mode=Final, gby=[], aggr=[]", " CoalescePartitionsExec", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: partitions=1, partition_sizes=[0]", + " MemoryExec: partitions=1, partition_sizes=[0]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -1770,18 +1796,18 @@ async fn test_remove_unnecessary_sort4() -> Result<()> { " SortExec: expr=[non_nullable_col@1 ASC], preserve_partitioning=[true]", " UnionExec", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: partitions=1, partition_sizes=[0]", + " MemoryExec: partitions=1, partition_sizes=[0]", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: partitions=1, partition_sizes=[0]"]; + " MemoryExec: partitions=1, partition_sizes=[0]"]; let expected_optimized = ["SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[true]", " FilterExec: NOT non_nullable_col@1", " UnionExec", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: partitions=1, partition_sizes=[0]", + " MemoryExec: partitions=1, partition_sizes=[0]", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: partitions=1, partition_sizes=[0]"]; + " MemoryExec: partitions=1, partition_sizes=[0]"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -1809,11 +1835,11 @@ async fn test_remove_unnecessary_sort6() -> Result<()> { let expected_input = [ "SortExec: expr=[non_nullable_col@1 ASC, nullable_col@0 ASC], preserve_partitioning=[false]", " SortExec: TopK(fetch=2), expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", - " DataSourceExec: partitions=1, partition_sizes=[0]", + " MemoryExec: partitions=1, partition_sizes=[0]", ]; let expected_optimized = [ "SortExec: TopK(fetch=2), expr=[non_nullable_col@1 ASC, nullable_col@0 ASC], preserve_partitioning=[false]", - " DataSourceExec: partitions=1, partition_sizes=[0]", + " MemoryExec: partitions=1, partition_sizes=[0]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -1843,12 +1869,12 @@ async fn test_remove_unnecessary_sort7() -> Result<()> { let expected_input = [ "SortExec: TopK(fetch=2), expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", " SortExec: expr=[non_nullable_col@1 ASC, nullable_col@0 ASC], preserve_partitioning=[false]", - " DataSourceExec: partitions=1, partition_sizes=[0]", + " MemoryExec: partitions=1, partition_sizes=[0]", ]; let expected_optimized = [ "GlobalLimitExec: skip=0, fetch=2", " SortExec: expr=[non_nullable_col@1 ASC, nullable_col@0 ASC], preserve_partitioning=[false]", - " DataSourceExec: partitions=1, partition_sizes=[0]", + " MemoryExec: partitions=1, partition_sizes=[0]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -1876,12 +1902,12 @@ async fn test_remove_unnecessary_sort8() -> Result<()> { "SortExec: expr=[non_nullable_col@1 ASC, nullable_col@0 ASC], preserve_partitioning=[false]", " LocalLimitExec: fetch=2", " SortExec: expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", - " DataSourceExec: partitions=1, partition_sizes=[0]", + " MemoryExec: partitions=1, partition_sizes=[0]", ]; let expected_optimized = [ "LocalLimitExec: fetch=2", " SortExec: TopK(fetch=2), expr=[non_nullable_col@1 ASC, nullable_col@0 ASC], preserve_partitioning=[false]", - " DataSourceExec: partitions=1, partition_sizes=[0]", + " MemoryExec: partitions=1, partition_sizes=[0]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -1904,13 +1930,13 @@ async fn test_do_not_pushdown_through_limit() -> Result<()> { "SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", " GlobalLimitExec: skip=0, fetch=5", " SortExec: expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", - " DataSourceExec: partitions=1, partition_sizes=[0]", + " MemoryExec: partitions=1, partition_sizes=[0]", ]; let expected_optimized = [ "SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", " GlobalLimitExec: skip=0, fetch=5", " SortExec: expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", - " DataSourceExec: partitions=1, partition_sizes=[0]", + " MemoryExec: partitions=1, partition_sizes=[0]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -1932,11 +1958,11 @@ async fn test_remove_unnecessary_spm1() -> Result<()> { "SortPreservingMergeExec: [nullable_col@0 ASC]", " SortPreservingMergeExec: [non_nullable_col@1 ASC]", " SortPreservingMergeExec: [non_nullable_col@1 ASC]", - " DataSourceExec: partitions=1, partition_sizes=[0]", + " MemoryExec: partitions=1, partition_sizes=[0]", ]; let expected_optimized = [ "SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " DataSourceExec: partitions=1, partition_sizes=[0]", + " MemoryExec: partitions=1, partition_sizes=[0]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -1980,11 +2006,11 @@ async fn test_change_wrong_sorting() -> Result<()> { let expected_input = [ "SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " DataSourceExec: partitions=1, partition_sizes=[0]", + " MemoryExec: partitions=1, partition_sizes=[0]", ]; let expected_optimized = [ "SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " DataSourceExec: partitions=1, partition_sizes=[0]", + " MemoryExec: partitions=1, partition_sizes=[0]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -2007,11 +2033,11 @@ async fn test_change_wrong_sorting2() -> Result<()> { "SortPreservingMergeExec: [non_nullable_col@1 ASC]", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", " SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", - " DataSourceExec: partitions=1, partition_sizes=[0]", + " MemoryExec: partitions=1, partition_sizes=[0]", ]; let expected_optimized = [ "SortExec: expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", - " DataSourceExec: partitions=1, partition_sizes=[0]", + " MemoryExec: partitions=1, partition_sizes=[0]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -2039,13 +2065,13 @@ async fn test_multiple_sort_window_exec() -> Result<()> { " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " DataSourceExec: partitions=1, partition_sizes=[0]"]; + " MemoryExec: partitions=1, partition_sizes=[0]"]; let expected_optimized = ["BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " DataSourceExec: partitions=1, partition_sizes=[0]"]; + " MemoryExec: partitions=1, partition_sizes=[0]"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -2073,7 +2099,7 @@ async fn test_commutativity() -> Result<()> { "SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", - " DataSourceExec: partitions=1, partition_sizes=[0]", + " MemoryExec: partitions=1, partition_sizes=[0]", ]; assert_eq!( expected_input, actual, @@ -2130,13 +2156,13 @@ async fn test_coalesce_propagate() -> Result<()> { " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", " CoalescePartitionsExec", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: partitions=1, partition_sizes=[0]", + " MemoryExec: partitions=1, partition_sizes=[0]", ]; let expected_optimized = [ "SortPreservingMergeExec: [nullable_col@0 ASC]", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[true]", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " DataSourceExec: partitions=1, partition_sizes=[0]", + " MemoryExec: partitions=1, partition_sizes=[0]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -2176,7 +2202,7 @@ async fn test_push_with_required_input_ordering_prohibited() -> Result<()> { // SortExec: expr=[b] <-- can't push this down // RequiredInputOrder expr=[a] <-- this requires input sorted by a, and preserves the input order // SortExec: expr=[a] - // DataSourceExec + // MemoryExec let schema = create_test_schema3()?; let sort_exprs_a = LexOrdering::new(vec![sort_expr("a", &schema)]); let sort_exprs_b = LexOrdering::new(vec![sort_expr("b", &schema)]); @@ -2192,7 +2218,7 @@ async fn test_push_with_required_input_ordering_prohibited() -> Result<()> { "SortExec: expr=[b@1 ASC], preserve_partitioning=[false]", " RequiredInputOrderingExec", " SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - " DataSourceExec: partitions=1, partition_sizes=[0]", + " MemoryExec: partitions=1, partition_sizes=[0]", ]; // should not be able to push shorts let expected_no_change = expected_input; @@ -2206,7 +2232,7 @@ async fn test_push_with_required_input_ordering_allowed() -> Result<()> { // SortExec: expr=[a,b] <-- can push this down (as it is compatible with the required input ordering) // RequiredInputOrder expr=[a] <-- this requires input sorted by a, and preserves the input order // SortExec: expr=[a] - // DataSourceExec + // MemoryExec let schema = create_test_schema3()?; let sort_exprs_a = LexOrdering::new(vec![sort_expr("a", &schema)]); let sort_exprs_ab = @@ -2223,13 +2249,13 @@ async fn test_push_with_required_input_ordering_allowed() -> Result<()> { "SortExec: expr=[a@0 ASC, b@1 ASC], preserve_partitioning=[false]", " RequiredInputOrderingExec", " SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - " DataSourceExec: partitions=1, partition_sizes=[0]", + " MemoryExec: partitions=1, partition_sizes=[0]", ]; // should able to push shorts let expected = [ "RequiredInputOrderingExec", " SortExec: expr=[a@0 ASC, b@1 ASC], preserve_partitioning=[false]", - " DataSourceExec: partitions=1, partition_sizes=[0]", + " MemoryExec: partitions=1, partition_sizes=[0]", ]; assert_optimized!(expected_input, expected, plan, true); Ok(()) diff --git a/datafusion/physical-expr/src/partitioning.rs b/datafusion/physical-expr/src/partitioning.rs index 2814470485a16..d7d2e74d521f4 100644 --- a/datafusion/physical-expr/src/partitioning.rs +++ b/datafusion/physical-expr/src/partitioning.rs @@ -235,6 +235,11 @@ impl PartialEq for Partitioning { { true } + (Partitioning::OnDemand(count1), Partitioning::OnDemand(count2)) + if count1 == count2 => + { + true + } _ => false, } } diff --git a/datafusion/physical-optimizer/src/coalesce_batches.rs b/datafusion/physical-optimizer/src/coalesce_batches.rs index 5cf2c877c61a4..48c336fc771f5 100644 --- a/datafusion/physical-optimizer/src/coalesce_batches.rs +++ b/datafusion/physical-optimizer/src/coalesce_batches.rs @@ -70,8 +70,7 @@ impl PhysicalOptimizerRule for CoalesceBatches { repart_exec.partitioning().clone(), Partitioning::RoundRobinBatch(_) ) - }) - .unwrap_or(false); + }).unwrap_or(false); if wrap_in_coalesce { Ok(Transformed::yes(Arc::new(CoalesceBatchesExec::new( plan, diff --git a/datafusion/physical-optimizer/src/enforce_distribution.rs b/datafusion/physical-optimizer/src/enforce_distribution.rs index 5e76edad1f569..6be72ebb5d9b0 100644 --- a/datafusion/physical-optimizer/src/enforce_distribution.rs +++ b/datafusion/physical-optimizer/src/enforce_distribution.rs @@ -52,6 +52,7 @@ use datafusion_physical_plan::joins::{ CrossJoinExec, HashJoinExec, PartitionMode, SortMergeJoinExec, }; use datafusion_physical_plan::projection::ProjectionExec; +use datafusion_physical_plan::repartition::on_demand_repartition::OnDemandRepartitionExec; use datafusion_physical_plan::repartition::RepartitionExec; use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; use datafusion_physical_plan::tree_node::PlanContext; @@ -1052,6 +1053,45 @@ fn replace_order_preserving_variants( context.update_plan_from_children() } +fn replace_round_robin_repartition_with_on_demand( + mut context: DistributionContext, +) -> Result { + context.children = context + .children + .into_iter() + .map(|child| { + if child.data { + replace_round_robin_repartition_with_on_demand(child) + } else { + Ok(child) + } + }) + .collect::>>()?; + + if let Some(repartition) = context.plan.as_any().downcast_ref::() { + if let Partitioning::RoundRobinBatch(n) = repartition.partitioning() { + let child_plan = Arc::clone(&context.children[0].plan); + context.plan = if repartition.preserve_order() { + Arc::new( + OnDemandRepartitionExec::try_new( + child_plan, + Partitioning::OnDemand(*n), + )? + .with_preserve_order(), + ) + } else { + Arc::new(OnDemandRepartitionExec::try_new( + child_plan, + Partitioning::OnDemand(*n), + )?) + }; + return Ok(context); + } + } + + context.update_plan_from_children() +} + /// A struct to keep track of repartition requirements for each child node. struct RepartitionRequirementStatus { /// The distribution requirement for the node. @@ -1168,6 +1208,8 @@ pub fn ensure_distribution( let target_partitions = config.execution.target_partitions; // When `false`, round robin repartition will not be added to increase parallelism let enable_round_robin = config.optimizer.enable_round_robin_repartition; + // When `false`, replace round robin repartition with on-demand repartition + let prefer_round_robin_repartition = config.optimizer.prefer_round_robin_repartition; let repartition_file_scans = config.optimizer.repartition_file_scans; let batch_size = config.execution.batch_size; let should_use_estimates = config @@ -1320,6 +1362,9 @@ pub fn ensure_distribution( } } } + if !prefer_round_robin_repartition { + child = replace_round_robin_repartition_with_on_demand(child)?; + } Ok(child) }, ) diff --git a/datafusion/physical-optimizer/src/enforce_sorting/mod.rs b/datafusion/physical-optimizer/src/enforce_sorting/mod.rs index 11f1d8751d83c..abc0fee060d65 100644 --- a/datafusion/physical-optimizer/src/enforce_sorting/mod.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/mod.rs @@ -60,6 +60,7 @@ use datafusion_physical_expr::{Distribution, Partitioning}; use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; use datafusion_physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; +use datafusion_physical_plan::repartition::on_demand_repartition::OnDemandRepartitionExec; use datafusion_physical_plan::repartition::RepartitionExec; use datafusion_physical_plan::sorts::partial_sort::PartialSortExec; use datafusion_physical_plan::sorts::sort::SortExec; @@ -621,7 +622,23 @@ fn remove_bottleneck_in_subplan( if can_remove { new_reqs = new_reqs.children.swap_remove(0) } + } else if let Some(repartition) = new_reqs + .plan + .as_any() + .downcast_ref::() + { + let input_partitioning = repartition.input().output_partitioning(); + // We can remove this repartitioning operator if it is now a no-op: + let mut can_remove = input_partitioning.eq(repartition.partitioning()); + // We can also remove it if we ended up with an ineffective RR: + if let Partitioning::OnDemand(n_out) = repartition.partitioning() { + can_remove |= *n_out == input_partitioning.partition_count(); + } + if can_remove { + new_reqs = new_reqs.children.swap_remove(0) + } } + Ok(new_reqs) } diff --git a/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs b/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs index 13d46940c87c4..9f9a4f56d7c2c 100644 --- a/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs @@ -38,6 +38,7 @@ use datafusion_physical_plan::joins::utils::{ }; use datafusion_physical_plan::joins::{HashJoinExec, SortMergeJoinExec}; use datafusion_physical_plan::projection::ProjectionExec; +use datafusion_physical_plan::repartition::on_demand_repartition::OnDemandRepartitionExec; use datafusion_physical_plan::repartition::RepartitionExec; use datafusion_physical_plan::sorts::sort::SortExec; use datafusion_physical_plan::tree_node::PlanContext; @@ -282,6 +283,7 @@ fn pushdown_requirement_to_children( } else if maintains_input_order.is_empty() || !maintains_input_order.iter().any(|o| *o) || plan.as_any().is::() + || plan.as_any().is::() || plan.as_any().is::() // TODO: Add support for Projection push down || plan.as_any().is::() diff --git a/datafusion/physical-optimizer/src/utils.rs b/datafusion/physical-optimizer/src/utils.rs index 636e78a06ce7b..973487af00212 100644 --- a/datafusion/physical-optimizer/src/utils.rs +++ b/datafusion/physical-optimizer/src/utils.rs @@ -21,6 +21,7 @@ use datafusion_physical_expr::LexRequirement; use datafusion_physical_expr_common::sort_expr::LexOrdering; use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; use datafusion_physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; +use datafusion_physical_plan::repartition::on_demand_repartition::OnDemandRepartitionExec; use datafusion_physical_plan::repartition::RepartitionExec; use datafusion_physical_plan::sorts::sort::SortExec; use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; @@ -100,6 +101,11 @@ pub fn is_repartition(plan: &Arc) -> bool { plan.as_any().is::() } +/// Checks whether the given operator is a [`OnDemandRepartitionExec`]. +pub fn is_on_demand_repartition(plan: &Arc) -> bool { + plan.as_any().is::() +} + /// Checks whether the given operator is a limit; /// i.e. either a [`LocalLimitExec`] or a [`GlobalLimitExec`]. pub fn is_limit(plan: &Arc) -> bool { diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 92157819e93aa..c1182adc88aa9 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -56,7 +56,7 @@ use datafusion_physical_expr_common::sort_expr::LexOrdering; use futures::stream::Stream; use futures::{ready, FutureExt, StreamExt, TryStreamExt}; use log::trace; -use on_demand_repartition::OnDemandRepartitionExec; +use on_demand_repartition::{OnDemandRepartitionExec, OnDemandRepartitionMetrics}; use parking_lot::Mutex; mod distributor_channels; @@ -69,22 +69,22 @@ type InputPartitionsToCurrentPartitionReceiver = Vec>, + partition_receivers: Option>>, } impl RepartitionExecStateBuilder { fn new() -> Self { Self { enable_pull_based: false, - partition_receiver: None, + partition_receivers: None, } } fn enable_pull_based(mut self, enable_pull_based: bool) -> Self { self.enable_pull_based = enable_pull_based; self } - fn partition_receiver(mut self, partition_receiver: Receiver) -> Self { - self.partition_receiver = Some(partition_receiver); + fn partition_receivers(mut self, partition_receivers: Vec>) -> Self { + self.partition_receivers = Some(partition_receivers); self } @@ -105,7 +105,7 @@ impl RepartitionExecStateBuilder { name, context, self.enable_pull_based, - self.partition_receiver.clone(), + self.partition_receivers.clone(), ) } } @@ -186,6 +186,7 @@ fn create_partition_channels_hashmap( channels } impl RepartitionExecState { + #[allow(clippy::too_many_arguments)] fn new( input: Arc, partitioning: Partitioning, @@ -194,7 +195,7 @@ impl RepartitionExecState { name: String, context: Arc, enable_pull_based: bool, - partition_receiver: Option>, + partition_receivers: Option>>, ) -> Self { let num_input_partitions = input.output_partitioning().partition_count(); let num_output_partitions = partitioning.partition_count(); @@ -218,19 +219,33 @@ impl RepartitionExecState { }) .collect(); - let r_metrics = RepartitionMetrics::new(i, num_output_partitions, &metrics); - let input_task = if enable_pull_based { + let partition_rx = if preserve_order { + partition_receivers.clone().expect( + "partition_receivers must be provided when preserve_order is enabled", + )[i] + .clone() + } else { + partition_receivers.clone().expect( + "partition_receivers must be provided when preserve_order is disabled", + )[0].clone() + }; + let r_metrics = + OnDemandRepartitionMetrics::new(i, num_output_partitions, &metrics); + SpawnedTask::spawn(OnDemandRepartitionExec::pull_from_input( Arc::clone(&input), i, txs.clone(), partitioning.clone(), - partition_receiver.clone().unwrap(), + partition_rx, r_metrics, Arc::clone(&context), )) } else { + let r_metrics = + RepartitionMetrics::new(i, num_output_partitions, &metrics); + SpawnedTask::spawn(RepartitionExec::pull_from_input( Arc::clone(&input), i, @@ -244,21 +259,13 @@ impl RepartitionExecState { // In a separate task, wait for each input to be done // (and pass along any errors, including panic!s) - let wait_for_task = if enable_pull_based { - SpawnedTask::spawn(OnDemandRepartitionExec::wait_for_task( - input_task, - txs.into_iter() - .map(|(partition, (tx, _reservation))| (partition, tx)) - .collect(), - )) - } else { - SpawnedTask::spawn(RepartitionExec::wait_for_task( - input_task, - txs.into_iter() - .map(|(partition, (tx, _reservation))| (partition, tx)) - .collect(), - )) - }; + let wait_for_task = SpawnedTask::spawn(RepartitionExec::wait_for_task( + input_task, + txs.into_iter() + .map(|(partition, (tx, _reservation))| (partition, tx)) + .collect(), + )); + spawned_tasks.push(wait_for_task); } @@ -607,7 +614,7 @@ pub struct RepartitionExec { } #[derive(Debug, Clone)] -pub(crate) struct RepartitionMetrics { +struct RepartitionMetrics { /// Time in nanos to execute child operator and fetch batches fetch_time: metrics::Time, /// Repartitioning elapsed time in nanos @@ -1195,6 +1202,9 @@ mod tests { use super::*; use crate::test::TestMemoryExec; use crate::{ + collect, + expressions::col, + memory::MemorySourceConfig, test::{ assert_is_pending, exec::{ @@ -1734,17 +1744,15 @@ mod tests { .unwrap() } } - #[cfg(test)] mod test { use arrow::compute::SortOptions; - use arrow::datatypes::{DataType, Field, Schema}; + use arrow::datatypes::Schema; use super::*; use crate::test::TestMemoryExec; use crate::union::UnionExec; - use datafusion_physical_expr::expressions::col; use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; /// Asserts that the plan is as expected @@ -1834,10 +1842,6 @@ mod test { Ok(()) } - fn test_schema() -> Arc { - Arc::new(Schema::new(vec![Field::new("c0", DataType::UInt32, false)])) - } - fn sort_exprs(schema: &Schema) -> LexOrdering { let options = SortOptions::default(); LexOrdering::new(vec![PhysicalSortExpr { diff --git a/datafusion/physical-plan/src/repartition/on_demand_repartition.rs b/datafusion/physical-plan/src/repartition/on_demand_repartition.rs index 10540a93d5acb..b1150829b1c7f 100644 --- a/datafusion/physical-plan/src/repartition/on_demand_repartition.rs +++ b/datafusion/physical-plan/src/repartition/on_demand_repartition.rs @@ -15,9 +15,12 @@ // specific language governing permissions and limitations // under the License. -//! This file implements the [`RepartitionExec`] operator, which maps N input +//! This file implements the [`OnDemandRepartitionExec`] operator, which maps N input //! partitions to M output partitions based on a partitioning scheme, optionally -//! maintaining the order of the input rows in the output. +//! maintaining the order of the input rows in the output. The operator is similar to the [`RepartitionExec`] +//! operator, but it doesn't distribute the data to the output streams until the downstreams request the data. +//! +//! [`RepartitionExec`]: https://docs.rs/datafusion/latest/datafusion/physical_plan/repartition/struct.RepartitionExec.html use std::pin::Pin; use std::sync::Arc; @@ -26,12 +29,13 @@ use std::{any::Any, vec}; use super::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use super::{ - BatchPartitioner, DisplayAs, ExecutionPlanProperties, MaybeBatch, RecordBatchStream, - RepartitionExecBase, RepartitionMetrics, SendableRecordBatchStream, + DisplayAs, ExecutionPlanProperties, MaybeBatch, RecordBatchStream, + RepartitionExecBase, SendableRecordBatchStream, }; use crate::common::SharedMemoryReservation; use crate::execution_plan::CardinalityEffect; -use crate::metrics::BaselineMetrics; +use crate::metrics::{self, BaselineMetrics, MetricBuilder}; +use crate::projection::{all_columns, make_with_child, ProjectionExec}; use crate::repartition::distributor_channels::{ DistributionReceiver, DistributionSender, }; @@ -44,7 +48,7 @@ use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; use async_channel::{Receiver, Sender}; -use datafusion_common::{internal_datafusion_err, DataFusionError, Result}; +use datafusion_common::{internal_datafusion_err, Result}; use datafusion_common_runtime::SpawnedTask; use datafusion_execution::memory_pool::MemoryConsumer; use datafusion_execution::TaskContext; @@ -55,9 +59,51 @@ use futures::{ready, FutureExt, StreamExt, TryStreamExt}; use log::trace; use parking_lot::Mutex; +type PartitionChannels = (Vec>, Vec>); + +/// The OnDemandRepartitionExec operator repartitions the input data based on a push-based model. +/// It is similar to the RepartitionExec operator, but it doesn't distribute the data to the output +/// partitions until the output partitions request the data. +/// +/// When polling, the operator sends the output partition number to the one partition channel, then the prefetch buffer will distribute the data based on the order of the partition number. +/// Each input steams has a prefetch buffer(channel) to distribute the data to the output partitions. +/// +/// The following diagram illustrates the data flow of the OnDemandRepartitionExec operator with 3 output partitions for the input stream 1: +/// ```text +/// /\ /\ /\ +/// ││ ││ ││ +/// ││ ││ ││ +/// ││ ││ ││ +/// ┌───────┴┴────────┐ ┌───────┴┴────────┐ ┌───────┴┴────────┐ +/// │ Stream │ │ Stream │ │ Stream │ +/// │ (1) │ │ (2) │ │ (3) │ +/// └────────┬────────┘ └───────┬─────────┘ └────────┬────────┘ +/// │ │ │ / \ +/// │ │ │ | | +/// │ │ │ | | +/// └────────────────┐ │ ┌──────────────────┘ | | +/// │ │ │ | | +/// ▼ ▼ ▼ | | +/// ┌─────────────────┐ | | +/// Send the partition │ partion channel │ | | +/// number when polling │ │ | | +/// └────────┬────────┘ | | +/// │ | | +/// │ | | +/// │ Get the partition number | | +/// ▼ then send data | | +/// ┌─────────────────┐ | | +/// │ Prefetch Buffer │───────────────────┘ | +/// │ (1) │─────────────────────┘ +/// └─────────────────┘ Distribute data to the output partitions +/// +/// ```text + #[derive(Debug, Clone)] pub struct OnDemandRepartitionExec { base: RepartitionExecBase, + /// Channel to send partition number to the downstream task + partition_channels: Arc>>, } impl OnDemandRepartitionExec { @@ -72,7 +118,7 @@ impl OnDemandRepartitionExec { } /// Get preserve_order flag of the RepartitionExecutor - /// `true` means `SortPreservingRepartitionExec`, `false` means `RepartitionExec` + /// `true` means `SortPreservingRepartitionExec`, `false` means `OnDemandRepartitionExec` pub fn preserve_order(&self) -> bool { self.base.preserve_order } @@ -83,7 +129,7 @@ impl OnDemandRepartitionExec { /// operator can take advantage of it. /// /// If the input is not ordered, or has only one partition, this is a no op, - /// and the node remains a `RepartitionExec`. + /// and the node remains a `OnDemandRepartitionExec`. pub fn with_preserve_order(mut self) -> Self { self.base = self.base.with_preserve_order(); self @@ -157,7 +203,7 @@ impl ExecutionPlan for OnDemandRepartitionExec { } fn benefits_from_input_partitioning(&self) -> Vec { - vec![matches!(self.partitioning(), Partitioning::Hash(_, _))] + vec![false] } fn maintains_input_order(&self) -> Vec { @@ -179,6 +225,7 @@ impl ExecutionPlan for OnDemandRepartitionExec { ); let lazy_state = Arc::clone(&self.base.state); + let partition_channels = Arc::clone(&self.partition_channels); let input = Arc::clone(&self.base.input); let partitioning = self.partitioning().clone(); let metrics = self.base.metrics.clone(); @@ -192,18 +239,34 @@ impl ExecutionPlan for OnDemandRepartitionExec { let stream = futures::stream::once(async move { let num_input_partitions = input.output_partitioning().partition_count(); - let input_captured = Arc::clone(&input); let metrics_captured = metrics.clone(); let name_captured = name.clone(); let context_captured = Arc::clone(&context); - let (partition_tx, partition_rx) = async_channel::unbounded(); + let partition_channels = partition_channels + .get_or_init(|| async move { + let (txs, rxs) = if preserve_order { + (0..num_input_partitions) + .map(|_| async_channel::unbounded()) + .unzip::<_, _, Vec<_>, Vec<_>>() + } else { + let (tx, rx) = async_channel::unbounded(); + (vec![tx], vec![rx]) + }; + Mutex::new((txs, rxs)) + }) + .await; + let (partition_txs, partition_rxs) = { + let channel = partition_channels.lock(); + (channel.0.clone(), channel.1.clone()) + }; + let state = lazy_state .get_or_init(|| async move { Mutex::new( RepartitionExecStateBuilder::new() .enable_pull_based(true) - .partition_receiver(partition_rx.clone()) + .partition_receivers(partition_rxs.clone()) .build( input_captured, partitioning.clone(), @@ -241,14 +304,17 @@ impl ExecutionPlan for OnDemandRepartitionExec { // Store streams from all the input partitions: let input_streams = rx .into_iter() - .map(|receiver| { + .enumerate() + .map(|(i, receiver)| { + // sender should be partition-wise Box::pin(OnDemandPerPartitionStream { schema: Arc::clone(&schema_captured), receiver, _drop_helper: Arc::clone(&abort_helper), reservation: Arc::clone(&reservation), - sender: partition_tx.clone(), + sender: partition_txs[i].clone(), partition, + is_requested: false, }) as SendableRecordBatchStream }) .collect::>(); @@ -277,8 +343,9 @@ impl ExecutionPlan for OnDemandRepartitionExec { input: rx.swap_remove(0), _drop_helper: abort_helper, reservation, - sender: partition_tx.clone(), + sender: partition_txs[0].clone(), partition, + is_requested: false, }) as SendableRecordBatchStream) } }) @@ -298,6 +365,30 @@ impl ExecutionPlan for OnDemandRepartitionExec { fn cardinality_effect(&self) -> CardinalityEffect { CardinalityEffect::Equal } + + fn try_swapping_with_projection( + &self, + projection: &ProjectionExec, + ) -> Result>> { + // If the projection does not narrow the schema, we should not try to push it down. + if projection.expr().len() >= projection.input().schema().fields().len() { + return Ok(None); + } + + // If pushdown is not beneficial or applicable, break it. + if projection.benefits_from_input_partitioning()[0] + || !all_columns(projection.expr()) + { + return Ok(None); + } + + let new_projection = make_with_child(projection, self.input())?; + + Ok(Some(Arc::new(OnDemandRepartitionExec::try_new( + new_projection, + self.partitioning().clone(), + )?))) + } } impl OnDemandRepartitionExec { @@ -322,9 +413,52 @@ impl OnDemandRepartitionExec { preserve_order, cache, }, + partition_channels: Default::default(), }) } + async fn process_input( + input: Arc, + partition: usize, + buffer_tx: Sender, + context: Arc, + fetch_time: metrics::Time, + send_buffer_time: metrics::Time, + ) -> Result<()> { + let timer = fetch_time.timer(); + let mut stream = input.execute(partition, context).map_err(|e| { + internal_datafusion_err!( + "Error executing input partition {} for on demand repartitioning: {}", + partition, + e + ) + })?; + timer.done(); + + loop { + let timer = fetch_time.timer(); + let batch = stream.next().await; + timer.done(); + + // send the batch to the buffer channel + if let Some(batch) = batch { + let timer = send_buffer_time.timer(); + buffer_tx.send(batch?).await.map_err(|e| { + internal_datafusion_err!( + "Error sending batch to buffer channel for partition {}: {}", + partition, + e + ) + })?; + timer.done(); + } else { + break; + } + } + + Ok(()) + } + /// Pulls data from the specified input plan, feeding it to the /// output partitions based on the desired partitioning /// @@ -338,20 +472,30 @@ impl OnDemandRepartitionExec { >, partitioning: Partitioning, output_partition_rx: Receiver, - metrics: RepartitionMetrics, + metrics: OnDemandRepartitionMetrics, context: Arc, ) -> Result<()> { - let mut partitioner = - BatchPartitioner::try_new(partitioning, metrics.repartition_time.clone())?; - - // execute the child operator - let timer = metrics.fetch_time.timer(); - let mut stream = input.execute(partition, context)?; - timer.done(); + // execute the child operator in a separate task + let (buffer_tx, buffer_rx) = async_channel::bounded::(2); + let processing_task = SpawnedTask::spawn(Self::process_input( + Arc::clone(&input), + partition, + buffer_tx, + Arc::clone(&context), + metrics.fetch_time.clone(), + metrics.send_buffer_time.clone(), + )); // While there are still outputs to send to, keep pulling inputs - let mut batches_until_yield = partitioner.num_partitions(); + let mut batches_until_yield = partitioning.partition_count(); while !output_channels.is_empty() { + // When the input is done, break the loop + let batch = match buffer_rx.recv().await { + Ok(batch) => batch, + _ => break, + }; + + // Get the partition number from the output partition let partition = output_partition_rx.recv().await.map_err(|e| { internal_datafusion_err!( "Error receiving partition number from output partition: {}", @@ -359,34 +503,20 @@ impl OnDemandRepartitionExec { ) })?; - // fetch the next batch - let timer = metrics.fetch_time.timer(); - let result = stream.next().await; - timer.done(); - - // Input is done - let batch = match result { - Some(result) => result?, - None => break, - }; - - for res in partitioner.partition_iter(batch)? { - let (_, batch) = res?; - let size = batch.get_array_memory_size(); + let size = batch.get_array_memory_size(); - let timer = metrics.send_time[partition].timer(); - // if there is still a receiver, send to it - if let Some((tx, reservation)) = output_channels.get_mut(&partition) { - reservation.lock().try_grow(size)?; + let timer = metrics.send_time[partition].timer(); + // if there is still a receiver, send to it + if let Some((tx, reservation)) = output_channels.get_mut(&partition) { + reservation.lock().try_grow(size)?; - if tx.send(Some(Ok(batch))).await.is_err() { - // If the other end has hung up, it was an early shutdown (e.g. LIMIT) - reservation.lock().shrink(size); - output_channels.remove(&partition); - } + if tx.send(Some(Ok(batch))).await.is_err() { + // If the other end has hung up, it was an early shutdown (e.g. LIMIT) + reservation.lock().shrink(size); + output_channels.remove(&partition); } - timer.done(); } + timer.done(); // If the input stream is endless, we may spin forever and // never yield back to tokio. See @@ -398,65 +528,61 @@ impl OnDemandRepartitionExec { // // Thus, heuristically yield after producing num_partition // batches - // - // In round robin this is ideal as each input will get a - // new batch. In hash partitioning it may yield too often - // on uneven distributions even if some partition can not - // make progress, but parallelism is going to be limited - // in that case anyways if batches_until_yield == 0 { tokio::task::yield_now().await; - batches_until_yield = partitioner.num_partitions(); + batches_until_yield = partitioning.partition_count(); } else { batches_until_yield -= 1; } } + processing_task.join().await.map_err(|e| { + internal_datafusion_err!("Error waiting for processing task to finish: {}", e) + })??; Ok(()) } +} - /// Waits for `input_task` which is consuming one of the inputs to - /// complete. Upon each successful completion, sends a `None` to - /// each of the output tx channels to signal one of the inputs is - /// complete. Upon error, propagates the errors to all output tx - /// channels. - pub(crate) async fn wait_for_task( - input_task: SpawnedTask>, - txs: HashMap>, - ) { - // wait for completion, and propagate error - // note we ignore errors on send (.ok) as that means the receiver has already shutdown. - - match input_task.join().await { - // Error in joining task - Err(e) => { - let e = Arc::new(e); - - for (_, tx) in txs { - let err = Err(DataFusionError::Context( - "Join Error".to_string(), - Box::new(DataFusionError::External(Box::new(Arc::clone(&e)))), - )); - tx.send(Some(err)).await.ok(); - } - } - // Error from running input task - Ok(Err(e)) => { - let e = Arc::new(e); - - for (_, tx) in txs { - // wrap it because need to send error to all output partitions - let err = Err(DataFusionError::External(Box::new(Arc::clone(&e)))); - tx.send(Some(err)).await.ok(); - } - } - // Input task completed successfully - Ok(Ok(())) => { - // notify each output partition that this input partition has no more data - for (_, tx) in txs { - tx.send(None).await.ok(); - } - } +#[derive(Debug, Clone)] +pub(crate) struct OnDemandRepartitionMetrics { + /// Time in nanos to execute child operator and fetch batches + fetch_time: metrics::Time, + /// Time in nanos for sending resulting batches to buffer channels. + send_buffer_time: metrics::Time, + /// Time in nanos for sending resulting batches to channels. + /// + /// One metric per output partition. + send_time: Vec, +} + +impl OnDemandRepartitionMetrics { + pub fn new( + input_partition: usize, + num_output_partitions: usize, + metrics: &ExecutionPlanMetricsSet, + ) -> Self { + // Time in nanos to execute child operator and fetch batches + let fetch_time = + MetricBuilder::new(metrics).subset_time("fetch_time", input_partition); + + // Time in nanos for sending resulting batches to channels + let send_time = (0..num_output_partitions) + .map(|output_partition| { + let label = + metrics::Label::new("outputPartition", output_partition.to_string()); + MetricBuilder::new(metrics) + .with_label(label) + .subset_time("send_time", input_partition) + }) + .collect(); + + // Time in nanos for sending resulting batches to buffer channels + let send_buffer_time = + MetricBuilder::new(metrics).subset_time("send_buffer_time", input_partition); + Self { + fetch_time, + send_time, + send_buffer_time, } } } @@ -481,6 +607,9 @@ struct OnDemandPerPartitionStream { /// Partition number partition: usize, + + /// Sender State + is_requested: bool, } impl Stream for OnDemandPerPartitionStream { @@ -490,30 +619,30 @@ impl Stream for OnDemandPerPartitionStream { mut self: Pin<&mut Self>, cx: &mut Context<'_>, ) -> Poll> { - if !self.sender.is_closed() { - self.sender.send_blocking(self.partition).map_err(|e| { + if !self.is_requested && !self.sender.is_closed() { + self.sender.try_send(self.partition).map_err(|e| { internal_datafusion_err!( - "Error sending partition number to input partitions: {}", + "Error sending partition number to the receiver for partition {}: {}", + self.partition, e ) })?; + self.is_requested = true; } - match ready!(self.receiver.recv().poll_unpin(cx)) { - Some(Some(v)) => { - if let Ok(batch) = &v { + let result = ready!(self.receiver.recv().poll_unpin(cx)); + self.is_requested = false; + + match result { + Some(Some(batch_result)) => { + if let Ok(batch) = &batch_result { self.reservation .lock() .shrink(batch.get_array_memory_size()); } - - Poll::Ready(Some(v)) - } - Some(None) => { - // Input partition has finished sending batches - Poll::Ready(None) + Poll::Ready(Some(batch_result)) } - None => Poll::Ready(None), + _ => Poll::Ready(None), } } } @@ -549,6 +678,9 @@ struct OnDemandRepartitionStream { /// Partition number partition: usize, + + /// Sender state + is_requested: bool, } impl Stream for OnDemandRepartitionStream { @@ -560,23 +692,27 @@ impl Stream for OnDemandRepartitionStream { ) -> Poll> { loop { // Send partition number to input partitions - if !self.sender.is_closed() { - self.sender.send_blocking(self.partition).map_err(|e| { + if !self.is_requested && !self.sender.is_closed() { + self.sender.try_send(self.partition).map_err(|e| { internal_datafusion_err!( - "Error sending partition number to input partitions: {}", + "Error sending partition number to the receiver for partition {}: {}", + self.partition, e ) })?; + self.is_requested = true; } - match ready!(self.input.recv().poll_unpin(cx)) { + let result = ready!(self.input.recv().poll_unpin(cx)); + self.is_requested = false; + + match result { Some(Some(v)) => { if let Ok(batch) = &v { self.reservation .lock() .shrink(batch.get_array_memory_size()); } - return Poll::Ready(Some(v)); } Some(None) => { @@ -611,6 +747,9 @@ mod tests { use super::*; use crate::{ + collect, + memory::MemorySourceConfig, + source::DataSourceExec, test::{ assert_is_pending, exec::{ @@ -618,16 +757,42 @@ mod tests { ErrorExec, MockExec, }, }, - {collect, memory::MemoryExec}, }; use arrow::array::{ArrayRef, StringArray, UInt32Array}; use arrow::datatypes::{DataType, Field, Schema}; use datafusion_common::cast::as_string_array; - use datafusion_common::{arrow_datafusion_err, assert_batches_sorted_eq, exec_err}; - use datafusion_execution::runtime_env::RuntimeEnvBuilder; + use datafusion_common::{assert_batches_sorted_eq, exec_err}; use tokio::task::JoinSet; + use arrow_schema::SortOptions; + + use crate::coalesce_partitions::CoalescePartitionsExec; + use crate::union::UnionExec; + use datafusion_physical_expr::expressions::col; + use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; + + /// Asserts that the plan is as expected + /// + /// `$EXPECTED_PLAN_LINES`: input plan + /// `$PLAN`: the plan to optimized + /// + macro_rules! assert_plan { + ($EXPECTED_PLAN_LINES: expr, $PLAN: expr) => { + let physical_plan = $PLAN; + let formatted = crate::displayable(&physical_plan).indent(true).to_string(); + let actual: Vec<&str> = formatted.trim().lines().collect(); + + let expected_plan_lines: Vec<&str> = $EXPECTED_PLAN_LINES + .iter().map(|s| *s).collect(); + + assert_eq!( + expected_plan_lines, actual, + "\n**Original Plan Mismatch\n\nexpected:\n\n{expected_plan_lines:#?}\nactual:\n\n{actual:#?}\n\n" + ); + }; + } + fn test_schema() -> Arc { Arc::new(Schema::new(vec![Field::new("c0", DataType::UInt32, false)])) } @@ -639,8 +804,12 @@ mod tests { ) -> Result>> { let task_ctx = Arc::new(TaskContext::default()); // create physical plan - let exec = MemoryExec::try_new(&input_partitions, Arc::clone(schema), None)?; - let exec = OnDemandRepartitionExec::try_new(Arc::new(exec), partitioning)?; + let exec = MemorySourceConfig::try_new_exec( + &input_partitions, + Arc::clone(schema), + None, + )?; + let exec = OnDemandRepartitionExec::try_new(exec, partitioning)?; // execute and collect results let mut output_partitions = vec![]; @@ -695,35 +864,73 @@ mod tests { } #[tokio::test] - async fn unsupported_partitioning() { + async fn many_to_many_on_demand_with_coalesce() -> Result<()> { + let schema = test_schema(); + let partition: Vec = create_vec_batches(2); + let partitions = vec![partition.clone(), partition.clone()]; + let input = + MemorySourceConfig::try_new_exec(&partitions, Arc::clone(&schema), None)?; + let exec = + OnDemandRepartitionExec::try_new(input, Partitioning::OnDemand(3)).unwrap(); + + let coalesce_exec = + CoalescePartitionsExec::new(Arc::new(exec) as Arc); + + let expected_plan = [ + "CoalescePartitionsExec", + " OnDemandRepartitionExec: partitioning=OnDemand(3), input_partitions=2", + " MemoryExec: partitions=2, partition_sizes=[2, 2]", + ]; + assert_plan!(expected_plan, coalesce_exec.clone()); + + // execute the plan let task_ctx = Arc::new(TaskContext::default()); - // have to send at least one batch through to provoke error - let batch = RecordBatch::try_from_iter(vec![( - "my_awesome_field", - Arc::new(StringArray::from(vec!["foo", "bar"])) as ArrayRef, - )]) - .unwrap(); + let stream = coalesce_exec.execute(0, task_ctx)?; + let batches = crate::common::collect(stream).await?; - let schema = batch.schema(); - let input = MockExec::new(vec![Ok(batch)], schema); - // This generates an error (partitioning type not supported) - // but only after the plan is executed. The error should be - // returned and no results produced - let partitioning = Partitioning::UnknownPartitioning(1); - let exec = - OnDemandRepartitionExec::try_new(Arc::new(input), partitioning).unwrap(); - let output_stream = exec.execute(0, task_ctx).unwrap(); + #[rustfmt::skip] + let expected = vec![ + "+----+", + "| c0 |", + "+----+", + "| 1 |", + "| 1 |", + "| 1 |", + "| 1 |", + "| 2 |", + "| 2 |", + "| 2 |", + "| 2 |", + "| 3 |", + "| 3 |", + "| 3 |", + "| 3 |", + "| 4 |", + "| 4 |", + "| 4 |", + "| 4 |", + "| 5 |", + "| 5 |", + "| 5 |", + "| 5 |", + "| 6 |", + "| 6 |", + "| 6 |", + "| 6 |", + "| 7 |", + "| 7 |", + "| 7 |", + "| 7 |", + "| 8 |", + "| 8 |", + "| 8 |", + "| 8 |", + "+----+", + ]; - // Expect that an error is returned - let result_string = crate::common::collect(output_stream) - .await - .unwrap_err() - .to_string(); - assert!( - result_string - .contains("Unsupported repartitioning scheme UnknownPartitioning(1)"), - "actual: {result_string}" - ); + assert_batches_sorted_eq!(&expected, &batches); + + Ok(()) } #[tokio::test] @@ -733,7 +940,7 @@ mod tests { let task_ctx = Arc::new(TaskContext::default()); let input = ErrorExec::new(); - let partitioning = Partitioning::RoundRobinBatch(1); + let partitioning = Partitioning::OnDemand(1); let exec = OnDemandRepartitionExec::try_new(Arc::new(input), partitioning).unwrap(); @@ -767,7 +974,7 @@ mod tests { let schema = batch.schema(); let input = MockExec::new(vec![Ok(batch), err], schema); - let partitioning = Partitioning::RoundRobinBatch(1); + let partitioning = Partitioning::OnDemand(1); let exec = OnDemandRepartitionExec::try_new(Arc::new(input), partitioning).unwrap(); @@ -806,7 +1013,7 @@ mod tests { let schema = batch1.schema(); let expected_batches = vec![batch1.clone(), batch2.clone()]; let input = MockExec::new(vec![Ok(batch1), Ok(batch2)], schema); - let partitioning = Partitioning::RoundRobinBatch(1); + let partitioning = Partitioning::OnDemand(1); let exec = OnDemandRepartitionExec::try_new(Arc::new(input), partitioning).unwrap(); @@ -973,41 +1180,6 @@ mod tests { Ok(()) } - #[tokio::test] - async fn oom() -> Result<()> { - // define input partitions - let schema = test_schema(); - let partition = create_vec_batches(50); - let input_partitions = vec![partition]; - let partitioning = Partitioning::RoundRobinBatch(4); - - // setup up context - let runtime = RuntimeEnvBuilder::default() - .with_memory_limit(1, 1.0) - .build_arc()?; - - let task_ctx = TaskContext::default().with_runtime(runtime); - let task_ctx = Arc::new(task_ctx); - - // create physical plan - let exec = MemoryExec::try_new(&input_partitions, Arc::clone(&schema), None)?; - let exec = OnDemandRepartitionExec::try_new(Arc::new(exec), partitioning)?; - - // pull partitions - for i in 0..exec.partitioning().partition_count() { - let mut stream = exec.execute(i, Arc::clone(&task_ctx))?; - let err = - arrow_datafusion_err!(stream.next().await.unwrap().unwrap_err().into()); - let err = err.find_root(); - assert!( - matches!(err, DataFusionError::ResourcesExhausted(_)), - "Wrong error type: {err}", - ); - } - - Ok(()) - } - /// Create vector batches fn create_vec_batches(n: usize) -> Vec { let batch = create_batch(); @@ -1023,39 +1195,6 @@ mod tests { ) .unwrap() } -} - -#[cfg(test)] -mod test { - use arrow_schema::{DataType, Field, Schema, SortOptions}; - - use crate::memory::MemoryExec; - use crate::union::UnionExec; - use datafusion_physical_expr::expressions::col; - use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; - - use super::*; - - /// Asserts that the plan is as expected - /// - /// `$EXPECTED_PLAN_LINES`: input plan - /// `$PLAN`: the plan to optimized - /// - macro_rules! assert_plan { - ($EXPECTED_PLAN_LINES: expr, $PLAN: expr) => { - let physical_plan = $PLAN; - let formatted = crate::displayable(&physical_plan).indent(true).to_string(); - let actual: Vec<&str> = formatted.trim().lines().collect(); - - let expected_plan_lines: Vec<&str> = $EXPECTED_PLAN_LINES - .iter().map(|s| *s).collect(); - - assert_eq!( - expected_plan_lines, actual, - "\n**Original Plan Mismatch\n\nexpected:\n\n{expected_plan_lines:#?}\nactual:\n\n{actual:#?}\n\n" - ); - }; - } #[tokio::test] async fn test_preserve_order() -> Result<()> { @@ -1065,16 +1204,14 @@ mod test { let source2 = sorted_memory_exec(&schema, sort_exprs); // output has multiple partitions, and is sorted let union = UnionExec::new(vec![source1, source2]); - let exec = OnDemandRepartitionExec::try_new( - Arc::new(union), - Partitioning::RoundRobinBatch(10), - ) - .unwrap() - .with_preserve_order(); + let exec = + OnDemandRepartitionExec::try_new(Arc::new(union), Partitioning::OnDemand(10)) + .unwrap() + .with_preserve_order(); // Repartition should preserve order let expected_plan = [ - "OnDemandRepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2, preserve_order=true, sort_exprs=c0@0 ASC", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=2, preserve_order=true, sort_exprs=c0@0 ASC", " UnionExec", " MemoryExec: partitions=1, partition_sizes=[0], output_ordering=c0@0 ASC", " MemoryExec: partitions=1, partition_sizes=[0], output_ordering=c0@0 ASC", @@ -1083,20 +1220,93 @@ mod test { Ok(()) } + #[tokio::test] + async fn test_preserve_order_with_coalesce() -> Result<()> { + let schema = Arc::new(Schema::new(vec![Field::new( + "my_awesome_field", + DataType::UInt32, + false, + )])); + let options = SortOptions::default(); + let sort_exprs = LexOrdering::new(vec![PhysicalSortExpr { + expr: col("my_awesome_field", &schema).unwrap(), + options, + }]); + + let batch = RecordBatch::try_from_iter(vec![( + "my_awesome_field", + Arc::new(UInt32Array::from(vec![1, 2, 3, 4])) as ArrayRef, + )])?; + + let source = Arc::new(DataSourceExec::new(Arc::new( + MemorySourceConfig::try_new( + &[vec![batch.clone()]], + Arc::clone(&schema), + None, + ) + .unwrap() + .try_with_sort_information(vec![sort_exprs]) + .unwrap(), + ))); + + // output has multiple partitions, and is sorted + let union = UnionExec::new(vec![source.clone(), source]); + let repartition_exec = + OnDemandRepartitionExec::try_new(Arc::new(union), Partitioning::OnDemand(5)) + .unwrap() + .with_preserve_order(); + + let coalesce_exec = CoalescePartitionsExec::new( + Arc::new(repartition_exec) as Arc + ); + + // Repartition should preserve order + let expected_plan = [ + "CoalescePartitionsExec", + " OnDemandRepartitionExec: partitioning=OnDemand(5), input_partitions=2, preserve_order=true, sort_exprs=my_awesome_field@0 ASC", + " UnionExec", + " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=my_awesome_field@0 ASC", + " DataSourceExec: partitions=1, partition_sizes=[1], output_ordering=my_awesome_field@0 ASC", + + ]; + assert_plan!(expected_plan, coalesce_exec.clone()); + + let task_ctx = Arc::new(TaskContext::default()); + let stream = coalesce_exec.execute(0, task_ctx)?; + let expected_batches = crate::common::collect(stream).await?; + + let expected = vec![ + "+------------------+", + "| my_awesome_field |", + "+------------------+", + "| 1 |", + "| 1 |", + "| 2 |", + "| 2 |", + "| 3 |", + "| 3 |", + "| 4 |", + "| 4 |", + "+------------------+", + ]; + + assert_batches_sorted_eq!(&expected, &expected_batches); + Ok(()) + } + #[tokio::test] async fn test_preserve_order_one_partition() -> Result<()> { let schema = test_schema(); let sort_exprs = sort_exprs(&schema); let source = sorted_memory_exec(&schema, sort_exprs); // output is sorted, but has only a single partition, so no need to sort - let exec = - OnDemandRepartitionExec::try_new(source, Partitioning::RoundRobinBatch(10)) - .unwrap() - .with_preserve_order(); + let exec = OnDemandRepartitionExec::try_new(source, Partitioning::OnDemand(10)) + .unwrap() + .with_preserve_order(); // Repartition should not preserve order let expected_plan = [ - "OnDemandRepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", " MemoryExec: partitions=1, partition_sizes=[0], output_ordering=c0@0 ASC", ]; assert_plan!(expected_plan, exec); @@ -1110,16 +1320,14 @@ mod test { let source2 = memory_exec(&schema); // output has multiple partitions, but is not sorted let union = UnionExec::new(vec![source1, source2]); - let exec = OnDemandRepartitionExec::try_new( - Arc::new(union), - Partitioning::RoundRobinBatch(10), - ) - .unwrap() - .with_preserve_order(); + let exec = + OnDemandRepartitionExec::try_new(Arc::new(union), Partitioning::OnDemand(10)) + .unwrap() + .with_preserve_order(); // Repartition should not preserve order, as there is no order to preserve let expected_plan = [ - "OnDemandRepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=2", " UnionExec", " MemoryExec: partitions=1, partition_sizes=[0]", " MemoryExec: partitions=1, partition_sizes=[0]", @@ -1128,10 +1336,6 @@ mod test { Ok(()) } - fn test_schema() -> Arc { - Arc::new(Schema::new(vec![Field::new("c0", DataType::UInt32, false)])) - } - fn sort_exprs(schema: &Schema) -> LexOrdering { let options = SortOptions::default(); LexOrdering::new(vec![PhysicalSortExpr { @@ -1141,18 +1345,18 @@ mod test { } fn memory_exec(schema: &SchemaRef) -> Arc { - Arc::new(MemoryExec::try_new(&[vec![]], Arc::clone(schema), None).unwrap()) + MemorySourceConfig::try_new_exec(&[vec![]], Arc::clone(schema), None).unwrap() } fn sorted_memory_exec( schema: &SchemaRef, sort_exprs: LexOrdering, ) -> Arc { - Arc::new( - MemoryExec::try_new(&[vec![]], Arc::clone(schema), None) + Arc::new(DataSourceExec::new(Arc::new( + MemorySourceConfig::try_new(&[vec![]], Arc::clone(schema), None) .unwrap() .try_with_sort_information(vec![sort_exprs]) .unwrap(), - ) + ))) } } diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index 4555f59e4c945..454a068551754 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -98,7 +98,6 @@ pub struct SortPreservingMergeExec { /// /// See [`Self::with_round_robin_repartition`] for more information. enable_round_robin_repartition: bool, - enable_pull_based_execution: bool, } impl SortPreservingMergeExec { @@ -112,7 +111,6 @@ impl SortPreservingMergeExec { fetch: None, cache, enable_round_robin_repartition: true, - enable_pull_based_execution: true, } } @@ -218,7 +216,6 @@ impl ExecutionPlan for SortPreservingMergeExec { fetch: limit, cache: self.cache.clone(), enable_round_robin_repartition: true, - enable_pull_based_execution: true, })) } @@ -300,34 +297,18 @@ impl ExecutionPlan for SortPreservingMergeExec { } }, _ => { - let streams = if self.enable_pull_based_execution { - // Direct stream connection without channels - let streams = (0..input_partitions) - .map(|partition| { - self.input.execute(partition, Arc::clone(&context)) - }) - .collect::>()?; - - debug!( - "Setting up direct streams for SortPreservingMergeExec::execute" - ); - streams - } else { - // Channel based stream connection - let receivers = (0..input_partitions) - .map(|partition| { - let stream = - self.input.execute(partition, Arc::clone(&context))?; - Ok(spawn_buffered(stream, 1)) - }) - .collect::>()?; - - debug!("Done setting up sender-receiver for SortPreservingMergeExec::execute"); - receivers - }; + let receivers = (0..input_partitions) + .map(|partition| { + let stream = + self.input.execute(partition, Arc::clone(&context))?; + Ok(spawn_buffered(stream, 1)) + }) + .collect::>()?; + + debug!("Done setting up sender-receiver for SortPreservingMergeExec::execute"); let result = StreamingMergeBuilder::new() - .with_streams(streams) + .with_streams(receivers) .with_schema(schema) .with_expressions(self.expr.as_ref()) .with_metrics(BaselineMetrics::new(&self.metrics, partition)) diff --git a/datafusion/physical-plan/src/sorts/streaming_merge.rs b/datafusion/physical-plan/src/sorts/streaming_merge.rs index 090ec498d71d3..a541f79dc7174 100644 --- a/datafusion/physical-plan/src/sorts/streaming_merge.rs +++ b/datafusion/physical-plan/src/sorts/streaming_merge.rs @@ -38,7 +38,8 @@ 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, $enable_round_robin_tie_breaker:ident) => {{ - FieldCursorStream::<$t>::new($sort, $streams, $reservation.new_empty()); + let streams = + FieldCursorStream::<$t>::new($sort, $streams, $reservation.new_empty()); return Ok(Box::pin(SortPreservingMergeStream::new( Box::new(streams), $schema, diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index 1cdfe6d216e32..ca32e48b555e7 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -1233,6 +1233,7 @@ message Partitioning { uint64 round_robin = 1; PhysicalHashRepartition hash = 2; uint64 unknown = 3; + uint64 on_demand = 4; } } diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index 6e09e9a797ea0..c164d2aa9b976 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -13251,6 +13251,11 @@ impl serde::Serialize for Partitioning { #[allow(clippy::needless_borrows_for_generic_args)] struct_ser.serialize_field("unknown", ToString::to_string(&v).as_str())?; } + partitioning::PartitionMethod::OnDemand(v) => { + #[allow(clippy::needless_borrow)] + #[allow(clippy::needless_borrows_for_generic_args)] + struct_ser.serialize_field("onDemand", ToString::to_string(&v).as_str())?; + } } } struct_ser.end() @@ -13267,6 +13272,8 @@ impl<'de> serde::Deserialize<'de> for Partitioning { "roundRobin", "hash", "unknown", + "on_demand", + "onDemand", ]; #[allow(clippy::enum_variant_names)] @@ -13274,6 +13281,7 @@ impl<'de> serde::Deserialize<'de> for Partitioning { RoundRobin, Hash, Unknown, + OnDemand, } impl<'de> serde::Deserialize<'de> for GeneratedField { fn deserialize(deserializer: D) -> std::result::Result @@ -13298,6 +13306,7 @@ impl<'de> serde::Deserialize<'de> for Partitioning { "roundRobin" | "round_robin" => Ok(GeneratedField::RoundRobin), "hash" => Ok(GeneratedField::Hash), "unknown" => Ok(GeneratedField::Unknown), + "onDemand" | "on_demand" => Ok(GeneratedField::OnDemand), _ => Err(serde::de::Error::unknown_field(value, FIELDS)), } } @@ -13339,6 +13348,12 @@ impl<'de> serde::Deserialize<'de> for Partitioning { } partition_method__ = map_.next_value::<::std::option::Option<::pbjson::private::NumberDeserialize<_>>>()?.map(|x| partitioning::PartitionMethod::Unknown(x.0)); } + GeneratedField::OnDemand => { + if partition_method__.is_some() { + return Err(serde::de::Error::duplicate_field("onDemand")); + } + partition_method__ = map_.next_value::<::std::option::Option<::pbjson::private::NumberDeserialize<_>>>()?.map(|x| partitioning::PartitionMethod::OnDemand(x.0)); + } } } Ok(Partitioning { diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index f5ec45da48f2a..00ff83ce3caa1 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -1832,7 +1832,7 @@ pub struct RepartitionExecNode { } #[derive(Clone, PartialEq, ::prost::Message)] pub struct Partitioning { - #[prost(oneof = "partitioning::PartitionMethod", tags = "1, 2, 3")] + #[prost(oneof = "partitioning::PartitionMethod", tags = "1, 2, 3, 4")] pub partition_method: ::core::option::Option, } /// Nested message and enum types in `Partitioning`. @@ -1845,6 +1845,8 @@ pub mod partitioning { Hash(super::PhysicalHashRepartition), #[prost(uint64, tag = "3")] Unknown(u64), + #[prost(uint64, tag = "4")] + OnDemand(u64), } } #[derive(Clone, PartialEq, ::prost::Message)] diff --git a/datafusion/proto/src/logical_plan/mod.rs b/datafusion/proto/src/logical_plan/mod.rs index 641dfe7b5fb84..a0217cc95612f 100644 --- a/datafusion/proto/src/logical_plan/mod.rs +++ b/datafusion/proto/src/logical_plan/mod.rs @@ -1417,6 +1417,7 @@ impl AsLogicalPlan for LogicalPlanNode { Partitioning::RoundRobinBatch(partition_count) => { PartitionMethod::RoundRobin(*partition_count as u64) } + Partitioning::DistributeBy(_) => { return not_impl_err!("DistributeBy") } diff --git a/datafusion/proto/src/physical_plan/from_proto.rs b/datafusion/proto/src/physical_plan/from_proto.rs index 34fb5bb6ddc19..f1f3caed80929 100644 --- a/datafusion/proto/src/physical_plan/from_proto.rs +++ b/datafusion/proto/src/physical_plan/from_proto.rs @@ -451,6 +451,9 @@ pub fn parse_protobuf_partitioning( )) => Ok(Some(Partitioning::RoundRobinBatch( *partition_count as usize, ))), + Some(protobuf::partitioning::PartitionMethod::OnDemand(partition_count)) => { + Ok(Some(Partitioning::OnDemand(*partition_count as usize))) + } Some(protobuf::partitioning::PartitionMethod::Hash(hash_repartition)) => { parse_protobuf_hash_partitioning( Some(hash_repartition), diff --git a/datafusion/proto/src/physical_plan/to_proto.rs b/datafusion/proto/src/physical_plan/to_proto.rs index 3f67842fe625c..8d7a8e92d72b8 100644 --- a/datafusion/proto/src/physical_plan/to_proto.rs +++ b/datafusion/proto/src/physical_plan/to_proto.rs @@ -398,6 +398,11 @@ pub fn serialize_partitioning( *partition_count as u64, )), }, + Partitioning::OnDemand(partition_count) => protobuf::Partitioning { + partition_method: Some(protobuf::partitioning::PartitionMethod::OnDemand( + *partition_count as u64, + )), + }, Partitioning::Hash(exprs, partition_count) => { let serialized_exprs = serialize_physical_exprs(exprs, codec)?; protobuf::Partitioning { diff --git a/datafusion/sqllogictest/test_files/information_schema.slt b/datafusion/sqllogictest/test_files/information_schema.slt index 5a1caad46732c..5c86137915168 100644 --- a/datafusion/sqllogictest/test_files/information_schema.slt +++ b/datafusion/sqllogictest/test_files/information_schema.slt @@ -249,6 +249,7 @@ datafusion.optimizer.max_passes 3 datafusion.optimizer.prefer_existing_sort false datafusion.optimizer.prefer_existing_union false datafusion.optimizer.prefer_hash_join true +datafusion.optimizer.prefer_round_robin_repartition true datafusion.optimizer.repartition_aggregations true datafusion.optimizer.repartition_file_min_size 10485760 datafusion.optimizer.repartition_file_scans true @@ -344,6 +345,7 @@ datafusion.optimizer.max_passes 3 Number of times that the optimizer will attemp datafusion.optimizer.prefer_existing_sort false When true, DataFusion will opportunistically remove sorts when the data is already sorted, (i.e. setting `preserve_order` to true on `RepartitionExec` and using `SortPreservingMergeExec`) When false, DataFusion will maximize plan parallelism using `RepartitionExec` even if this requires subsequently resorting data using a `SortExec`. datafusion.optimizer.prefer_existing_union false When set to true, the optimizer will not attempt to convert Union to Interleave datafusion.optimizer.prefer_hash_join true When set to true, the physical plan optimizer will prefer HashJoin over SortMergeJoin. HashJoin can work more efficiently than SortMergeJoin but consumes more memory +datafusion.optimizer.prefer_round_robin_repartition true When set to false, the physical plan optimizer will replace the round robin repartitioning with on demand repartitioning datafusion.optimizer.repartition_aggregations true Should DataFusion repartition data using the aggregate keys to execute aggregates in parallel using the provided `target_partitions` level datafusion.optimizer.repartition_file_min_size 10485760 Minimum total files size in bytes to perform file scan repartitioning. datafusion.optimizer.repartition_file_scans true When set to `true`, file groups will be repartitioned to achieve maximum parallelism. Currently Parquet and CSV formats are supported. If set to `true`, all files will be repartitioned evenly (i.e., a single large file might be partitioned into smaller chunks) for parallel scanning. If set to `false`, different files will be read in parallel, but repartitioning won't happen within a single file. diff --git a/datafusion/sqllogictest/test_files/join.slt.part b/datafusion/sqllogictest/test_files/join.slt.part index 21126a7479673..18806d431727e 100644 --- a/datafusion/sqllogictest/test_files/join.slt.part +++ b/datafusion/sqllogictest/test_files/join.slt.part @@ -671,7 +671,7 @@ query TT explain select * from t1 inner join t2 on true; ---- logical_plan -01)Cross Join: +01)Cross Join: 02)--TableScan: t1 projection=[t1_id, t1_name, t1_int] 03)--TableScan: t2 projection=[t2_id, t2_name, t2_int] physical_plan diff --git a/datafusion/sqllogictest/test_files/on_demand_repartition.slt b/datafusion/sqllogictest/test_files/on_demand_repartition.slt new file mode 100644 index 0000000000000..5e7235c4012d2 --- /dev/null +++ b/datafusion/sqllogictest/test_files/on_demand_repartition.slt @@ -0,0 +1,445 @@ +# 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. + +########## +# Tests for On-Demand Repartitioning +########## + +# Set 4 partitions for deterministic output plans +statement ok +set datafusion.execution.target_partitions = 4; + +# enable round robin repartitioning +statement ok +set datafusion.optimizer.enable_round_robin_repartition = true; + +########## +# Read from parquet source with on-demand repartitioning +########## + +statement ok +COPY (VALUES (1, 2), (2, 5), (3, 2), (4, 5), (5, 0)) TO 'test_files/scratch/repartition/parquet_table/2.parquet' +STORED AS PARQUET; + +statement ok +CREATE EXTERNAL TABLE parquet_table(column1 int, column2 int) +STORED AS PARQUET +LOCATION 'test_files/scratch/repartition/parquet_table/'; + +# Enable on-demand repartitioning +statement ok +set datafusion.optimizer.prefer_round_robin_repartition = false; + +query TT +EXPLAIN SELECT column1, SUM(column2) FROM parquet_table GROUP BY column1; +---- +logical_plan +01)Aggregate: groupBy=[[parquet_table.column1]], aggr=[[sum(CAST(parquet_table.column2 AS Int64))]] +02)--TableScan: parquet_table projection=[column1, column2] +physical_plan +01)AggregateExec: mode=FinalPartitioned, gby=[column1@0 as column1], aggr=[sum(parquet_table.column2)] +02)--CoalesceBatchesExec: target_batch_size=8192 +03)----RepartitionExec: partitioning=Hash([column1@0], 4), input_partitions=4 +04)------AggregateExec: mode=Partial, gby=[column1@0 as column1], aggr=[sum(parquet_table.column2)] +05)--------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 +06)----------ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition/parquet_table/2.parquet]]}, projection=[column1, column2] + +# Disable on-demand repartitioning +statement ok +set datafusion.optimizer.prefer_round_robin_repartition = true; + +query TT +EXPLAIN SELECT column1, SUM(column2) FROM parquet_table GROUP BY column1; +---- +logical_plan +01)Aggregate: groupBy=[[parquet_table.column1]], aggr=[[sum(CAST(parquet_table.column2 AS Int64))]] +02)--TableScan: parquet_table projection=[column1, column2] +physical_plan +01)AggregateExec: mode=FinalPartitioned, gby=[column1@0 as column1], aggr=[sum(parquet_table.column2)] +02)--CoalesceBatchesExec: target_batch_size=8192 +03)----RepartitionExec: partitioning=Hash([column1@0], 4), input_partitions=4 +04)------AggregateExec: mode=Partial, gby=[column1@0 as column1], aggr=[sum(parquet_table.column2)] +05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +06)----------ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition/parquet_table/2.parquet]]}, projection=[column1, column2] + +# Cleanup +statement ok +DROP TABLE parquet_table; + +########## +# Read from CSV source with on-demand repartitioning +########## + +# Test CSV source with on-demand repartitioning +statement ok +set datafusion.optimizer.enable_round_robin_repartition = true; + +statement ok +set datafusion.optimizer.prefer_round_robin_repartition = false; + + +# create_external_table_with_quote_escape +statement ok +CREATE EXTERNAL TABLE csv_with_quote ( +column1 VARCHAR, +column2 VARCHAR +) STORED AS CSV +LOCATION '../core/tests/data/quote.csv' +OPTIONS ('format.quote' '~', + 'format.delimiter' ',', + 'format.has_header' 'true'); + +statement ok +CREATE EXTERNAL TABLE csv_with_escape ( +column1 VARCHAR, +column2 VARCHAR +) STORED AS CSV +OPTIONS ('format.escape' '\', + 'format.delimiter' ',', + 'format.has_header' 'true') +LOCATION '../core/tests/data/escape.csv'; + +query TT +EXPLAIN SELECT column1 FROM csv_with_quote GROUP BY column1; +---- +logical_plan +01)Aggregate: groupBy=[[csv_with_quote.column1]], aggr=[[]] +02)--TableScan: csv_with_quote projection=[column1] +physical_plan +01)AggregateExec: mode=FinalPartitioned, gby=[column1@0 as column1], aggr=[] +02)--CoalesceBatchesExec: target_batch_size=8192 +03)----RepartitionExec: partitioning=Hash([column1@0], 4), input_partitions=4 +04)------AggregateExec: mode=Partial, gby=[column1@0 as column1], aggr=[] +05)--------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 +06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/quote.csv]]}, projection=[column1], has_header=true + +query TT +EXPLAIN SELECT column1 FROM csv_with_escape GROUP BY column1; +---- +logical_plan +01)Aggregate: groupBy=[[csv_with_escape.column1]], aggr=[[]] +02)--TableScan: csv_with_escape projection=[column1] +physical_plan +01)AggregateExec: mode=FinalPartitioned, gby=[column1@0 as column1], aggr=[] +02)--CoalesceBatchesExec: target_batch_size=8192 +03)----RepartitionExec: partitioning=Hash([column1@0], 4), input_partitions=4 +04)------AggregateExec: mode=Partial, gby=[column1@0 as column1], aggr=[] +05)--------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 +06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/escape.csv]]}, projection=[column1], has_header=true + +statement ok +DROP TABLE csv_with_quote; + +statement ok +DROP TABLE csv_with_escape; + +########## +# Read from arrow source with on-demand repartitioning +########## + +statement ok +CREATE EXTERNAL TABLE arrow_simple( +column1 INT, +column2 VARCHAR, +column3 BOOLEAN +) STORED AS ARROW +LOCATION '../core/tests/data/example.arrow'; + +query TT +EXPLAIN SELECT column1 FROM arrow_simple GROUP BY column1; +---- +logical_plan +01)Aggregate: groupBy=[[arrow_simple.column1]], aggr=[[]] +02)--TableScan: arrow_simple projection=[column1] +physical_plan +01)AggregateExec: mode=FinalPartitioned, gby=[column1@0 as column1], aggr=[] +02)--CoalesceBatchesExec: target_batch_size=8192 +03)----RepartitionExec: partitioning=Hash([column1@0], 4), input_partitions=4 +04)------AggregateExec: mode=Partial, gby=[column1@0 as column1], aggr=[] +05)--------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 +06)----------ArrowExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.arrow]]}, projection=[column1] + +statement ok +DROP TABLE arrow_simple; + +########## +# Read from steaming table source with on-demand repartitioning +########## + +# Unbounded repartition +# See https://github.com/apache/datafusion/issues/5278 +# Set up unbounded table and run a query - the query plan should display a `RepartitionExec` +# and a `CoalescePartitionsExec` +statement ok +CREATE UNBOUNDED EXTERNAL TABLE sink_table ( + c1 VARCHAR NOT NULL, + c2 TINYINT NOT NULL, + c3 SMALLINT NOT NULL, + c4 SMALLINT NOT NULL, + c5 INTEGER NOT NULL, + c6 BIGINT NOT NULL, + c7 SMALLINT NOT NULL, + c8 INT NOT NULL, + c9 INT UNSIGNED NOT NULL, + c10 BIGINT UNSIGNED NOT NULL, + c11 FLOAT NOT NULL, + c12 DOUBLE NOT NULL, + c13 VARCHAR NOT NULL + ) +STORED AS CSV +LOCATION '../../testing/data/csv/aggregate_test_100.csv' +OPTIONS ('format.has_header' 'true'); + +query TII +SELECT c1, c2, c3 FROM sink_table WHERE c3 > 0 LIMIT 5; +---- +c 2 1 +b 1 29 +e 3 104 +a 3 13 +d 1 38 + +statement ok +set datafusion.execution.target_partitions = 3; + +statement ok +set datafusion.optimizer.enable_round_robin_repartition = true; + +statement ok +set datafusion.optimizer.prefer_round_robin_repartition = false; + +query TT +EXPLAIN SELECT c1, c2, c3 FROM sink_table WHERE c3 > 0 LIMIT 5; +---- +logical_plan +01)Limit: skip=0, fetch=5 +02)--Filter: sink_table.c3 > Int16(0) +03)----TableScan: sink_table projection=[c1, c2, c3] +physical_plan +01)CoalescePartitionsExec: fetch=5 +02)--CoalesceBatchesExec: target_batch_size=8192, fetch=5 +03)----FilterExec: c3@2 > 0 +04)------OnDemandRepartitionExec: partitioning=OnDemand(3), input_partitions=1 +05)--------StreamingTableExec: partition_sizes=1, projection=[c1, c2, c3], infinite_source=true + +# Start repratition on empty column test. +# See https://github.com/apache/datafusion/issues/12057 + +statement ok +CREATE TABLE t1(v1 int); + +statement ok +INSERT INTO t1 values(42); + +query I +SELECT sum(1) OVER (PARTITION BY false=false) +FROM t1 WHERE ((false > (v1 = v1)) IS DISTINCT FROM true); +---- +1 + +statement ok +DROP TABLE t1; + +# End repartition on empty columns test + +########## +# Read from memory table source with on-demand repartitioning +########## + +statement ok +CREATE TABLE memory_table AS SELECT * FROM (VALUES (1, 2), (2, 5), (3, 2), (4, 5), (5, 0)) AS t(column1, column2); + +query TT +EXPLAIN SELECT column1 FROM memory_table GROUP BY column1; +---- +logical_plan +01)Aggregate: groupBy=[[memory_table.column1]], aggr=[[]] +02)--TableScan: memory_table projection=[column1] +physical_plan +01)AggregateExec: mode=FinalPartitioned, gby=[column1@0 as column1], aggr=[] +02)--CoalesceBatchesExec: target_batch_size=8192 +03)----RepartitionExec: partitioning=Hash([column1@0], 3), input_partitions=3 +04)------OnDemandRepartitionExec: partitioning=OnDemand(3), input_partitions=1 +05)--------AggregateExec: mode=Partial, gby=[column1@0 as column1], aggr=[] +06)----------MemoryExec: partitions=1, partition_sizes=[1] + +statement ok +DROP TABLE memory_table; + + +########## +# Tests for Join with On-Demand Repartitioning +########## + +statement ok +set datafusion.execution.target_partitions = 1; + +statement ok +set datafusion.execution.batch_size = 2; + +statement ok +CREATE TABLE join_t1(t1_id INT UNSIGNED, t1_name VARCHAR, t1_int INT UNSIGNED) +AS VALUES +(11, 'a', 1), +(22, 'b', 2), +(33, 'c', 3), +(44, 'd', 4); + +statement ok +CREATE TABLE join_t2(t2_id INT UNSIGNED, t2_name VARCHAR, t2_int INT UNSIGNED) +AS VALUES +(11, 'z', 3), +(22, 'y', 1), +(44, 'x', 3), +(55, 'w', 3); + +statement ok +set datafusion.optimizer.repartition_joins = true; + +statement ok +set datafusion.execution.target_partitions = 2; + +# left join +query TT +EXPLAIN +select join_t1.t1_id, join_t2.t2_id +from join_t1 +left join join_t2 on join_t1.t1_id = join_t2.t2_id; +---- +logical_plan +01)Left Join: join_t1.t1_id = join_t2.t2_id +02)--TableScan: join_t1 projection=[t1_id] +03)--TableScan: join_t2 projection=[t2_id] +physical_plan +01)CoalesceBatchesExec: target_batch_size=2 +02)--HashJoinExec: mode=Partitioned, join_type=Left, on=[(t1_id@0, t2_id@0)] +03)----CoalesceBatchesExec: target_batch_size=2 +04)------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 +05)--------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 +06)----------MemoryExec: partitions=1, partition_sizes=[1] +07)----CoalesceBatchesExec: target_batch_size=2 +08)------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 +09)--------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 +10)----------MemoryExec: partitions=1, partition_sizes=[1] + +# right join +query TT +EXPLAIN +select join_t1.t1_id, join_t2.t2_id +from join_t1 +right join join_t2 on join_t1.t1_id = join_t2.t2_id; +---- +logical_plan +01)Right Join: join_t1.t1_id = join_t2.t2_id +02)--TableScan: join_t1 projection=[t1_id] +03)--TableScan: join_t2 projection=[t2_id] +physical_plan +01)CoalesceBatchesExec: target_batch_size=2 +02)--HashJoinExec: mode=Partitioned, join_type=Right, on=[(t1_id@0, t2_id@0)] +03)----CoalesceBatchesExec: target_batch_size=2 +04)------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 +05)--------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 +06)----------MemoryExec: partitions=1, partition_sizes=[1] +07)----CoalesceBatchesExec: target_batch_size=2 +08)------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 +09)--------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 +10)----------MemoryExec: partitions=1, partition_sizes=[1] + +# inner join +query TT +EXPLAIN +select join_t1.t1_id, join_t2.t2_id +from join_t1 +inner join join_t2 on join_t1.t1_id = join_t2.t2_id; +---- +logical_plan +01)Inner Join: join_t1.t1_id = join_t2.t2_id +02)--TableScan: join_t1 projection=[t1_id] +03)--TableScan: join_t2 projection=[t2_id] +physical_plan +01)CoalesceBatchesExec: target_batch_size=2 +02)--HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t1_id@0, t2_id@0)] +03)----CoalesceBatchesExec: target_batch_size=2 +04)------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 +05)--------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 +06)----------MemoryExec: partitions=1, partition_sizes=[1] +07)----CoalesceBatchesExec: target_batch_size=2 +08)------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 +09)--------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 +10)----------MemoryExec: partitions=1, partition_sizes=[1] + +statement ok +DROP TABLE join_t1; + +statement ok +DROP TABLE join_t2; + + +########## +# Tests for Join with On-Demand Repartitioning +########## +statement ok +CREATE EXTERNAL TABLE aggregate_test_100 ( + c1 VARCHAR NOT NULL, + c2 TINYINT NOT NULL, + c3 SMALLINT NOT NULL, + c4 SMALLINT, + c5 INT, + c6 BIGINT NOT NULL, + c7 SMALLINT NOT NULL, + c8 INT NOT NULL, + c9 BIGINT UNSIGNED NOT NULL, + c10 VARCHAR NOT NULL, + c11 FLOAT NOT NULL, + c12 DOUBLE NOT NULL, + c13 VARCHAR NOT NULL +) +STORED AS CSV +LOCATION '../../testing/data/csv/aggregate_test_100.csv' +OPTIONS ('format.has_header' 'true'); + +statement ok +set datafusion.execution.batch_size = 4096; + +statement ok +set datafusion.optimizer.repartition_windows = true; + +query TT +EXPLAIN SELECT + SUM(c4) OVER(PARTITION BY c1, c2 ORDER BY c2 ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING), + COUNT(*) OVER(PARTITION BY c1 ORDER BY c2 ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING) + FROM aggregate_test_100 +---- +logical_plan +01)Projection: sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING +02)--WindowAggr: windowExpr=[[count(Int64(1)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING]] +03)----Projection: aggregate_test_100.c1, aggregate_test_100.c2, sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING +04)------WindowAggr: windowExpr=[[sum(CAST(aggregate_test_100.c4 AS Int64)) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING]] +05)--------TableScan: aggregate_test_100 projection=[c1, c2, c4] +physical_plan +01)ProjectionExec: expr=[sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@2 as sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@3 as count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING] +02)--BoundedWindowAggExec: wdw=[count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] +03)----SortExec: expr=[c1@0 ASC NULLS LAST, c2@1 ASC NULLS LAST], preserve_partitioning=[true] +04)------CoalesceBatchesExec: target_batch_size=4096 +05)--------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 +06)----------ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@3 as sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING] +07)------------BoundedWindowAggExec: wdw=[sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)), is_causal: false }], mode=[Sorted] +08)--------------SortExec: expr=[c1@0 ASC NULLS LAST, c2@1 ASC NULLS LAST], preserve_partitioning=[true] +09)----------------CoalesceBatchesExec: target_batch_size=4096 +10)------------------RepartitionExec: partitioning=Hash([c1@0, c2@1], 2), input_partitions=2 +11)--------------------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 +12)----------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c4], has_header=true diff --git a/docs/source/user-guide/configs.md b/docs/source/user-guide/configs.md index 999735f4c0592..9ab42ea80c447 100644 --- a/docs/source/user-guide/configs.md +++ b/docs/source/user-guide/configs.md @@ -97,6 +97,7 @@ Environment variables are read during `SessionConfig` initialisation so they mus | datafusion.execution.enforce_batch_size_in_joins | false | Should DataFusion enforce batch size in joins or not. By default, DataFusion will not enforce batch size in joins. Enforcing batch size in joins can reduce memory usage when joining large tables with a highly-selective join filter, but is also slightly slower. | | datafusion.optimizer.enable_distinct_aggregation_soft_limit | true | When set to true, the optimizer will push a limit operation into grouped aggregations which have no aggregate expressions, as a soft limit, emitting groups once the limit is reached, before all rows in the group are read. | | datafusion.optimizer.enable_round_robin_repartition | true | When set to true, the physical plan optimizer will try to add round robin repartitioning to increase parallelism to leverage more CPU cores | +| datafusion.optimizer.prefer_round_robin_repartition | true | When set to false, the physical plan optimizer will replace the round robin repartitioning with on demand repartitioning | | datafusion.optimizer.enable_topk_aggregation | true | When set to true, the optimizer will attempt to perform limit operations during aggregations, if possible | | datafusion.optimizer.filter_null_join_keys | false | When set to true, the optimizer will insert filters before a join between a nullable and non-nullable column to filter out nulls on the nullable side. This filter can add additional overhead when the file format does not fully support predicate push down. | | datafusion.optimizer.repartition_aggregations | true | Should DataFusion repartition data using the aggregate keys to execute aggregates in parallel using the provided `target_partitions` level | From b8016624917374fb88492df17aa11eb16925f2c1 Mon Sep 17 00:00:00 2001 From: Weijun-H Date: Fri, 7 Feb 2025 12:59:37 +0800 Subject: [PATCH 10/25] chore --- .../enforce_distribution.rs | 387 ++++++++---------- .../physical_optimizer/enforce_sorting.rs | 336 +++++++-------- .../src/coalesce_batches.rs | 3 +- 3 files changed, 339 insertions(+), 387 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index 1def872129a02..46daf54c0af02 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -58,6 +58,7 @@ use datafusion_physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; use datafusion_physical_plan::projection::ProjectionExec; use datafusion_physical_plan::sorts::sort::SortExec; use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; +use datafusion_physical_plan::source::DataSourceExec; use datafusion_physical_plan::union::UnionExec; use datafusion_physical_plan::ExecutionPlanProperties; use datafusion_physical_plan::PlanProperties; @@ -161,23 +162,22 @@ impl ExecutionPlan for SortRequiredExec { } } -fn parquet_exec() -> Arc { +fn parquet_exec() -> Arc { parquet_exec_with_sort(vec![]) } -fn parquet_exec_multiple() -> Arc { +fn parquet_exec_multiple() -> Arc { parquet_exec_multiple_sorted(vec![]) } /// Created a sorted parquet exec with multiple files -fn parquet_exec_multiple_sorted(output_ordering: Vec) -> Arc { - ParquetExec::builder( - FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema()) - .with_file_groups(vec![ - vec![PartitionedFile::new("x".to_string(), 100)], - vec![PartitionedFile::new("y".to_string(), 100)], - ]) - .with_output_ordering(output_ordering), +fn parquet_exec_multiple_sorted( + output_ordering: Vec, +) -> Arc { + FileScanConfig::new( + ObjectStoreUrl::parse("test:///").unwrap(), + schema(), + Arc::new(ParquetSource::default()), ) .with_file_groups(vec![ vec![PartitionedFile::new("x".to_string(), 100)], @@ -187,54 +187,31 @@ fn parquet_exec_multiple_sorted(output_ordering: Vec) -> Arc Arc { +fn csv_exec() -> Arc { csv_exec_with_sort(vec![]) } -fn csv_exec_with_sort(output_ordering: Vec) -> Arc { - Arc::new( - CsvExec::builder( - FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema()) - .with_file(PartitionedFile::new("x".to_string(), 100)) - .with_output_ordering(output_ordering), - ) - .with_has_header(false) - .with_delimeter(b',') - .with_quote(b'"') - .with_escape(None) - .with_comment(None) - .with_newlines_in_values(false) - .with_file_compression_type(FileCompressionType::UNCOMPRESSED) - .build(), +fn csv_exec_with_sort(output_ordering: Vec) -> Arc { + FileScanConfig::new( + ObjectStoreUrl::parse("test:///").unwrap(), + schema(), + Arc::new(CsvSource::new(false, b',', b'"')), ) .with_file(PartitionedFile::new("x".to_string(), 100)) .with_output_ordering(output_ordering) .build() } -fn csv_exec_multiple() -> Arc { +fn csv_exec_multiple() -> Arc { csv_exec_multiple_sorted(vec![]) } // Created a sorted parquet exec with multiple files -fn csv_exec_multiple_sorted(output_ordering: Vec) -> Arc { - Arc::new( - CsvExec::builder( - FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema()) - .with_file_groups(vec![ - vec![PartitionedFile::new("x".to_string(), 100)], - vec![PartitionedFile::new("y".to_string(), 100)], - ]) - .with_output_ordering(output_ordering), - ) - .with_has_header(false) - .with_delimeter(b',') - .with_quote(b'"') - .with_escape(None) - .with_comment(None) - .with_newlines_in_values(false) - .with_file_compression_type(FileCompressionType::UNCOMPRESSED) - .build(), +fn csv_exec_multiple_sorted(output_ordering: Vec) -> Arc { + FileScanConfig::new( + ObjectStoreUrl::parse("test:///").unwrap(), + schema(), + Arc::new(CsvSource::new(false, b',', b'"')), ) .with_file_groups(vec![ vec![PartitionedFile::new("x".to_string(), 100)], @@ -591,14 +568,14 @@ fn multi_hash_joins() -> Result<()> { join_plan.as_str(), "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ], // Should include 4 RepartitionExecs _ => vec![ @@ -607,14 +584,14 @@ fn multi_hash_joins() -> Result<()> { join_plan.as_str(), "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ], }; assert_optimized!(expected, top_join.clone(), true); @@ -654,14 +631,14 @@ fn multi_hash_joins() -> Result<()> { join_plan.as_str(), "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ], // Should include 4 RepartitionExecs _ => @@ -671,14 +648,14 @@ fn multi_hash_joins() -> Result<()> { join_plan.as_str(), "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ], }; assert_optimized!(expected, top_join.clone(), true); @@ -730,13 +707,13 @@ fn multi_joins_after_alias() -> Result<()> { "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, b@1)]", "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "RepartitionExec: partitioning=Hash([b@1], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected, top_join.clone(), true); assert_optimized!(expected, top_join, false); @@ -756,13 +733,13 @@ fn multi_joins_after_alias() -> Result<()> { "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, b@1)]", "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "RepartitionExec: partitioning=Hash([b@1], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected, top_join.clone(), true); assert_optimized!(expected, top_join, false); @@ -809,13 +786,13 @@ fn multi_joins_after_multi_alias() -> Result<()> { "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, b@1)]", "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "RepartitionExec: partitioning=Hash([b@1], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected, top_join.clone(), true); @@ -851,12 +828,12 @@ fn join_after_agg_alias() -> Result<()> { "RepartitionExec: partitioning=Hash([a1@0], 10), input_partitions=10", "AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[]", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "AggregateExec: mode=FinalPartitioned, gby=[a2@0 as a2], aggr=[]", "RepartitionExec: partitioning=Hash([a2@0], 10), input_partitions=10", "AggregateExec: mode=Partial, gby=[a@0 as a2], aggr=[]", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected, join.clone(), true); assert_optimized!(expected, join, false); @@ -904,12 +881,12 @@ fn hash_join_key_ordering() -> Result<()> { "RepartitionExec: partitioning=Hash([b1@0, a1@1], 10), input_partitions=10", "AggregateExec: mode=Partial, gby=[b@1 as b1, a@0 as a1], aggr=[]", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "AggregateExec: mode=FinalPartitioned, gby=[b@0 as b, a@1 as a], aggr=[]", "RepartitionExec: partitioning=Hash([b@0, a@1], 10), input_partitions=10", "AggregateExec: mode=Partial, gby=[b@1 as b, a@0 as a], aggr=[]", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected, join.clone(), true); assert_optimized!(expected, join, false); @@ -1023,19 +1000,19 @@ fn multi_hash_join_key_ordering() -> Result<()> { "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(b@1, b1@1), (c@2, c1@2), (a@0, a1@0)]", "RepartitionExec: partitioning=Hash([b@1, c@2, a@0], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "RepartitionExec: partitioning=Hash([b1@1, c1@2, a1@0], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(b@1, b1@1), (c@2, c1@2), (a@0, a1@0)]", "RepartitionExec: partitioning=Hash([b@1, c@2, a@0], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "RepartitionExec: partitioning=Hash([b1@1, c1@2, a1@0], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected, filter_top_join.clone(), true); assert_optimized!(expected, filter_top_join, false); @@ -1161,19 +1138,19 @@ fn reorder_join_keys_to_left_input() -> Result<()> { "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a1@0), (b@1, b1@1), (c@2, c1@2)]", "RepartitionExec: partitioning=Hash([a@0, b@1, c@2], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "RepartitionExec: partitioning=Hash([a1@0, b1@1, c1@2], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@2, c1@2), (b@1, b1@1), (a@0, a1@0)]", "RepartitionExec: partitioning=Hash([c@2, b@1, a@0], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "RepartitionExec: partitioning=Hash([c1@2, b1@1, a1@0], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_plan_txt!(expected, reordered); @@ -1295,19 +1272,19 @@ fn reorder_join_keys_to_right_input() -> Result<()> { "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a1@0), (b@1, b1@1)]", "RepartitionExec: partitioning=Hash([a@0, b@1], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "RepartitionExec: partitioning=Hash([a1@0, b1@1], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@2, c1@2), (b@1, b1@1), (a@0, a1@0)]", "RepartitionExec: partitioning=Hash([c@2, b@1, a@0], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "RepartitionExec: partitioning=Hash([c1@2, b1@1, a1@0], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_plan_txt!(expected, reordered); @@ -1368,16 +1345,16 @@ fn multi_smj_joins() -> Result<()> { "SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "SortExec: expr=[b1@1 ASC], preserve_partitioning=[true]", "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "SortExec: expr=[c@2 ASC], preserve_partitioning=[true]", "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ], // Should include 7 RepartitionExecs (4 hash, 3 round-robin), 4 SortExecs // Since ordering of the left child is not preserved after SortMergeJoin @@ -1397,16 +1374,16 @@ fn multi_smj_joins() -> Result<()> { "SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "SortExec: expr=[b1@1 ASC], preserve_partitioning=[true]", "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "SortExec: expr=[c@2 ASC], preserve_partitioning=[true]", "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ], }; assert_optimized!(expected, top_join.clone(), true, true); @@ -1420,16 +1397,16 @@ fn multi_smj_joins() -> Result<()> { "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10, preserve_order=true, sort_exprs=b1@1 ASC", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[b1@1 ASC], preserve_partitioning=[false]", "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10, preserve_order=true, sort_exprs=c@2 ASC", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ], // Should include 8 RepartitionExecs (4 hash, 8 round-robin), 4 SortExecs // Since ordering of the left child is not preserved after SortMergeJoin @@ -1451,16 +1428,16 @@ fn multi_smj_joins() -> Result<()> { "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10, preserve_order=true, sort_exprs=b1@1 ASC", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[b1@1 ASC], preserve_partitioning=[false]", "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10, preserve_order=true, sort_exprs=c@2 ASC", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ], }; assert_optimized!(expected_first_sort_enforcement, top_join, false, true); @@ -1486,16 +1463,16 @@ fn multi_smj_joins() -> Result<()> { "SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "SortExec: expr=[b1@1 ASC], preserve_partitioning=[true]", "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "SortExec: expr=[c@2 ASC], preserve_partitioning=[true]", "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ], // Should include 7 RepartitionExecs (4 hash, 3 round-robin) and 4 SortExecs JoinType::Left | JoinType::Full => vec![ @@ -1506,16 +1483,16 @@ fn multi_smj_joins() -> Result<()> { "SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "SortExec: expr=[b1@1 ASC], preserve_partitioning=[true]", "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "SortExec: expr=[c@2 ASC], preserve_partitioning=[true]", "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ], // this match arm cannot be reached _ => unreachable!() @@ -1530,16 +1507,16 @@ fn multi_smj_joins() -> Result<()> { "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10, preserve_order=true, sort_exprs=b1@1 ASC", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[b1@1 ASC], preserve_partitioning=[false]", "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10, preserve_order=true, sort_exprs=c@2 ASC", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ], // Should include 8 RepartitionExecs (4 of them preserves order) and 4 SortExecs JoinType::Left | JoinType::Full => vec![ @@ -1552,16 +1529,16 @@ fn multi_smj_joins() -> Result<()> { "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10, preserve_order=true, sort_exprs=b1@1 ASC", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[b1@1 ASC], preserve_partitioning=[false]", "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10, preserve_order=true, sort_exprs=c@2 ASC", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ], // this match arm cannot be reached _ => unreachable!() @@ -1631,14 +1608,14 @@ fn smj_join_key_ordering() -> Result<()> { "RepartitionExec: partitioning=Hash([b1@0, a1@1], 10), input_partitions=10", "AggregateExec: mode=Partial, gby=[b@1 as b1, a@0 as a1], aggr=[]", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "SortExec: expr=[b2@1 ASC, a2@0 ASC], preserve_partitioning=[true]", "ProjectionExec: expr=[a@1 as a2, b@0 as b2]", "AggregateExec: mode=FinalPartitioned, gby=[b@0 as b, a@1 as a], aggr=[]", "RepartitionExec: partitioning=Hash([b@0, a@1], 10), input_partitions=10", "AggregateExec: mode=Partial, gby=[b@1 as b, a@0 as a], aggr=[]", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected, join.clone(), true, true); @@ -1654,7 +1631,7 @@ fn smj_join_key_ordering() -> Result<()> { "RepartitionExec: partitioning=Hash([b1@0, a1@1], 10), input_partitions=10", "AggregateExec: mode=Partial, gby=[b@1 as b1, a@0 as a1], aggr=[]", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "RepartitionExec: partitioning=Hash([b2@1, a2@0], 10), input_partitions=10, preserve_order=true, sort_exprs=b2@1 ASC, a2@0 ASC", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[b2@1 ASC, a2@0 ASC], preserve_partitioning=[false]", @@ -1664,7 +1641,7 @@ fn smj_join_key_ordering() -> Result<()> { "RepartitionExec: partitioning=Hash([b@0, a@1], 10), input_partitions=10", "AggregateExec: mode=Partial, gby=[b@1 as b, a@0 as a], aggr=[]", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected_first_sort_enforcement, join, false, true); @@ -1695,7 +1672,7 @@ fn merge_does_not_need_sort() -> Result<()> { let expected = &[ "SortPreservingMergeExec: [a@0 ASC]", "CoalesceBatchesExec: target_batch_size=4096", - "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", + "DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", ]; assert_optimized!(expected, exec, true); @@ -1707,7 +1684,7 @@ fn merge_does_not_need_sort() -> Result<()> { "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", "CoalescePartitionsExec", "CoalesceBatchesExec: target_batch_size=4096", - "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", + "DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", ]; assert_optimized!(expected, exec, false); @@ -1743,12 +1720,12 @@ fn union_to_interleave() -> Result<()> { "RepartitionExec: partitioning=Hash([a1@0], 10), input_partitions=10", "AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[]", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "AggregateExec: mode=FinalPartitioned, gby=[a1@0 as a1], aggr=[]", "RepartitionExec: partitioning=Hash([a1@0], 10), input_partitions=10", "AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[]", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected, plan.clone(), true); assert_optimized!(expected, plan.clone(), false); @@ -1786,12 +1763,12 @@ fn union_not_to_interleave() -> Result<()> { "RepartitionExec: partitioning=Hash([a1@0], 10), input_partitions=10", "AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[]", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", "AggregateExec: mode=FinalPartitioned, gby=[a1@0 as a1], aggr=[]", "RepartitionExec: partitioning=Hash([a1@0], 10), input_partitions=10", "AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[]", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; // no sort in the plan but since we need it as a parameter, make it default false let prefer_existing_sort = false; @@ -1826,7 +1803,7 @@ fn added_repartition_to_single_partition() -> Result<()> { "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected, plan.clone(), true); assert_optimized!(expected, plan, false); @@ -1845,7 +1822,7 @@ fn repartition_deepest_node() -> Result<()> { "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", "FilterExec: c@2 = 0", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected, plan.clone(), true); assert_optimized!(expected, plan, false); @@ -1865,7 +1842,7 @@ fn repartition_unsorted_limit() -> Result<()> { "FilterExec: c@2 = 0", // nothing sorts the data, so the local limit doesn't require sorted data either "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected, plan.clone(), true); @@ -1888,7 +1865,7 @@ fn repartition_sorted_limit() -> Result<()> { "LocalLimitExec: fetch=100", // data is sorted so can't repartition here "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected, plan.clone(), true); assert_optimized!(expected, plan, false); @@ -1915,7 +1892,7 @@ fn repartition_sorted_limit_with_filter() -> Result<()> { // is still satisfied. "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected, plan.clone(), true); @@ -1946,7 +1923,7 @@ fn repartition_ignores_limit() -> Result<()> { "GlobalLimitExec: skip=0, fetch=100", "LocalLimitExec: fetch=100", // Expect no repartition to happen for local limit - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected, plan.clone(), true); assert_optimized!(expected, plan, false); @@ -1960,12 +1937,12 @@ fn repartition_ignores_union() -> Result<()> { let expected = &[ "UnionExec", - // Expect no repartition of ParquetExec - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + // Expect no repartition of DataSourceExec + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected, plan.clone(), true); @@ -1987,7 +1964,7 @@ fn repartition_through_sort_preserving_merge() -> Result<()> { // need resort as the data was not sorted correctly let expected = &[ "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected, plan.clone(), true); assert_optimized!(expected, plan, false); @@ -2012,7 +1989,7 @@ fn repartition_ignores_sort_preserving_merge() -> Result<()> { // should not repartition, since increased parallelism is not beneficial for SortPReservingMerge let expected = &[ "SortPreservingMergeExec: [c@2 ASC]", - "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + "DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", ]; assert_optimized!(expected, plan.clone(), true); @@ -2020,7 +1997,7 @@ fn repartition_ignores_sort_preserving_merge() -> Result<()> { let expected = &[ "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", "CoalescePartitionsExec", - "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + "DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", ]; assert_optimized!(expected, plan, false); @@ -2042,8 +2019,8 @@ fn repartition_ignores_sort_preserving_merge_with_union() -> Result<()> { let expected = &[ "SortPreservingMergeExec: [c@2 ASC]", "UnionExec", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", ]; assert_optimized!(expected, plan.clone(), true); @@ -2052,8 +2029,8 @@ fn repartition_ignores_sort_preserving_merge_with_union() -> Result<()> { "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", "CoalescePartitionsExec", "UnionExec", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", ]; assert_optimized!(expected, plan, false); @@ -2079,7 +2056,7 @@ fn repartition_does_not_destroy_sort() -> Result<()> { "SortRequiredExec: [d@3 ASC]", "FilterExec: c@2 = 0", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[d@3 ASC]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[d@3 ASC], file_type=parquet", ]; assert_optimized!(expected, plan.clone(), true, true); @@ -2117,11 +2094,11 @@ fn repartition_does_not_destroy_sort_more_complex() -> Result<()> { "UnionExec", // union input 1: no repartitioning "SortRequiredExec: [c@2 ASC]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", // union input 2: should repartition "FilterExec: c@2 = 0", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected, plan.clone(), true); assert_optimized!(expected, plan, false); @@ -2154,7 +2131,7 @@ fn repartition_transitively_with_projection() -> Result<()> { // Since this projection is not trivial, increasing parallelism is beneficial "ProjectionExec: expr=[a@0 + b@1 as sum]", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected, plan.clone(), true); @@ -2165,7 +2142,7 @@ fn repartition_transitively_with_projection() -> Result<()> { // Since this projection is not trivial, increasing parallelism is beneficial "ProjectionExec: expr=[a@0 + b@1 as sum]", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected_first_sort_enforcement, plan, false); @@ -2197,7 +2174,7 @@ fn repartition_ignores_transitively_with_projection() -> Result<()> { "SortRequiredExec: [c@2 ASC]", // Since this projection is trivial, increasing parallelism is not beneficial "ProjectionExec: expr=[a@0 as a, b@1 as b, c@2 as c]", - "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + "DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", ]; assert_optimized!(expected, plan.clone(), true); assert_optimized!(expected, plan, false); @@ -2230,7 +2207,7 @@ fn repartition_transitively_past_sort_with_projection() -> Result<()> { "SortExec: expr=[c@2 ASC], preserve_partitioning=[true]", // Since this projection is trivial, increasing parallelism is not beneficial "ProjectionExec: expr=[a@0 as a, b@1 as b, c@2 as c]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected, plan.clone(), true); assert_optimized!(expected, plan, false); @@ -2253,7 +2230,7 @@ fn repartition_transitively_past_sort_with_filter() -> Result<()> { // Expect repartition on the input to the sort (as it can benefit from additional parallelism) "FilterExec: c@2 = 0", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected, plan.clone(), true); @@ -2264,7 +2241,7 @@ fn repartition_transitively_past_sort_with_filter() -> Result<()> { "FilterExec: c@2 = 0", // Expect repartition on the input of the filter (as it can benefit from additional parallelism) "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected_first_sort_enforcement, plan, false); @@ -2300,7 +2277,7 @@ fn repartition_transitively_past_sort_with_projection_and_filter() -> Result<()> "FilterExec: c@2 = 0", // repartition is lowest down "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected, plan.clone(), true); @@ -2311,7 +2288,7 @@ fn repartition_transitively_past_sort_with_projection_and_filter() -> Result<()> "ProjectionExec: expr=[a@0 as a, b@1 as b, c@2 as c]", "FilterExec: c@2 = 0", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected_first_sort_enforcement, plan, false); @@ -2328,13 +2305,13 @@ fn parallelization_single_partition() -> Result<()> { "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", "RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2", "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", - "ParquetExec: file_groups={2 groups: [[x:0..50], [x:50..100]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={2 groups: [[x:0..50], [x:50..100]]}, projection=[a, b, c, d, e], file_type=parquet", ]; let expected_csv = [ "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", "RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2", "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", - "CsvExec: file_groups={2 groups: [[x:0..50], [x:50..100]]}, projection=[a, b, c, d, e], has_header=false", + "DataSourceExec: file_groups={2 groups: [[x:0..50], [x:50..100]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", ]; assert_optimized!(expected_parquet, plan_parquet, true, false, 2, true, 10); assert_optimized!(expected_csv, plan_csv, true, false, 2, true, 10); @@ -2359,7 +2336,7 @@ fn parallelization_multiple_files() -> Result<()> { let expected = [ "SortRequiredExec: [a@0 ASC]", "FilterExec: c@2 = 0", - "ParquetExec: file_groups={3 groups: [[x:0..50], [y:0..100], [x:50..100]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", ]; + "DataSourceExec: file_groups={3 groups: [[x:0..50], [y:0..100], [x:50..100]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", ]; let target_partitions = 3; let repartition_size = 1; assert_optimized!( @@ -2376,7 +2353,7 @@ fn parallelization_multiple_files() -> Result<()> { let expected = [ "SortRequiredExec: [a@0 ASC]", "FilterExec: c@2 = 0", - "ParquetExec: file_groups={8 groups: [[x:0..25], [y:0..25], [x:25..50], [y:25..50], [x:50..75], [y:50..75], [x:75..100], [y:75..100]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", + "DataSourceExec: file_groups={8 groups: [[x:0..25], [y:0..25], [x:25..50], [y:25..50], [x:50..75], [y:50..75], [x:75..100], [y:75..100]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", ]; let target_partitions = 8; let repartition_size = 1; @@ -2395,7 +2372,7 @@ fn parallelization_multiple_files() -> Result<()> { } #[test] -/// CsvExec on compressed csv file will not be partitioned +/// DataSourceExec on compressed csv file will not be partitioned /// (Not able to decompress chunked csv file) fn parallelization_compressed_csv() -> Result<()> { let compression_types = [ @@ -2411,14 +2388,14 @@ fn parallelization_compressed_csv() -> Result<()> { "RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2", "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", "RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", - "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", ]; let expected_partitioned = [ "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", "RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2", "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", - "CsvExec: file_groups={2 groups: [[x:0..50], [x:50..100]]}, projection=[a, b, c, d, e], has_header=false", + "DataSourceExec: file_groups={2 groups: [[x:0..50], [x:50..100]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", ]; for compression_type in compression_types { @@ -2455,14 +2432,14 @@ fn parallelization_two_partitions() -> Result<()> { "RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2", "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", // Plan already has two partitions - "ParquetExec: file_groups={2 groups: [[x:0..100], [y:0..100]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={2 groups: [[x:0..100], [y:0..100]]}, projection=[a, b, c, d, e], file_type=parquet", ]; let expected_csv = [ "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", "RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2", "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", // Plan already has two partitions - "CsvExec: file_groups={2 groups: [[x:0..100], [y:0..100]]}, projection=[a, b, c, d, e], has_header=false", + "DataSourceExec: file_groups={2 groups: [[x:0..100], [y:0..100]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", ]; assert_optimized!(expected_parquet, plan_parquet, true, false, 2, true, 10); assert_optimized!(expected_csv, plan_csv, true, false, 2, true, 10); @@ -2480,14 +2457,14 @@ fn parallelization_two_partitions_into_four() -> Result<()> { "RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4", "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", // Multiple source files splitted across partitions - "ParquetExec: file_groups={4 groups: [[x:0..50], [x:50..100], [y:0..50], [y:50..100]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={4 groups: [[x:0..50], [x:50..100], [y:0..50], [y:50..100]]}, projection=[a, b, c, d, e], file_type=parquet", ]; let expected_csv = [ "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", "RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4", "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", // Multiple source files splitted across partitions - "CsvExec: file_groups={4 groups: [[x:0..50], [x:50..100], [y:0..50], [y:50..100]]}, projection=[a, b, c, d, e], has_header=false", + "DataSourceExec: file_groups={4 groups: [[x:0..50], [x:50..100], [y:0..50], [y:50..100]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", ]; assert_optimized!(expected_parquet, plan_parquet, true, false, 4, true, 10); assert_optimized!(expected_csv, plan_csv, true, false, 4, true, 10); @@ -2511,7 +2488,7 @@ fn parallelization_sorted_limit() -> Result<()> { // data is sorted so can't repartition here "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", // Doesn't parallelize for SortExec without preserve_partitioning - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; let expected_csv = &[ "GlobalLimitExec: skip=0, fetch=100", @@ -2519,7 +2496,7 @@ fn parallelization_sorted_limit() -> Result<()> { // data is sorted so can't repartition here "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", // Doesn't parallelize for SortExec without preserve_partitioning - "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", ]; assert_optimized!(expected_parquet, plan_parquet, true); assert_optimized!(expected_csv, plan_csv, true); @@ -2551,7 +2528,7 @@ fn parallelization_limit_with_filter() -> Result<()> { "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", // SortExec doesn't benefit from input partitioning - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; let expected_csv = &[ "GlobalLimitExec: skip=0, fetch=100", @@ -2563,7 +2540,7 @@ fn parallelization_limit_with_filter() -> Result<()> { "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", // SortExec doesn't benefit from input partitioning - "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", ]; assert_optimized!(expected_parquet, plan_parquet, true); assert_optimized!(expected_csv, plan_csv, true); @@ -2595,7 +2572,7 @@ fn parallelization_ignores_limit() -> Result<()> { "GlobalLimitExec: skip=0, fetch=100", // Limit doesn't benefit from input partitioning - no parallelism "LocalLimitExec: fetch=100", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; let expected_csv = &[ "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", @@ -2611,7 +2588,7 @@ fn parallelization_ignores_limit() -> Result<()> { "GlobalLimitExec: skip=0, fetch=100", // Limit doesn't benefit from input partitioning - no parallelism "LocalLimitExec: fetch=100", - "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", ]; assert_optimized!(expected_parquet, plan_parquet, true); assert_optimized!(expected_csv, plan_csv, true); @@ -2627,20 +2604,20 @@ fn parallelization_union_inputs() -> Result<()> { let expected_parquet = &[ "UnionExec", // Union doesn't benefit from input partitioning - no parallelism - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; let expected_csv = &[ "UnionExec", // Union doesn't benefit from input partitioning - no parallelism - "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", ]; assert_optimized!(expected_parquet, plan_parquet, true); assert_optimized!(expected_csv, plan_csv, true); @@ -2665,10 +2642,10 @@ fn parallelization_prior_to_sort_preserving_merge() -> Result<()> { // parallelization is not beneficial for SortPreservingMerge let expected_parquet = &[ - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", ]; let expected_csv = &[ - "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], has_header=false", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=csv, has_header=false", ]; assert_optimized!(expected_parquet, plan_parquet, true); assert_optimized!(expected_csv, plan_csv, true); @@ -2695,14 +2672,14 @@ fn parallelization_sort_preserving_merge_with_union() -> Result<()> { let expected_parquet = &[ "SortPreservingMergeExec: [c@2 ASC]", "UnionExec", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", ]; let expected_csv = &[ "SortPreservingMergeExec: [c@2 ASC]", "UnionExec", - "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], has_header=false", - "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], has_header=false", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=csv, has_header=false", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=csv, has_header=false", ]; assert_optimized!(expected_parquet, plan_parquet, true); assert_optimized!(expected_csv, plan_csv, true); @@ -2729,11 +2706,11 @@ fn parallelization_does_not_benefit() -> Result<()> { // no parallelization, because SortRequiredExec doesn't benefit from increased parallelism let expected_parquet = &[ "SortRequiredExec: [c@2 ASC]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", ]; let expected_csv = &[ "SortRequiredExec: [c@2 ASC]", - "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], has_header=false", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=csv, has_header=false", ]; assert_optimized!(expected_parquet, plan_parquet, true); assert_optimized!(expected_csv, plan_csv, true); @@ -2766,14 +2743,14 @@ fn parallelization_ignores_transitively_with_projection_parquet() -> Result<()> let expected = &[ "SortPreservingMergeExec: [c2@1 ASC]", " ProjectionExec: expr=[a@0 as a2, c@2 as c2]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", ]; plans_matches_expected!(expected, &plan_parquet); // data should not be repartitioned / resorted let expected_parquet = &[ "ProjectionExec: expr=[a@0 as a2, c@2 as c2]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", ]; assert_optimized!(expected_parquet, plan_parquet, true); @@ -2805,14 +2782,14 @@ fn parallelization_ignores_transitively_with_projection_csv() -> Result<()> { let expected = &[ "SortPreservingMergeExec: [c2@1 ASC]", " ProjectionExec: expr=[a@0 as a2, c@2 as c2]", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], has_header=false", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=csv, has_header=false", ]; plans_matches_expected!(expected, &plan_csv); // data should not be repartitioned / resorted let expected_csv = &[ "ProjectionExec: expr=[a@0 as a2, c@2 as c2]", - "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], has_header=false", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=csv, has_header=false", ]; assert_optimized!(expected_csv, plan_csv, true); @@ -2829,14 +2806,14 @@ fn remove_redundant_roundrobins() -> Result<()> { " FilterExec: c@2 = 0", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; plans_matches_expected!(expected, &physical_plan); let expected = &[ "FilterExec: c@2 = 0", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected, physical_plan.clone(), true); assert_optimized!(expected, physical_plan, false); @@ -2860,7 +2837,7 @@ fn remove_unnecessary_spm_after_filter() -> Result<()> { "CoalescePartitionsExec", "FilterExec: c@2 = 0", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2, preserve_order=true, sort_exprs=c@2 ASC", - "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + "DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", ]; // last flag sets config.optimizer.PREFER_EXISTING_SORT assert_optimized!(expected, physical_plan.clone(), true, true); @@ -2883,7 +2860,7 @@ fn preserve_ordering_through_repartition() -> Result<()> { "SortPreservingMergeExec: [d@3 ASC]", "FilterExec: c@2 = 0", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2, preserve_order=true, sort_exprs=d@3 ASC", - "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[d@3 ASC]", + "DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[d@3 ASC], file_type=parquet", ]; // last flag sets config.optimizer.PREFER_EXISTING_SORT assert_optimized!(expected, physical_plan.clone(), true, true); @@ -2907,7 +2884,7 @@ fn do_not_preserve_ordering_through_repartition() -> Result<()> { "SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", "FilterExec: c@2 = 0", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", - "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", + "DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", ]; assert_optimized!(expected, physical_plan.clone(), true); @@ -2917,7 +2894,7 @@ fn do_not_preserve_ordering_through_repartition() -> Result<()> { "CoalescePartitionsExec", "FilterExec: c@2 = 0", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", - "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", + "DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", ]; assert_optimized!(expected, physical_plan, false); @@ -2940,7 +2917,7 @@ fn no_need_for_sort_after_filter() -> Result<()> { // Since after this stage c is constant. c@2 ASC ordering is already satisfied. "FilterExec: c@2 = 0", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", - "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + "DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", ]; assert_optimized!(expected, physical_plan.clone(), true); assert_optimized!(expected, physical_plan, false); @@ -2968,7 +2945,7 @@ fn do_not_preserve_ordering_through_repartition2() -> Result<()> { "SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", "FilterExec: c@2 = 0", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", - "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + "DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", ]; assert_optimized!(expected, physical_plan.clone(), true); @@ -2979,7 +2956,7 @@ fn do_not_preserve_ordering_through_repartition2() -> Result<()> { "SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", "FilterExec: c@2 = 0", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", - "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + "DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", ]; assert_optimized!(expected, physical_plan, false); @@ -2999,7 +2976,7 @@ fn do_not_preserve_ordering_through_repartition3() -> Result<()> { let expected = &[ "FilterExec: c@2 = 0", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", - "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + "DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", ]; assert_optimized!(expected, physical_plan.clone(), true); assert_optimized!(expected, physical_plan, false); @@ -3021,7 +2998,7 @@ fn do_not_put_sort_when_input_is_invalid() -> Result<()> { // by existing ordering at the source. "SortRequiredExec: [a@0 ASC]", "FilterExec: c@2 = 0", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_plan_txt!(expected, physical_plan); @@ -3031,7 +3008,7 @@ fn do_not_put_sort_when_input_is_invalid() -> Result<()> { // EnforceDistribution rule doesn't satisfy this requirement either. "FilterExec: c@2 = 0", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; let mut config = ConfigOptions::new(); @@ -3059,7 +3036,7 @@ fn put_sort_when_input_is_valid() -> Result<()> { // by existing ordering at the source. "SortRequiredExec: [a@0 ASC]", "FilterExec: c@2 = 0", - "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", + "DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", ]; assert_plan_txt!(expected, physical_plan); @@ -3068,7 +3045,7 @@ fn put_sort_when_input_is_valid() -> Result<()> { // EnforceDistribution rule satisfy this requirement also. "SortRequiredExec: [a@0 ASC]", "FilterExec: c@2 = 0", - "ParquetExec: file_groups={10 groups: [[x:0..20], [y:0..20], [x:20..40], [y:20..40], [x:40..60], [y:40..60], [x:60..80], [y:60..80], [x:80..100], [y:80..100]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", + "DataSourceExec: file_groups={10 groups: [[x:0..20], [y:0..20], [x:20..40], [y:20..40], [x:40..60], [y:40..60], [x:60..80], [y:60..80], [x:80..100], [y:80..100]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet", ]; let mut config = ConfigOptions::new(); @@ -3095,7 +3072,7 @@ fn do_not_add_unnecessary_hash() -> Result<()> { let expected = &[ "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", ]; // Make sure target partition number is 1. In this case hash repartition is unnecessary assert_optimized!(expected, physical_plan.clone(), true, false, 1, false, 1024); @@ -3125,7 +3102,7 @@ fn do_not_add_unnecessary_hash2() -> Result<()> { "RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4", "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", "RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2", - "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + "DataSourceExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], file_type=parquet", ]; // Make sure target partition number is larger than 2 (e.g partition number at the source). assert_optimized!(expected, physical_plan.clone(), true, false, 4, false, 1024); @@ -3140,12 +3117,12 @@ fn optimize_away_unnecessary_repartition() -> Result<()> { let expected = &[ "CoalescePartitionsExec", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; plans_matches_expected!(expected, physical_plan.clone()); let expected = - &["ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]"]; + &["DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet"]; assert_optimized!(expected, physical_plan.clone(), true); assert_optimized!(expected, physical_plan, false); @@ -3163,7 +3140,7 @@ fn optimize_away_unnecessary_repartition2() -> Result<()> { " CoalescePartitionsExec", " FilterExec: c@2 = 0", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; plans_matches_expected!(expected, physical_plan.clone()); @@ -3171,7 +3148,7 @@ fn optimize_away_unnecessary_repartition2() -> Result<()> { "FilterExec: c@2 = 0", "FilterExec: c@2 = 0", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=parquet", ]; assert_optimized!(expected, physical_plan.clone(), true); assert_optimized!(expected, physical_plan, false); diff --git a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs index 64bd72579436d..3412b962d8598 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs @@ -46,10 +46,9 @@ use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeE use datafusion_physical_plan::{get_plan_string, ExecutionPlan}; use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::{TreeNode, TransformedResult}; -use datafusion::datasource::physical_plan::{CsvExec, FileScanConfig, ParquetExec}; +use datafusion::datasource::physical_plan::{CsvSource, FileScanConfig, ParquetSource}; use datafusion_execution::object_store::ObjectStoreUrl; use datafusion::datasource::listing::PartitionedFile; -use datafusion::datasource::file_format::file_compression_type::FileCompressionType; use datafusion_physical_optimizer::enforce_distribution::EnforceDistribution; use datafusion_physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; use datafusion_physical_plan::sorts::sort::SortExec; @@ -63,23 +62,10 @@ fn csv_exec_ordered( ) -> Arc { let sort_exprs = sort_exprs.into_iter().collect(); - Arc::new( - CsvExec::builder( - FileScanConfig::new( - ObjectStoreUrl::parse("test:///").unwrap(), - schema.clone(), - ) - .with_file(PartitionedFile::new("file_path".to_string(), 100)) - .with_output_ordering(vec![sort_exprs]), - ) - .with_has_header(true) - .with_delimeter(0) - .with_quote(b'"') - .with_escape(None) - .with_comment(None) - .with_newlines_in_values(false) - .with_file_compression_type(FileCompressionType::UNCOMPRESSED) - .build(), + FileScanConfig::new( + ObjectStoreUrl::parse("test:///").unwrap(), + schema.clone(), + Arc::new(CsvSource::new(true, 0, b'"')), ) .with_file(PartitionedFile::new("file_path".to_string(), 100)) .with_output_ordering(vec![sort_exprs]) @@ -93,10 +79,11 @@ pub fn parquet_exec_sorted( ) -> Arc { let sort_exprs = sort_exprs.into_iter().collect(); - ParquetExec::builder( - FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema.clone()) - .with_file(PartitionedFile::new("x".to_string(), 100)) - .with_output_ordering(vec![sort_exprs]), + let source = Arc::new(ParquetSource::default()); + FileScanConfig::new( + ObjectStoreUrl::parse("test:///").unwrap(), + schema.clone(), + source, ) .with_file(PartitionedFile::new("x".to_string(), 100)) .with_output_ordering(vec![sort_exprs]) @@ -110,23 +97,10 @@ fn csv_exec_sorted( ) -> Arc { let sort_exprs = sort_exprs.into_iter().collect(); - Arc::new( - CsvExec::builder( - FileScanConfig::new( - ObjectStoreUrl::parse("test:///").unwrap(), - schema.clone(), - ) - .with_file(PartitionedFile::new("x".to_string(), 100)) - .with_output_ordering(vec![sort_exprs]), - ) - .with_has_header(false) - .with_delimeter(0) - .with_quote(0) - .with_escape(None) - .with_comment(None) - .with_newlines_in_values(false) - .with_file_compression_type(FileCompressionType::UNCOMPRESSED) - .build(), + FileScanConfig::new( + ObjectStoreUrl::parse("test:///").unwrap(), + schema.clone(), + Arc::new(CsvSource::new(false, 0, 0)), ) .with_file(PartitionedFile::new("x".to_string(), 100)) .with_output_ordering(vec![sort_exprs]) @@ -237,12 +211,12 @@ async fn test_remove_unnecessary_sort5() -> Result<()> { let expected_input = ["SortExec: expr=[a@2 ASC], preserve_partitioning=[false]", " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(col_a@0, c@2)]", - " MemoryExec: partitions=1, partition_sizes=[0]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]"]; + " DataSourceExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet"]; let expected_optimized = ["HashJoinExec: mode=Partitioned, join_type=Inner, on=[(col_a@0, c@2)]", - " MemoryExec: partitions=1, partition_sizes=[0]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]"]; + " DataSourceExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=parquet"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -281,11 +255,11 @@ async fn test_bounded_window_set_monotonic_no_partition() -> Result<()> { "SortExec: expr=[count@2 ASC NULLS LAST], preserve_partitioning=[false]", " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", " SortExec: expr=[nullable_col@0 DESC NULLS LAST], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", ]; let expected_optimized = [ "BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -332,13 +306,13 @@ async fn test_bounded_plain_window_set_monotonic_with_partitions() -> Result<()> "SortExec: expr=[count@2 ASC NULLS LAST], preserve_partitioning=[false]", " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", " SortExec: expr=[nullable_col@0 DESC NULLS LAST], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", ]; let expected_optimized = [ "SortExec: expr=[count@2 ASC NULLS LAST], preserve_partitioning=[false]", " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", " SortExec: expr=[nullable_col@0 ASC NULLS LAST], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -395,12 +369,12 @@ async fn test_bounded_plain_window_set_monotonic_with_partitions_partial() -> Re "SortExec: expr=[nullable_col@0 DESC NULLS LAST, count@2 ASC NULLS LAST], preserve_partitioning=[false]", " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", " SortExec: expr=[nullable_col@0 DESC NULLS LAST], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", ]; let expected_optimized = [ "BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", " SortExec: expr=[nullable_col@0 DESC NULLS LAST, count@2 ASC NULLS LAST], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -438,12 +412,12 @@ async fn test_bounded_window_non_set_monotonic_sort() -> Result<()> { "SortExec: expr=[avg@5 ASC NULLS LAST], preserve_partitioning=[false]", " BoundedWindowAggExec: wdw=[avg: Ok(Field { name: \"avg\", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", " SortExec: expr=[a@0 DESC NULLS LAST], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 DESC NULLS LAST]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 DESC NULLS LAST], file_type=parquet", ]; let expected_optimized = [ "SortExec: expr=[avg@5 ASC NULLS LAST], preserve_partitioning=[false]", " BoundedWindowAggExec: wdw=[avg: Ok(Field { name: \"avg\", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 DESC NULLS LAST]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 DESC NULLS LAST], file_type=parquet", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -472,22 +446,22 @@ async fn test_do_not_remove_sort_with_limit() -> Result<()> { let expected_input = ["SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", " UnionExec", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC], file_type=parquet", " GlobalLimitExec: skip=0, fetch=100", " LocalLimitExec: fetch=100", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; // We should keep the bottom `SortExec`. let expected_optimized = ["SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[true]", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", " UnionExec", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC], file_type=parquet", " GlobalLimitExec: skip=0, fetch=100", " LocalLimitExec: fetch=100", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -510,9 +484,9 @@ async fn test_union_inputs_sorted() -> Result<()> { let expected_input = vec![ "SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC], file_type=parquet", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", ]; // should not add a sort at the output of the union, input plan should not be changed let expected_optimized = expected_input.clone(); @@ -542,9 +516,9 @@ async fn test_union_inputs_different_sorted() -> Result<()> { let expected_input = vec![ "SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC, non_nullable_col@1 ASC]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC, non_nullable_col@1 ASC], file_type=parquet", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", ]; // should not add a sort at the output of the union, input plan should not be changed let expected_optimized = expected_input.clone(); @@ -571,20 +545,20 @@ async fn test_union_inputs_different_sorted2() -> Result<()> { let physical_plan = sort_preserving_merge_exec(sort_exprs, union); // Input is an invalid plan. In this case rule should add required sorting in appropriate places. - // First ParquetExec has output ordering(nullable_col@0 ASC). However, it doesn't satisfy the + // First DataSourceExec has output ordering(nullable_col@0 ASC). However, it doesn't satisfy the // required ordering of SortPreservingMergeExec. let expected_input = ["SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", " UnionExec", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC], file_type=parquet", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; let expected_optimized = ["SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", " UnionExec", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC], file_type=parquet", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -615,18 +589,18 @@ async fn test_union_inputs_different_sorted3() -> Result<()> { let expected_input = ["SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC], file_type=parquet", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; // should adjust sorting in the first input of the union such that it is not unnecessarily fine let expected_optimized = ["SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC], file_type=parquet", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -657,18 +631,18 @@ async fn test_union_inputs_different_sorted4() -> Result<()> { let expected_input = ["SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", " UnionExec", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC], file_type=parquet", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; let expected_optimized = ["SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", " UnionExec", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC], file_type=parquet", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -707,15 +681,15 @@ async fn test_union_inputs_different_sorted5() -> Result<()> { let expected_input = ["SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 DESC NULLS LAST], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; let expected_optimized = ["SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -750,21 +724,21 @@ async fn test_union_inputs_different_sorted6() -> Result<()> { let expected_input = ["SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC], file_type=parquet", " SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; // Should adjust the requirement in the third input of the union so // that it is not unnecessarily fine. let expected_optimized = ["SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC], file_type=parquet", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[true]", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -790,16 +764,16 @@ async fn test_union_inputs_different_sorted7() -> Result<()> { let expected_input = ["SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; // Union preserves the inputs ordering and we should not change any of the SortExecs under UnionExec let expected_output = ["SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; assert_optimized!(expected_input, expected_output, physical_plan, true); Ok(()) @@ -841,14 +815,14 @@ async fn test_union_inputs_different_sorted8() -> Result<()> { // example below. let expected_input = ["UnionExec", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", " SortExec: expr=[nullable_col@0 DESC NULLS LAST, non_nullable_col@1 DESC NULLS LAST], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; // Since `UnionExec` doesn't preserve ordering in the plan above. // We shouldn't keep SortExecs in the plan. let expected_optimized = ["UnionExec", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -890,15 +864,15 @@ async fn test_window_multi_path_sort() -> Result<()> { " SortPreservingMergeExec: [nullable_col@0 DESC NULLS LAST]", " UnionExec", " SortExec: expr=[nullable_col@0 DESC NULLS LAST], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC, non_nullable_col@1 ASC]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC, non_nullable_col@1 ASC], file_type=parquet", " SortExec: expr=[nullable_col@0 DESC NULLS LAST], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC], file_type=parquet"]; let expected_optimized = [ "WindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(NULL), is_causal: false }]", " SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC, non_nullable_col@1 ASC]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC, non_nullable_col@1 ASC], file_type=parquet", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC], file_type=parquet"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -928,14 +902,14 @@ async fn test_window_multi_path_sort2() -> Result<()> { " SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", " UnionExec", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC], file_type=parquet", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC], file_type=parquet"]; let expected_optimized = ["BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", " SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC], file_type=parquet", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC], file_type=parquet"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -975,19 +949,19 @@ async fn test_union_inputs_different_sorted_with_limit() -> Result<()> { let expected_input = ["SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", " GlobalLimitExec: skip=0, fetch=100", " LocalLimitExec: fetch=100", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 DESC NULLS LAST], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; let expected_optimized = ["SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", " GlobalLimitExec: skip=0, fetch=100", " LocalLimitExec: fetch=100", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 DESC NULLS LAST], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -1032,8 +1006,8 @@ async fn test_sort_merge_join_order_by_left() -> Result<()> { ); let expected_input = ["SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", join_plan2.as_str(), - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b], file_type=parquet"]; let expected_optimized = match join_type { JoinType::Inner | JoinType::Left @@ -1043,9 +1017,9 @@ async fn test_sort_merge_join_order_by_left() -> Result<()> { vec![ join_plan.as_str(), " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", " SortExec: expr=[col_a@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b], file_type=parquet", ] } _ => { @@ -1054,9 +1028,9 @@ async fn test_sort_merge_join_order_by_left() -> Result<()> { "SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", join_plan2.as_str(), " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", " SortExec: expr=[col_a@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b], file_type=parquet", ] } }; @@ -1107,17 +1081,17 @@ async fn test_sort_merge_join_order_by_right() -> Result<()> { ); let expected_input = [spm_plan, join_plan2.as_str(), - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b], file_type=parquet"]; let expected_optimized = match join_type { JoinType::Inner | JoinType::Right | JoinType::RightAnti => { // can push down the sort requirements and save 1 SortExec vec![ join_plan.as_str(), " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", " SortExec: expr=[col_a@0 ASC, col_b@1 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b], file_type=parquet", ] } _ => { @@ -1126,9 +1100,9 @@ async fn test_sort_merge_join_order_by_right() -> Result<()> { "SortExec: expr=[col_a@2 ASC, col_b@3 ASC], preserve_partitioning=[false]", join_plan2.as_str(), " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", " SortExec: expr=[col_a@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b], file_type=parquet", ] } }; @@ -1162,16 +1136,16 @@ async fn test_sort_merge_join_complex_order_by() -> Result<()> { let expected_input = ["SortPreservingMergeExec: [col_b@3 ASC, col_a@2 ASC]", " SortMergeJoin: join_type=Inner, on=[(nullable_col@0, col_a@0)]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b], file_type=parquet"]; // can not push down the sort requirements, need to add SortExec let expected_optimized = ["SortExec: expr=[col_b@3 ASC, col_a@2 ASC], preserve_partitioning=[false]", " SortMergeJoin: join_type=Inner, on=[(nullable_col@0, col_a@0)]", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", " SortExec: expr=[col_a@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b], file_type=parquet"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); // order by (nullable_col, col_b, col_a) @@ -1184,16 +1158,16 @@ async fn test_sort_merge_join_complex_order_by() -> Result<()> { let expected_input = ["SortPreservingMergeExec: [nullable_col@0 ASC, col_b@3 ASC, col_a@2 ASC]", " SortMergeJoin: join_type=Inner, on=[(nullable_col@0, col_a@0)]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b], file_type=parquet"]; // can not push down the sort requirements, need to add SortExec let expected_optimized = ["SortExec: expr=[nullable_col@0 ASC, col_b@3 ASC, col_a@2 ASC], preserve_partitioning=[false]", " SortMergeJoin: join_type=Inner, on=[(nullable_col@0, col_a@0)]", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet", " SortExec: expr=[col_a@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b], file_type=parquet"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -1223,12 +1197,12 @@ async fn test_multilayer_coalesce_partitions() -> Result<()> { " FilterExec: NOT non_nullable_col@1", " CoalescePartitionsExec", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; let expected_optimized = ["SortPreservingMergeExec: [nullable_col@0 ASC]", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[true]", " FilterExec: NOT non_nullable_col@1", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], file_type=parquet"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -1251,12 +1225,12 @@ async fn test_with_lost_ordering_bounded() -> Result<()> { " CoalescePartitionsExec", " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], has_header=false"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=csv, has_header=false"]; let expected_optimized = ["SortPreservingMergeExec: [a@0 ASC]", " SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], has_header=false"]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=csv, has_header=false"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -1296,7 +1270,7 @@ async fn test_with_lost_ordering_unbounded_bounded( " CoalescePartitionsExec", " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], has_header=true", + " DataSourceExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=csv, has_header=true", ]; // Expected unbounded result (same for with and without flag) @@ -1313,14 +1287,14 @@ async fn test_with_lost_ordering_unbounded_bounded( " CoalescePartitionsExec", " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], has_header=true", + " DataSourceExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=csv, has_header=true", ]; let expected_optimized_bounded_parallelize_sort = vec![ "SortPreservingMergeExec: [a@0 ASC]", " SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], has_header=true", + " DataSourceExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], file_type=csv, has_header=true", ]; let (expected_input, expected_optimized, expected_optimized_sort_parallelize) = if source_unbounded { @@ -1364,11 +1338,11 @@ async fn test_do_not_pushdown_through_spm() -> Result<()> { let expected_input = ["SortExec: expr=[b@1 ASC], preserve_partitioning=[false]", " SortPreservingMergeExec: [a@0 ASC, b@1 ASC]", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC, b@1 ASC], has_header=false",]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC, b@1 ASC], file_type=csv, has_header=false",]; let expected_optimized = ["SortExec: expr=[b@1 ASC], preserve_partitioning=[false]", " SortPreservingMergeExec: [a@0 ASC, b@1 ASC]", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC, b@1 ASC], has_header=false",]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC, b@1 ASC], file_type=csv, has_header=false",]; assert_optimized!(expected_input, expected_optimized, physical_plan, false); Ok(()) @@ -1393,11 +1367,11 @@ async fn test_pushdown_through_spm() -> Result<()> { let expected_input = ["SortExec: expr=[a@0 ASC, b@1 ASC, c@2 ASC], preserve_partitioning=[false]", " SortPreservingMergeExec: [a@0 ASC, b@1 ASC]", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC, b@1 ASC], has_header=false",]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC, b@1 ASC], file_type=csv, has_header=false",]; let expected_optimized = ["SortPreservingMergeExec: [a@0 ASC, b@1 ASC]", " SortExec: expr=[a@0 ASC, b@1 ASC, c@2 ASC], preserve_partitioning=[true]", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC, b@1 ASC], has_header=false",]; + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC, b@1 ASC], file_type=csv, has_header=false",]; assert_optimized!(expected_input, expected_optimized, physical_plan, false); Ok(()) @@ -1421,7 +1395,7 @@ async fn test_window_multi_layer_requirement() -> Result<()> { " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC, b@1 ASC", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", " SortExec: expr=[a@0 ASC, b@1 ASC], preserve_partitioning=[false]", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", ]; let expected_optimized = [ "BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", @@ -1429,7 +1403,7 @@ async fn test_window_multi_layer_requirement() -> Result<()> { " CoalescePartitionsExec", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], file_type=csv, has_header=false", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, false); @@ -1451,7 +1425,7 @@ async fn test_not_replaced_with_partial_sort_for_bounded_input() -> Result<()> { ); let expected_input = [ "SortExec: expr=[a@0 ASC, b@1 ASC, c@2 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[b@1 ASC, c@2 ASC]" + " DataSourceExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[b@1 ASC, c@2 ASC], file_type=parquet" ]; let expected_no_change = expected_input; assert_optimized!(expected_input, expected_no_change, physical_plan, false); @@ -1554,11 +1528,11 @@ async fn test_remove_unnecessary_sort() -> Result<()> { let expected_input = [ "SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", " SortExec: expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", ]; let expected_optimized = [ "SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -1612,14 +1586,14 @@ async fn test_remove_unnecessary_sort_window_multilayer() -> Result<()> { " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", " CoalesceBatchesExec: target_batch_size=128", " SortExec: expr=[non_nullable_col@1 DESC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]"]; + " DataSourceExec: partitions=1, partition_sizes=[0]"]; let expected_optimized = ["WindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(NULL), is_causal: false }]", " FilterExec: NOT non_nullable_col@1", " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", " CoalesceBatchesExec: target_batch_size=128", " SortExec: expr=[non_nullable_col@1 DESC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]"]; + " DataSourceExec: partitions=1, partition_sizes=[0]"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -1636,11 +1610,11 @@ async fn test_add_required_sort() -> Result<()> { let expected_input = [ "SortPreservingMergeExec: [nullable_col@0 ASC]", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", ]; let expected_optimized = [ "SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -1663,11 +1637,11 @@ async fn test_remove_unnecessary_sort1() -> Result<()> { " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", " SortPreservingMergeExec: [nullable_col@0 ASC]", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", ]; let expected_optimized = [ "SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -1701,13 +1675,13 @@ async fn test_remove_unnecessary_sort2() -> Result<()> { " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", " SortPreservingMergeExec: [non_nullable_col@1 ASC]", " SortExec: expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", ]; let expected_optimized = [ "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -1745,14 +1719,14 @@ async fn test_remove_unnecessary_sort3() -> Result<()> { " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", " SortPreservingMergeExec: [non_nullable_col@1 ASC]", " SortExec: expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", ]; let expected_optimized = [ "AggregateExec: mode=Final, gby=[], aggr=[]", " CoalescePartitionsExec", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -1796,18 +1770,18 @@ async fn test_remove_unnecessary_sort4() -> Result<()> { " SortExec: expr=[non_nullable_col@1 ASC], preserve_partitioning=[true]", " UnionExec", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[0]"]; + " DataSourceExec: partitions=1, partition_sizes=[0]"]; let expected_optimized = ["SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[true]", " FilterExec: NOT non_nullable_col@1", " UnionExec", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[0]"]; + " DataSourceExec: partitions=1, partition_sizes=[0]"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -1835,11 +1809,11 @@ async fn test_remove_unnecessary_sort6() -> Result<()> { let expected_input = [ "SortExec: expr=[non_nullable_col@1 ASC, nullable_col@0 ASC], preserve_partitioning=[false]", " SortExec: TopK(fetch=2), expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", ]; let expected_optimized = [ "SortExec: TopK(fetch=2), expr=[non_nullable_col@1 ASC, nullable_col@0 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -1869,12 +1843,12 @@ async fn test_remove_unnecessary_sort7() -> Result<()> { let expected_input = [ "SortExec: TopK(fetch=2), expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", " SortExec: expr=[non_nullable_col@1 ASC, nullable_col@0 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", ]; let expected_optimized = [ "GlobalLimitExec: skip=0, fetch=2", " SortExec: expr=[non_nullable_col@1 ASC, nullable_col@0 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -1902,12 +1876,12 @@ async fn test_remove_unnecessary_sort8() -> Result<()> { "SortExec: expr=[non_nullable_col@1 ASC, nullable_col@0 ASC], preserve_partitioning=[false]", " LocalLimitExec: fetch=2", " SortExec: expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", ]; let expected_optimized = [ "LocalLimitExec: fetch=2", " SortExec: TopK(fetch=2), expr=[non_nullable_col@1 ASC, nullable_col@0 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -1930,13 +1904,13 @@ async fn test_do_not_pushdown_through_limit() -> Result<()> { "SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", " GlobalLimitExec: skip=0, fetch=5", " SortExec: expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", ]; let expected_optimized = [ "SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", " GlobalLimitExec: skip=0, fetch=5", " SortExec: expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -1958,11 +1932,11 @@ async fn test_remove_unnecessary_spm1() -> Result<()> { "SortPreservingMergeExec: [nullable_col@0 ASC]", " SortPreservingMergeExec: [non_nullable_col@1 ASC]", " SortPreservingMergeExec: [non_nullable_col@1 ASC]", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", ]; let expected_optimized = [ "SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -2006,11 +1980,11 @@ async fn test_change_wrong_sorting() -> Result<()> { let expected_input = [ "SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", ]; let expected_optimized = [ "SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -2033,11 +2007,11 @@ async fn test_change_wrong_sorting2() -> Result<()> { "SortPreservingMergeExec: [non_nullable_col@1 ASC]", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", " SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", ]; let expected_optimized = [ "SortExec: expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -2065,13 +2039,13 @@ async fn test_multiple_sort_window_exec() -> Result<()> { " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]"]; + " DataSourceExec: partitions=1, partition_sizes=[0]"]; let expected_optimized = ["BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]"]; + " DataSourceExec: partitions=1, partition_sizes=[0]"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -2099,7 +2073,7 @@ async fn test_commutativity() -> Result<()> { "SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", ]; assert_eq!( expected_input, actual, @@ -2156,13 +2130,13 @@ async fn test_coalesce_propagate() -> Result<()> { " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", " CoalescePartitionsExec", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", ]; let expected_optimized = [ "SortPreservingMergeExec: [nullable_col@0 ASC]", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[true]", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -2202,7 +2176,7 @@ async fn test_push_with_required_input_ordering_prohibited() -> Result<()> { // SortExec: expr=[b] <-- can't push this down // RequiredInputOrder expr=[a] <-- this requires input sorted by a, and preserves the input order // SortExec: expr=[a] - // MemoryExec + // DataSourceExec let schema = create_test_schema3()?; let sort_exprs_a = LexOrdering::new(vec![sort_expr("a", &schema)]); let sort_exprs_b = LexOrdering::new(vec![sort_expr("b", &schema)]); @@ -2218,7 +2192,7 @@ async fn test_push_with_required_input_ordering_prohibited() -> Result<()> { "SortExec: expr=[b@1 ASC], preserve_partitioning=[false]", " RequiredInputOrderingExec", " SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", ]; // should not be able to push shorts let expected_no_change = expected_input; @@ -2232,7 +2206,7 @@ async fn test_push_with_required_input_ordering_allowed() -> Result<()> { // SortExec: expr=[a,b] <-- can push this down (as it is compatible with the required input ordering) // RequiredInputOrder expr=[a] <-- this requires input sorted by a, and preserves the input order // SortExec: expr=[a] - // MemoryExec + // DataSourceExec let schema = create_test_schema3()?; let sort_exprs_a = LexOrdering::new(vec![sort_expr("a", &schema)]); let sort_exprs_ab = @@ -2249,13 +2223,13 @@ async fn test_push_with_required_input_ordering_allowed() -> Result<()> { "SortExec: expr=[a@0 ASC, b@1 ASC], preserve_partitioning=[false]", " RequiredInputOrderingExec", " SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", ]; // should able to push shorts let expected = [ "RequiredInputOrderingExec", " SortExec: expr=[a@0 ASC, b@1 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", ]; assert_optimized!(expected_input, expected, plan, true); Ok(()) diff --git a/datafusion/physical-optimizer/src/coalesce_batches.rs b/datafusion/physical-optimizer/src/coalesce_batches.rs index 48c336fc771f5..5cf2c877c61a4 100644 --- a/datafusion/physical-optimizer/src/coalesce_batches.rs +++ b/datafusion/physical-optimizer/src/coalesce_batches.rs @@ -70,7 +70,8 @@ impl PhysicalOptimizerRule for CoalesceBatches { repart_exec.partitioning().clone(), Partitioning::RoundRobinBatch(_) ) - }).unwrap_or(false); + }) + .unwrap_or(false); if wrap_in_coalesce { Ok(Transformed::yes(Arc::new(CoalesceBatchesExec::new( plan, From 9b4a9584a1509709c9d0ded2f13644c694e98575 Mon Sep 17 00:00:00 2001 From: Weijun-H Date: Fri, 7 Feb 2025 13:12:22 +0800 Subject: [PATCH 11/25] chore --- .../src/repartition/on_demand_repartition.rs | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/datafusion/physical-plan/src/repartition/on_demand_repartition.rs b/datafusion/physical-plan/src/repartition/on_demand_repartition.rs index b1150829b1c7f..6b222537c51df 100644 --- a/datafusion/physical-plan/src/repartition/on_demand_repartition.rs +++ b/datafusion/physical-plan/src/repartition/on_demand_repartition.rs @@ -879,7 +879,7 @@ mod tests { let expected_plan = [ "CoalescePartitionsExec", " OnDemandRepartitionExec: partitioning=OnDemand(3), input_partitions=2", - " MemoryExec: partitions=2, partition_sizes=[2, 2]", + " DataSourceExec: partitions=2, partition_sizes=[2, 2]", ]; assert_plan!(expected_plan, coalesce_exec.clone()); @@ -1213,8 +1213,8 @@ mod tests { let expected_plan = [ "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=2, preserve_order=true, sort_exprs=c0@0 ASC", " UnionExec", - " MemoryExec: partitions=1, partition_sizes=[0], output_ordering=c0@0 ASC", - " MemoryExec: partitions=1, partition_sizes=[0], output_ordering=c0@0 ASC", + " DataSourceExec: partitions=1, partition_sizes=[0], output_ordering=c0@0 ASC", + " DataSourceExec: partitions=1, partition_sizes=[0], output_ordering=c0@0 ASC", ]; assert_plan!(expected_plan, exec); Ok(()) @@ -1250,7 +1250,7 @@ mod tests { ))); // output has multiple partitions, and is sorted - let union = UnionExec::new(vec![source.clone(), source]); + let union = UnionExec::new(vec![Arc::::clone(&source), source]); let repartition_exec = OnDemandRepartitionExec::try_new(Arc::new(union), Partitioning::OnDemand(5)) .unwrap() @@ -1307,7 +1307,7 @@ mod tests { // Repartition should not preserve order let expected_plan = [ "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[0], output_ordering=c0@0 ASC", + " DataSourceExec: partitions=1, partition_sizes=[0], output_ordering=c0@0 ASC", ]; assert_plan!(expected_plan, exec); Ok(()) @@ -1329,8 +1329,8 @@ mod tests { let expected_plan = [ "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=2", " UnionExec", - " MemoryExec: partitions=1, partition_sizes=[0]", - " MemoryExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", + " DataSourceExec: partitions=1, partition_sizes=[0]", ]; assert_plan!(expected_plan, exec); Ok(()) From ac85464c4fc557e1d21e97dc5a959d7c9ac21508 Mon Sep 17 00:00:00 2001 From: Weijun-H Date: Fri, 7 Feb 2025 13:55:56 +0800 Subject: [PATCH 12/25] chore: Fix slt --- .../test_files/on_demand_repartition.slt | 26 +++++++++---------- 1 file changed, 13 insertions(+), 13 deletions(-) diff --git a/datafusion/sqllogictest/test_files/on_demand_repartition.slt b/datafusion/sqllogictest/test_files/on_demand_repartition.slt index 5e7235c4012d2..afc92eac83d1a 100644 --- a/datafusion/sqllogictest/test_files/on_demand_repartition.slt +++ b/datafusion/sqllogictest/test_files/on_demand_repartition.slt @@ -56,7 +56,7 @@ physical_plan 03)----RepartitionExec: partitioning=Hash([column1@0], 4), input_partitions=4 04)------AggregateExec: mode=Partial, gby=[column1@0 as column1], aggr=[sum(parquet_table.column2)] 05)--------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 -06)----------ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition/parquet_table/2.parquet]]}, projection=[column1, column2] +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition/parquet_table/2.parquet]]}, projection=[column1, column2], file_type=parquet # Disable on-demand repartitioning statement ok @@ -74,7 +74,7 @@ physical_plan 03)----RepartitionExec: partitioning=Hash([column1@0], 4), input_partitions=4 04)------AggregateExec: mode=Partial, gby=[column1@0 as column1], aggr=[sum(parquet_table.column2)] 05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -06)----------ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition/parquet_table/2.parquet]]}, projection=[column1, column2] +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/repartition/parquet_table/2.parquet]]}, projection=[column1, column2], file_type=parquet # Cleanup statement ok @@ -125,7 +125,7 @@ physical_plan 03)----RepartitionExec: partitioning=Hash([column1@0], 4), input_partitions=4 04)------AggregateExec: mode=Partial, gby=[column1@0 as column1], aggr=[] 05)--------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 -06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/quote.csv]]}, projection=[column1], has_header=true +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/quote.csv]]}, projection=[column1], file_type=csv, has_header=true query TT EXPLAIN SELECT column1 FROM csv_with_escape GROUP BY column1; @@ -139,7 +139,7 @@ physical_plan 03)----RepartitionExec: partitioning=Hash([column1@0], 4), input_partitions=4 04)------AggregateExec: mode=Partial, gby=[column1@0 as column1], aggr=[] 05)--------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 -06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/escape.csv]]}, projection=[column1], has_header=true +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/escape.csv]]}, projection=[column1], file_type=csv, has_header=true statement ok DROP TABLE csv_with_quote; @@ -171,7 +171,7 @@ physical_plan 03)----RepartitionExec: partitioning=Hash([column1@0], 4), input_partitions=4 04)------AggregateExec: mode=Partial, gby=[column1@0 as column1], aggr=[] 05)--------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 -06)----------ArrowExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.arrow]]}, projection=[column1] +06)----------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/example.arrow]]}, projection=[column1], file_type=arrow statement ok DROP TABLE arrow_simple; @@ -275,7 +275,7 @@ physical_plan 03)----RepartitionExec: partitioning=Hash([column1@0], 3), input_partitions=3 04)------OnDemandRepartitionExec: partitioning=OnDemand(3), input_partitions=1 05)--------AggregateExec: mode=Partial, gby=[column1@0 as column1], aggr=[] -06)----------MemoryExec: partitions=1, partition_sizes=[1] +06)----------DataSourceExec: partitions=1, partition_sizes=[1] statement ok DROP TABLE memory_table; @@ -330,11 +330,11 @@ physical_plan 03)----CoalesceBatchesExec: target_batch_size=2 04)------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 05)--------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 -06)----------MemoryExec: partitions=1, partition_sizes=[1] +06)----------DataSourceExec: partitions=1, partition_sizes=[1] 07)----CoalesceBatchesExec: target_batch_size=2 08)------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 09)--------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 -10)----------MemoryExec: partitions=1, partition_sizes=[1] +10)----------DataSourceExec: partitions=1, partition_sizes=[1] # right join query TT @@ -353,11 +353,11 @@ physical_plan 03)----CoalesceBatchesExec: target_batch_size=2 04)------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 05)--------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 -06)----------MemoryExec: partitions=1, partition_sizes=[1] +06)----------DataSourceExec: partitions=1, partition_sizes=[1] 07)----CoalesceBatchesExec: target_batch_size=2 08)------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 09)--------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 -10)----------MemoryExec: partitions=1, partition_sizes=[1] +10)----------DataSourceExec: partitions=1, partition_sizes=[1] # inner join query TT @@ -376,11 +376,11 @@ physical_plan 03)----CoalesceBatchesExec: target_batch_size=2 04)------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 05)--------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 -06)----------MemoryExec: partitions=1, partition_sizes=[1] +06)----------DataSourceExec: partitions=1, partition_sizes=[1] 07)----CoalesceBatchesExec: target_batch_size=2 08)------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 09)--------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 -10)----------MemoryExec: partitions=1, partition_sizes=[1] +10)----------DataSourceExec: partitions=1, partition_sizes=[1] statement ok DROP TABLE join_t1; @@ -442,4 +442,4 @@ physical_plan 09)----------------CoalesceBatchesExec: target_batch_size=4096 10)------------------RepartitionExec: partitioning=Hash([c1@0, c2@1], 2), input_partitions=2 11)--------------------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 -12)----------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c4], has_header=true +12)----------------------DataSourceExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c4], file_type=csv, has_header=true From 48df1d383b7ed11699f77baa6f6066c83e086388 Mon Sep 17 00:00:00 2001 From: Weijun-H Date: Fri, 7 Feb 2025 15:13:46 +0800 Subject: [PATCH 13/25] chore --- datafusion/core/src/physical_planner.rs | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 392bd89db73a7..a74cdcc5920bf 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -787,10 +787,7 @@ impl DefaultPhysicalPlanner { let input_dfschema = input.as_ref().schema(); let physical_partitioning = match partitioning_scheme { LogicalPartitioning::RoundRobinBatch(n) => { - return Ok(Arc::new(RepartitionExec::try_new( - physical_input, - Partitioning::RoundRobinBatch(*n), - )?)); + Partitioning::RoundRobinBatch(*n) } LogicalPartitioning::Hash(expr, n) => { let runtime_expr = expr From 12d35231cde4fb542f4bc4175c3eab9c8a1c84f1 Mon Sep 17 00:00:00 2001 From: Weijun-H Date: Fri, 7 Feb 2025 17:09:38 +0800 Subject: [PATCH 14/25] fix: Replaced all roundrobin repartition after setting --- .../src/enforce_distribution.rs | 60 +++++++++++++++++-- .../src/enforce_sorting/mod.rs | 15 ++++- .../replace_with_order_preserving_variants.rs | 29 ++++++++- 3 files changed, 95 insertions(+), 9 deletions(-) diff --git a/datafusion/physical-optimizer/src/enforce_distribution.rs b/datafusion/physical-optimizer/src/enforce_distribution.rs index 6be72ebb5d9b0..8d4ed73081b5a 100644 --- a/datafusion/physical-optimizer/src/enforce_distribution.rs +++ b/datafusion/physical-optimizer/src/enforce_distribution.rs @@ -27,8 +27,8 @@ use std::sync::Arc; use crate::optimizer::PhysicalOptimizerRule; use crate::output_requirements::OutputRequirementExec; use crate::utils::{ - add_sort_above_with_check, is_coalesce_partitions, is_repartition, - is_sort_preserving_merge, + add_sort_above_with_check, is_coalesce_partitions, is_on_demand_repartition, + is_repartition, is_sort_preserving_merge, }; use arrow::compute::SortOptions; @@ -417,6 +417,10 @@ pub fn adjust_input_keys_ordering( requirements.data.clear(); } } else if plan.as_any().downcast_ref::().is_some() + || plan + .as_any() + .downcast_ref::() + .is_some() || plan .as_any() .downcast_ref::() @@ -870,6 +874,32 @@ fn add_roundrobin_on_top( } } +fn add_on_demand_on_top( + input: DistributionContext, + n_target: usize, +) -> Result { + // Adding repartition is helpful: + if input.plan.output_partitioning().partition_count() < n_target { + // When there is an existing ordering, we preserve ordering + // during repartition. This will be un-done in the future + // If any of the following conditions is true + // - Preserving ordering is not helpful in terms of satisfying ordering requirements + // - Usage of order preserving variants is not desirable + // (determined by flag `config.optimizer.prefer_existing_sort`) + let partitioning = Partitioning::OnDemand(n_target); + let repartition = + OnDemandRepartitionExec::try_new(Arc::clone(&input.plan), partitioning)? + .with_preserve_order(); + + let new_plan = Arc::new(repartition) as _; + + Ok(DistributionContext::new(new_plan, true, vec![input])) + } else { + // Partition is not helpful, we already have desired number of partitions. + Ok(input) + } +} + /// Adds a hash repartition operator: /// - to increase parallelism, and/or /// - to satisfy requirements of the subsequent operators. @@ -989,6 +1019,7 @@ fn remove_dist_changing_operators( mut distribution_context: DistributionContext, ) -> Result { while is_repartition(&distribution_context.plan) + || is_on_demand_repartition(&distribution_context.plan) || is_coalesce_partitions(&distribution_context.plan) || is_sort_preserving_merge(&distribution_context.plan) { @@ -1048,6 +1079,18 @@ fn replace_order_preserving_variants( )?); return Ok(context); } + } else if let Some(repartition) = context + .plan + .as_any() + .downcast_ref::() + { + if repartition.preserve_order() { + context.plan = Arc::new(OnDemandRepartitionExec::try_new( + Arc::clone(&context.children[0].plan), + repartition.partitioning().clone(), + )?); + return Ok(context); + } } context.update_plan_from_children() @@ -1303,7 +1346,11 @@ pub fn ensure_distribution( if add_roundrobin { // Add round-robin repartitioning on top of the operator // to increase parallelism. - child = add_roundrobin_on_top(child, target_partitions)?; + child = if prefer_round_robin_repartition { + add_roundrobin_on_top(child, target_partitions)? + } else { + add_on_demand_on_top(child, target_partitions)? + }; } // When inserting hash is necessary to satisfy hash requirement, insert hash repartition. if hash_necessary { @@ -1315,7 +1362,11 @@ pub fn ensure_distribution( if add_roundrobin { // Add round-robin repartitioning on top of the operator // to increase parallelism. - child = add_roundrobin_on_top(child, target_partitions)?; + child = if prefer_round_robin_repartition { + add_roundrobin_on_top(child, target_partitions)? + } else { + add_on_demand_on_top(child, target_partitions)? + } } } }; @@ -1450,5 +1501,6 @@ fn update_children(mut dist_context: DistributionContext) -> Result() + { + node.plan = Arc::new(OnDemandRepartitionExec::try_new( + Arc::clone(&node.children[0].plan), + repartition.properties().output_partitioning().clone(), + )?) as _; } }; // Deleting a merging sort may invalidate distribution requirements. diff --git a/datafusion/physical-optimizer/src/enforce_sorting/replace_with_order_preserving_variants.rs b/datafusion/physical-optimizer/src/enforce_sorting/replace_with_order_preserving_variants.rs index 2c5c0d4d510ec..ec978ba832686 100644 --- a/datafusion/physical-optimizer/src/enforce_sorting/replace_with_order_preserving_variants.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/replace_with_order_preserving_variants.rs @@ -22,7 +22,8 @@ use std::sync::Arc; use crate::utils::{ - is_coalesce_partitions, is_repartition, is_sort, is_sort_preserving_merge, + is_coalesce_partitions, is_on_demand_repartition, is_repartition, is_sort, + is_sort_preserving_merge, }; use datafusion_common::config::ConfigOptions; @@ -31,6 +32,7 @@ use datafusion_common::Result; use datafusion_physical_expr_common::sort_expr::LexOrdering; use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; use datafusion_physical_plan::execution_plan::EmissionType; +use datafusion_physical_plan::repartition::on_demand_repartition::OnDemandRepartitionExec; use datafusion_physical_plan::repartition::RepartitionExec; use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; use datafusion_physical_plan::tree_node::PlanContext; @@ -57,6 +59,7 @@ pub fn update_order_preservation_ctx_children_data(opc: &mut OrderPreservationCo maintains_input_order[idx] || is_coalesce_partitions(plan) || is_repartition(plan) + || is_on_demand_repartition(plan) }; // We cut the path towards nodes that do not maintain ordering. @@ -68,7 +71,8 @@ pub fn update_order_preservation_ctx_children_data(opc: &mut OrderPreservationCo *data = if plan_children.is_empty() { false } else if !children[0].data - && ((is_repartition(plan) && !maintains_input_order[0]) + && (((is_repartition(plan) || is_on_demand_repartition(plan)) + && !maintains_input_order[0]) || (is_coalesce_partitions(plan) && plan_children[0].output_ordering().is_some())) { @@ -135,6 +139,20 @@ fn plan_with_order_preserving_variants( ) as _; sort_input.children[0].data = true; return Ok(sort_input); + } + if is_on_demand_repartition(&sort_input.plan) + && !sort_input.plan.maintains_input_order()[0] + && is_spr_better + { + // When a `OnDemandRepartitionExec` doesn't preserve ordering, replace it with + // a sort-preserving variant if appropriate: + let child = Arc::clone(&sort_input.children[0].plan); + let partitioning = sort_input.plan.output_partitioning().clone(); + sort_input.plan = Arc::new( + OnDemandRepartitionExec::try_new(child, partitioning)?.with_preserve_order(), + ) as _; + sort_input.children[0].data = true; + return Ok(sort_input); } else if is_coalesce_partitions(&sort_input.plan) && is_spm_better { let child = &sort_input.children[0].plan; if let Some(ordering) = child.output_ordering() { @@ -188,6 +206,13 @@ fn plan_with_order_breaking_variants( let child = Arc::clone(&sort_input.children[0].plan); let partitioning = plan.output_partitioning().clone(); sort_input.plan = Arc::new(RepartitionExec::try_new(child, partitioning)?) as _; + } else if is_on_demand_repartition(plan) && plan.maintains_input_order()[0] { + // When a `OnDemandRepartitionExec` preserves ordering, replace it with a + // non-sort-preserving variant: + let child = Arc::clone(&sort_input.children[0].plan); + let partitioning = plan.output_partitioning().clone(); + sort_input.plan = + Arc::new(OnDemandRepartitionExec::try_new(child, partitioning)?) as _; } else if is_sort_preserving_merge(plan) { // Replace `SortPreservingMergeExec` with a `CoalescePartitionsExec`: let child = Arc::clone(&sort_input.children[0].plan); From 4e7e0ef3a442e2cc66f01b7e09ff454625ed5427 Mon Sep 17 00:00:00 2001 From: Weijun-H Date: Fri, 7 Feb 2025 17:56:42 +0800 Subject: [PATCH 15/25] chore: Updated the code review --- .../src/repartition/on_demand_repartition.rs | 42 ++++++++++--------- 1 file changed, 22 insertions(+), 20 deletions(-) diff --git a/datafusion/physical-plan/src/repartition/on_demand_repartition.rs b/datafusion/physical-plan/src/repartition/on_demand_repartition.rs index 6b222537c51df..409e266e64e72 100644 --- a/datafusion/physical-plan/src/repartition/on_demand_repartition.rs +++ b/datafusion/physical-plan/src/repartition/on_demand_repartition.rs @@ -416,7 +416,7 @@ impl OnDemandRepartitionExec { partition_channels: Default::default(), }) } - + // Executes the input plan and poll stream into the buffer, records fetch_time and buffer_time metrics async fn process_input( input: Arc, partition: usize, @@ -440,20 +440,20 @@ impl OnDemandRepartitionExec { let batch = stream.next().await; timer.done(); - // send the batch to the buffer channel - if let Some(batch) = batch { - let timer = send_buffer_time.timer(); - buffer_tx.send(batch?).await.map_err(|e| { - internal_datafusion_err!( - "Error sending batch to buffer channel for partition {}: {}", - partition, - e - ) - })?; - timer.done(); - } else { + let Some(batch) = batch else { break; - } + }; + let timer = send_buffer_time.timer(); + // Feed the buffer with batch, since the buffer channel has limited capacity + // The process waits here until one is consumed + buffer_tx.send(batch?).await.map_err(|e| { + internal_datafusion_err!( + "Error sending batch to buffer channel for partition {}: {}", + partition, + e + ) + })?; + timer.done(); } Ok(()) @@ -465,7 +465,7 @@ impl OnDemandRepartitionExec { /// txs hold the output sending channels for each output partition pub(crate) async fn pull_from_input( input: Arc, - partition: usize, + input_partition: usize, mut output_channels: HashMap< usize, (DistributionSender, SharedMemoryReservation), @@ -475,27 +475,29 @@ impl OnDemandRepartitionExec { metrics: OnDemandRepartitionMetrics, context: Arc, ) -> Result<()> { - // execute the child operator in a separate task + // initialize buffer channel so that we can pre-fetch from input let (buffer_tx, buffer_rx) = async_channel::bounded::(2); + // execute the child operator in a separate task + // that pushes batches into buffer channel with limited capacity let processing_task = SpawnedTask::spawn(Self::process_input( Arc::clone(&input), - partition, + input_partition, buffer_tx, Arc::clone(&context), metrics.fetch_time.clone(), metrics.send_buffer_time.clone(), )); - // While there are still outputs to send to, keep pulling inputs let mut batches_until_yield = partitioning.partition_count(); + // When the input is done, break the loop while !output_channels.is_empty() { - // When the input is done, break the loop + // Fetch the batch from the buffer, ideally this should reduce the time gap between the requester and the input stream let batch = match buffer_rx.recv().await { Ok(batch) => batch, _ => break, }; - // Get the partition number from the output partition + // Wait until a partition is requested, then get the output partition information let partition = output_partition_rx.recv().await.map_err(|e| { internal_datafusion_err!( "Error receiving partition number from output partition: {}", From 5af3eebbceb5e1f94553a9054c95f281ae81287c Mon Sep 17 00:00:00 2001 From: Weijun-H Date: Sat, 8 Feb 2025 14:58:39 +0800 Subject: [PATCH 16/25] chore: Added comment for is_request --- .../src/repartition/on_demand_repartition.rs | 53 +++++++++++-------- 1 file changed, 31 insertions(+), 22 deletions(-) diff --git a/datafusion/physical-plan/src/repartition/on_demand_repartition.rs b/datafusion/physical-plan/src/repartition/on_demand_repartition.rs index 409e266e64e72..70a93a92d2515 100644 --- a/datafusion/physical-plan/src/repartition/on_demand_repartition.rs +++ b/datafusion/physical-plan/src/repartition/on_demand_repartition.rs @@ -46,9 +46,9 @@ use crate::{DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, Stat use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; -use async_channel::{Receiver, Sender}; +use async_channel::{Receiver, Sender, TrySendError}; -use datafusion_common::{internal_datafusion_err, Result}; +use datafusion_common::{internal_datafusion_err, internal_err, Result}; use datafusion_common_runtime::SpawnedTask; use datafusion_execution::memory_pool::MemoryConsumer; use datafusion_execution::TaskContext; @@ -610,7 +610,7 @@ struct OnDemandPerPartitionStream { /// Partition number partition: usize, - /// Sender State + /// Avoid sending partition number multiple times, set to true after sending partition number is_requested: bool, } @@ -621,18 +621,21 @@ impl Stream for OnDemandPerPartitionStream { mut self: Pin<&mut Self>, cx: &mut Context<'_>, ) -> Poll> { - if !self.is_requested && !self.sender.is_closed() { - self.sender.try_send(self.partition).map_err(|e| { - internal_datafusion_err!( - "Error sending partition number to the receiver for partition {}: {}", - self.partition, - e - ) - })?; - self.is_requested = true; + if !self.is_requested { + match self.sender.try_send(self.partition) { + Ok(_) => self.is_requested = true, + // unobunded channel, should not be full + Err(TrySendError::Full(_)) => { + return internal_err!("Partition sender {} is full", self.partition)?; + } + Err(TrySendError::Closed(_)) => { + return Poll::Ready(None); + } + } } let result = ready!(self.receiver.recv().poll_unpin(cx)); + // set is_requested to false, when receiving a batch self.is_requested = false; match result { @@ -681,7 +684,7 @@ struct OnDemandRepartitionStream { /// Partition number partition: usize, - /// Sender state + /// Avoid sending partition number multiple times, set to true after sending partition number is_requested: bool, } @@ -694,18 +697,24 @@ impl Stream for OnDemandRepartitionStream { ) -> Poll> { loop { // Send partition number to input partitions - if !self.is_requested && !self.sender.is_closed() { - self.sender.try_send(self.partition).map_err(|e| { - internal_datafusion_err!( - "Error sending partition number to the receiver for partition {}: {}", - self.partition, - e - ) - })?; - self.is_requested = true; + if !self.is_requested { + match self.sender.try_send(self.partition) { + Ok(_) => self.is_requested = true, + // unobunded channel, should not be full + Err(TrySendError::Full(_)) => { + return internal_err!( + "Partition sender {} is full", + self.partition + )?; + } + Err(TrySendError::Closed(_)) => { + return Poll::Ready(None); + } + } } let result = ready!(self.input.recv().poll_unpin(cx)); + // set is_requested to false, when receiving a batch self.is_requested = false; match result { From 115929952fa0d182be9824031da98d8a18cf4975 Mon Sep 17 00:00:00 2001 From: Weijun-H Date: Sat, 8 Feb 2025 14:59:07 +0800 Subject: [PATCH 17/25] chore: Added tests for is_request --- .../sort_preserving_repartition_fuzz.rs | 104 ++++++++++++++---- 1 file changed, 81 insertions(+), 23 deletions(-) diff --git a/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs b/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs index 06b93d41af362..660d681731818 100644 --- a/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs @@ -42,6 +42,7 @@ mod sp_repartition_fuzz_tests { expressions::{col, Column}, ConstExpr, PhysicalExpr, PhysicalSortExpr, }; + use datafusion_physical_plan::repartition::on_demand_repartition::OnDemandRepartitionExec; use test_utils::add_empty_batches; use datafusion::datasource::memory::MemorySourceConfig; @@ -296,25 +297,40 @@ mod sp_repartition_fuzz_tests { // behaviour. We can choose, n_distinct as we like. However, // we chose it a large number to decrease probability of having same rows in the table. let n_distinct = 1_000_000; - for (is_first_roundrobin, is_first_sort_preserving) in - [(false, false), (false, true), (true, false), (true, true)] - { - for is_second_roundrobin in [false, true] { - let mut handles = Vec::new(); - - for seed in seed_start..seed_end { - #[allow(clippy::disallowed_methods)] // spawn allowed only in tests - let job = tokio::spawn(run_sort_preserving_repartition_test( - make_staggered_batches::(n_row, n_distinct, seed as u64), - is_first_roundrobin, - is_first_sort_preserving, - is_second_roundrobin, - )); - handles.push(job); - } - - for job in handles { - job.await.unwrap(); + for use_on_demand_repartition in [false, true] { + for (is_first_roundrobin, is_first_sort_preserving) in + [(false, false), (false, true), (true, false), (true, true)] + { + for is_second_roundrobin in [false, true] { + // On demand repartition only replaces the roundrobin repartition + if use_on_demand_repartition + && !is_first_roundrobin + && !is_second_roundrobin + { + continue; + } + let mut handles = Vec::new(); + + for seed in seed_start..seed_end { + #[allow(clippy::disallowed_methods)] + // spawn allowed only in tests + let job = tokio::spawn(run_sort_preserving_repartition_test( + make_staggered_batches::( + n_row, + n_distinct, + seed as u64, + ), + is_first_roundrobin, + is_first_sort_preserving, + is_second_roundrobin, + use_on_demand_repartition, + )); + handles.push(job); + } + + for job in handles { + job.await.unwrap(); + } } } } @@ -343,9 +359,17 @@ mod sp_repartition_fuzz_tests { // If `true`, second repartition executor after `DataSourceExec` will be in `RoundRobin` mode // else it will be in `Hash` mode is_second_roundrobin: bool, + // If `true`, `OnDemandRepartitionExec` will be used instead of `RepartitionExec` + use_on_demand_repartition: bool, ) { let schema = input1[0].schema(); - let session_config = SessionConfig::new().with_batch_size(50); + let mut session_config = SessionConfig::new().with_batch_size(50); + if use_on_demand_repartition { + session_config + .options_mut() + .optimizer + .prefer_round_robin_repartition = false; + } let ctx = SessionContext::new_with_config(session_config); let mut sort_keys = LexOrdering::default(); for ordering_col in ["a", "b", "c"] { @@ -367,8 +391,20 @@ mod sp_repartition_fuzz_tests { let hash_exprs = vec![col("c", &schema).unwrap()]; let intermediate = match (is_first_roundrobin, is_first_sort_preserving) { - (true, true) => sort_preserving_repartition_exec_round_robin(running_source), - (true, false) => repartition_exec_round_robin(running_source), + (true, true) => { + if use_on_demand_repartition { + sort_preserving_repartition_exec_on_demand(running_source) + } else { + sort_preserving_repartition_exec_round_robin(running_source) + } + } + (true, false) => { + if use_on_demand_repartition { + repartition_exec_on_demand(running_source) + } else { + repartition_exec_round_robin(running_source) + } + } (false, true) => { sort_preserving_repartition_exec_hash(running_source, hash_exprs.clone()) } @@ -376,7 +412,11 @@ mod sp_repartition_fuzz_tests { }; let intermediate = if is_second_roundrobin { - sort_preserving_repartition_exec_round_robin(intermediate) + if use_on_demand_repartition { + sort_preserving_repartition_exec_on_demand(intermediate) + } else { + sort_preserving_repartition_exec_round_robin(intermediate) + } } else { sort_preserving_repartition_exec_hash(intermediate, hash_exprs.clone()) }; @@ -399,6 +439,16 @@ mod sp_repartition_fuzz_tests { ) } + fn sort_preserving_repartition_exec_on_demand( + input: Arc, + ) -> Arc { + Arc::new( + OnDemandRepartitionExec::try_new(input, Partitioning::OnDemand(2)) + .unwrap() + .with_preserve_order(), + ) + } + fn repartition_exec_round_robin( input: Arc, ) -> Arc { @@ -407,6 +457,14 @@ mod sp_repartition_fuzz_tests { ) } + fn repartition_exec_on_demand( + input: Arc, + ) -> Arc { + Arc::new( + OnDemandRepartitionExec::try_new(input, Partitioning::OnDemand(2)).unwrap(), + ) + } + fn sort_preserving_repartition_exec_hash( input: Arc, hash_expr: Vec>, From dbab4770adc289b2fd86adb3757071416b235b0e Mon Sep 17 00:00:00 2001 From: Weijun-H Date: Sat, 8 Feb 2025 17:02:08 +0800 Subject: [PATCH 18/25] chore --- datafusion/sqllogictest/test_files/join.slt.part | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/sqllogictest/test_files/join.slt.part b/datafusion/sqllogictest/test_files/join.slt.part index 18806d431727e..21126a7479673 100644 --- a/datafusion/sqllogictest/test_files/join.slt.part +++ b/datafusion/sqllogictest/test_files/join.slt.part @@ -671,7 +671,7 @@ query TT explain select * from t1 inner join t2 on true; ---- logical_plan -01)Cross Join: +01)Cross Join: 02)--TableScan: t1 projection=[t1_id, t1_name, t1_int] 03)--TableScan: t2 projection=[t2_id, t2_name, t2_int] physical_plan From 24e70ed4625bba51be19a61061552ac73e5b5d98 Mon Sep 17 00:00:00 2001 From: Weijun-H Date: Sat, 8 Feb 2025 22:22:58 +0800 Subject: [PATCH 19/25] chore --- .../src/repartition/on_demand_repartition.rs | 29 ++++++++++++++----- 1 file changed, 21 insertions(+), 8 deletions(-) diff --git a/datafusion/physical-plan/src/repartition/on_demand_repartition.rs b/datafusion/physical-plan/src/repartition/on_demand_repartition.rs index 70a93a92d2515..f6c72228ee5f2 100644 --- a/datafusion/physical-plan/src/repartition/on_demand_repartition.rs +++ b/datafusion/physical-plan/src/repartition/on_demand_repartition.rs @@ -420,7 +420,7 @@ impl OnDemandRepartitionExec { async fn process_input( input: Arc, partition: usize, - buffer_tx: Sender, + buffer_tx: tokio::sync::mpsc::Sender, context: Arc, fetch_time: metrics::Time, send_buffer_time: metrics::Time, @@ -476,7 +476,7 @@ impl OnDemandRepartitionExec { context: Arc, ) -> Result<()> { // initialize buffer channel so that we can pre-fetch from input - let (buffer_tx, buffer_rx) = async_channel::bounded::(2); + let (buffer_tx, mut buffer_rx) = tokio::sync::mpsc::channel(2); // execute the child operator in a separate task // that pushes batches into buffer channel with limited capacity let processing_task = SpawnedTask::spawn(Self::process_input( @@ -491,12 +491,6 @@ impl OnDemandRepartitionExec { let mut batches_until_yield = partitioning.partition_count(); // When the input is done, break the loop while !output_channels.is_empty() { - // Fetch the batch from the buffer, ideally this should reduce the time gap between the requester and the input stream - let batch = match buffer_rx.recv().await { - Ok(batch) => batch, - _ => break, - }; - // Wait until a partition is requested, then get the output partition information let partition = output_partition_rx.recv().await.map_err(|e| { internal_datafusion_err!( @@ -505,6 +499,25 @@ impl OnDemandRepartitionExec { ) })?; + // Fetch the batch from the buffer, ideally this should reduce the time gap between the requester and the input stream + let batch_opt = loop { + match buffer_rx.try_recv() { + Ok(batch) => break Some(batch), + Err(tokio::sync::mpsc::error::TryRecvError::Empty) => { + tokio::task::yield_now().await; + } + Err(tokio::sync::mpsc::error::TryRecvError::Disconnected) => { + break None + } + } + }; + + let batch = if let Some(batch) = batch_opt { + batch + } else { + break; + }; + let size = batch.get_array_memory_size(); let timer = metrics.send_time[partition].timer(); From 8e38fa05685611c193d6508a0f93d6e29bbdd77b Mon Sep 17 00:00:00 2001 From: Weijun-H Date: Sun, 9 Feb 2025 17:32:20 +0800 Subject: [PATCH 20/25] feat: Added prefer_round_robin for clickbench_partitioned --- benchmarks/bench.sh | 2 +- benchmarks/src/clickbench.rs | 8 ++++++++ 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/benchmarks/bench.sh b/benchmarks/bench.sh index 12e4d8002edec..a5574448f8842 100755 --- a/benchmarks/bench.sh +++ b/benchmarks/bench.sh @@ -481,7 +481,7 @@ run_clickbench_partitioned() { RESULTS_FILE="${RESULTS_DIR}/clickbench_partitioned.json" echo "RESULTS_FILE: ${RESULTS_FILE}" echo "Running clickbench (partitioned, 100 files) benchmark..." - $CARGO_COMMAND --bin dfbench -- clickbench --iterations 5 --path "${DATA_DIR}/hits_partitioned" --queries-path "${SCRIPT_DIR}/queries/clickbench/queries.sql" -o "${RESULTS_FILE}" + $CARGO_COMMAND --bin dfbench -- clickbench --iterations 5 --path "${DATA_DIR}/hits_partitioned" --prefer_round_robin "${PREFER_ROUND_ROBIN}" --queries-path "${SCRIPT_DIR}/queries/clickbench/queries.sql" -o "${RESULTS_FILE}" } # Runs the clickbench "extended" benchmark with a single large parquet file diff --git a/benchmarks/src/clickbench.rs b/benchmarks/src/clickbench.rs index a9750d9b4b846..4a9ec7714fe48 100644 --- a/benchmarks/src/clickbench.rs +++ b/benchmarks/src/clickbench.rs @@ -27,6 +27,9 @@ use datafusion_common::exec_datafusion_err; use datafusion_common::instant::Instant; use structopt::StructOpt; +// hack to avoid `default_value is meaningless for bool` errors +type BoolDefaultTrue = bool; + /// Run the clickbench benchmark /// /// The ClickBench[1] benchmarks are widely cited in the industry and @@ -68,6 +71,11 @@ pub struct RunOpt { /// If present, write results json here #[structopt(parse(from_os_str), short = "o", long = "output")] output_path: Option, + + /// If true then round robin repartitioning is used, if false then on demand repartitioning + /// True by default. + #[structopt(short = "r", long = "prefer_round_robin", default_value = "true")] + prefer_round_robin: BoolDefaultTrue, } struct AllQueries { From 156c348680af9d09e75213073a6f8a41985fa9cc Mon Sep 17 00:00:00 2001 From: Weijun-H Date: Mon, 10 Feb 2025 11:33:36 +0800 Subject: [PATCH 21/25] refactor: Used tokio mpsc for OnDemandRepartiion --- .../src/repartition/distributor_channels.rs | 24 ++ .../physical-plan/src/repartition/mod.rs | 143 +++------- .../src/repartition/on_demand_repartition.rs | 259 ++++++++++++++++-- 3 files changed, 290 insertions(+), 136 deletions(-) diff --git a/datafusion/physical-plan/src/repartition/distributor_channels.rs b/datafusion/physical-plan/src/repartition/distributor_channels.rs index 6e06c87a48213..b12343940a2e2 100644 --- a/datafusion/physical-plan/src/repartition/distributor_channels.rs +++ b/datafusion/physical-plan/src/repartition/distributor_channels.rs @@ -79,6 +79,17 @@ pub fn channels( (senders, receivers) } +pub fn tokio_channels( + n: usize, +) -> ( + Vec>, + Vec>, +) { + // only used for OnDemandRepartitionExec, so no need for unbounded capacity + let (senders, receivers) = (0..n).map(|_| tokio::sync::mpsc::channel(2)).unzip(); + (senders, receivers) +} + type PartitionAwareSenders = Vec>>; type PartitionAwareReceivers = Vec>>; @@ -92,6 +103,19 @@ pub fn partition_aware_channels( (0..n_in).map(|_| channels(n_out)).unzip() } +type OnDemandPartitionAwareSenders = Vec>>; +type OnDemandPartitionAwareReceivers = Vec>>; + +pub fn on_demand_partition_aware_channels( + n_in: usize, + n_out: usize, +) -> ( + OnDemandPartitionAwareSenders, + OnDemandPartitionAwareReceivers, +) { + (0..n_in).map(|_| tokio_channels(n_out)).unzip() +} + /// Erroring during [send](DistributionSender::send). /// /// This occurs when the [receiver](DistributionReceiver) is gone. diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index c1182adc88aa9..63c4e4bf01e1b 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -43,7 +43,6 @@ use crate::{DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, Stat use arrow::array::{PrimitiveArray, RecordBatch, RecordBatchOptions}; use arrow::compute::take_arrays; use arrow::datatypes::{SchemaRef, UInt32Type}; -use async_channel::Receiver; use datafusion_common::utils::transpose; use datafusion_common::HashMap; use datafusion_common::{not_impl_err, DataFusionError, Result}; @@ -56,7 +55,6 @@ use datafusion_physical_expr_common::sort_expr::LexOrdering; use futures::stream::Stream; use futures::{ready, FutureExt, StreamExt, TryStreamExt}; use log::trace; -use on_demand_repartition::{OnDemandRepartitionExec, OnDemandRepartitionMetrics}; use parking_lot::Mutex; mod distributor_channels; @@ -66,68 +64,6 @@ type MaybeBatch = Option>; type InputPartitionsToCurrentPartitionSender = Vec>; type InputPartitionsToCurrentPartitionReceiver = Vec>; -struct RepartitionExecStateBuilder { - /// Whether to enable pull based execution. - enable_pull_based: bool, - partition_receivers: Option>>, -} - -impl RepartitionExecStateBuilder { - fn new() -> Self { - Self { - enable_pull_based: false, - partition_receivers: None, - } - } - fn enable_pull_based(mut self, enable_pull_based: bool) -> Self { - self.enable_pull_based = enable_pull_based; - self - } - fn partition_receivers(mut self, partition_receivers: Vec>) -> Self { - self.partition_receivers = Some(partition_receivers); - self - } - - fn build( - &self, - input: Arc, - partitioning: Partitioning, - metrics: ExecutionPlanMetricsSet, - preserve_order: bool, - name: String, - context: Arc, - ) -> RepartitionExecState { - RepartitionExecState::new( - input, - partitioning, - metrics, - preserve_order, - name, - context, - self.enable_pull_based, - self.partition_receivers.clone(), - ) - } -} - -/// Inner state of [`RepartitionExec`]. -#[derive(Debug)] -struct RepartitionExecState { - /// Channels for sending batches from input partitions to output partitions. - /// Key is the partition number. - channels: HashMap< - usize, - ( - InputPartitionsToCurrentPartitionSender, - InputPartitionsToCurrentPartitionReceiver, - SharedMemoryReservation, - ), - >, - - /// Helper that ensures that that background job is killed once it is no longer needed. - abort_helper: Arc>>, -} - /// create channels for sending batches from input partitions to output partitions. fn create_repartition_channels( preserve_order: bool, @@ -185,8 +121,26 @@ fn create_partition_channels_hashmap( channels } + +/// Inner state of [`RepartitionExec`]. +#[derive(Debug)] +struct RepartitionExecState { + /// Channels for sending batches from input partitions to output partitions. + /// Key is the partition number. + channels: HashMap< + usize, + ( + InputPartitionsToCurrentPartitionSender, + InputPartitionsToCurrentPartitionReceiver, + SharedMemoryReservation, + ), + >, + + /// Helper that ensures that that background job is killed once it is no longer needed. + abort_helper: Arc>>, +} + impl RepartitionExecState { - #[allow(clippy::too_many_arguments)] fn new( input: Arc, partitioning: Partitioning, @@ -194,8 +148,6 @@ impl RepartitionExecState { preserve_order: bool, name: String, context: Arc, - enable_pull_based: bool, - partition_receivers: Option>>, ) -> Self { let num_input_partitions = input.output_partitioning().partition_count(); let num_output_partitions = partitioning.partition_count(); @@ -219,42 +171,16 @@ impl RepartitionExecState { }) .collect(); - let input_task = if enable_pull_based { - let partition_rx = if preserve_order { - partition_receivers.clone().expect( - "partition_receivers must be provided when preserve_order is enabled", - )[i] - .clone() - } else { - partition_receivers.clone().expect( - "partition_receivers must be provided when preserve_order is disabled", - )[0].clone() - }; - let r_metrics = - OnDemandRepartitionMetrics::new(i, num_output_partitions, &metrics); - - SpawnedTask::spawn(OnDemandRepartitionExec::pull_from_input( - Arc::clone(&input), - i, - txs.clone(), - partitioning.clone(), - partition_rx, - r_metrics, - Arc::clone(&context), - )) - } else { - let r_metrics = - RepartitionMetrics::new(i, num_output_partitions, &metrics); - - SpawnedTask::spawn(RepartitionExec::pull_from_input( - Arc::clone(&input), - i, - txs.clone(), - partitioning.clone(), - r_metrics, - Arc::clone(&context), - )) - }; + let r_metrics = RepartitionMetrics::new(i, num_output_partitions, &metrics); + + let input_task = SpawnedTask::spawn(RepartitionExec::pull_from_input( + Arc::clone(&input), + i, + txs.clone(), + partitioning.clone(), + r_metrics, + Arc::clone(&context), + )); // In a separate task, wait for each input to be done // (and pass along any errors, including panic!s) @@ -268,7 +194,6 @@ impl RepartitionExecState { spawned_tasks.push(wait_for_task); } - Self { channels, abort_helper: Arc::new(spawned_tasks), @@ -467,8 +392,6 @@ pub struct RepartitionExecBase { preserve_order: bool, /// Cache holding plan properties like equivalences, output partitioning etc. cache: PlanProperties, - /// Inner state that is initialized when the first output stream is created. - state: LazyState, } impl RepartitionExecBase { @@ -611,6 +534,8 @@ impl RepartitionExecBase { pub struct RepartitionExec { /// Common fields for all repartitioning executors base: RepartitionExecBase, + /// Inner state that is initialized when the first output stream is created. + state: LazyState, } #[derive(Debug, Clone)] @@ -776,7 +701,7 @@ impl ExecutionPlan for RepartitionExec { partition ); - let lazy_state = Arc::clone(&self.base.state); + let lazy_state = Arc::clone(&self.state); let input = Arc::clone(&self.base.input); let partitioning = self.partitioning().clone(); let metrics = self.base.metrics.clone(); @@ -797,7 +722,7 @@ impl ExecutionPlan for RepartitionExec { let context_captured = Arc::clone(&context); let state = lazy_state .get_or_init(|| async move { - Mutex::new(RepartitionExecStateBuilder::new().build( + Mutex::new(RepartitionExecState::new( input_captured, partitioning.clone(), metrics_captured, @@ -945,11 +870,11 @@ impl RepartitionExec { Ok(RepartitionExec { base: RepartitionExecBase { input, - state: Default::default(), metrics: ExecutionPlanMetricsSet::new(), preserve_order, cache, }, + state: Default::default(), }) } diff --git a/datafusion/physical-plan/src/repartition/on_demand_repartition.rs b/datafusion/physical-plan/src/repartition/on_demand_repartition.rs index f6c72228ee5f2..055c1d0537e00 100644 --- a/datafusion/physical-plan/src/repartition/on_demand_repartition.rs +++ b/datafusion/physical-plan/src/repartition/on_demand_repartition.rs @@ -27,6 +27,7 @@ use std::sync::Arc; use std::task::{Context, Poll}; use std::{any::Any, vec}; +use super::distributor_channels::{on_demand_partition_aware_channels, tokio_channels}; use super::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use super::{ DisplayAs, ExecutionPlanProperties, MaybeBatch, RecordBatchStream, @@ -36,10 +37,6 @@ use crate::common::SharedMemoryReservation; use crate::execution_plan::CardinalityEffect; use crate::metrics::{self, BaselineMetrics, MetricBuilder}; use crate::projection::{all_columns, make_with_child, ProjectionExec}; -use crate::repartition::distributor_channels::{ - DistributionReceiver, DistributionSender, -}; -use crate::repartition::RepartitionExecStateBuilder; use crate::sorts::streaming_merge::StreamingMergeBuilder; use crate::stream::RecordBatchStreamAdapter; use crate::{DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties, Statistics}; @@ -48,6 +45,8 @@ use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; use async_channel::{Receiver, Sender, TrySendError}; +use datafusion_common::utils::transpose; +use datafusion_common::DataFusionError; use datafusion_common::{internal_datafusion_err, internal_err, Result}; use datafusion_common_runtime::SpawnedTask; use datafusion_execution::memory_pool::MemoryConsumer; @@ -55,7 +54,7 @@ use datafusion_execution::TaskContext; use datafusion_common::HashMap; use futures::stream::Stream; -use futures::{ready, FutureExt, StreamExt, TryStreamExt}; +use futures::{ready, StreamExt, TryStreamExt}; use log::trace; use parking_lot::Mutex; @@ -97,13 +96,174 @@ type PartitionChannels = (Vec>, Vec>); /// │ (1) │─────────────────────┘ /// └─────────────────┘ Distribute data to the output partitions /// -/// ```text +/// ``` +type OnDemandDistributionSender = tokio::sync::mpsc::Sender; +type OnDemandDistributionReceiver = tokio::sync::mpsc::Receiver; + +type OnDemandInputPartitionsToCurrentPartitionSender = Vec; +type OnDemandInputPartitionsToCurrentPartitionReceiver = + Vec; +/// Inner state of [`OnDemandRepartitionExec`]. +#[derive(Debug)] +struct OnDemandRepartitionExecState { + /// Channels for sending batches from input partitions to output partitions. + /// Key is the partition number. + channels: HashMap< + usize, + ( + OnDemandInputPartitionsToCurrentPartitionSender, + OnDemandInputPartitionsToCurrentPartitionReceiver, + SharedMemoryReservation, + ), + >, + + /// Helper that ensures that that background job is killed once it is no longer needed. + abort_helper: Arc>>, +} + +/// create channels for sending batches from input partitions to output partitions for on-demand repartitioning. +fn create_on_demand_repartition_channels( + preserve_order: bool, + num_input_partitions: usize, + num_output_partitions: usize, +) -> ( + Vec, + Vec, +) { + if preserve_order { + let (txs, rxs) = on_demand_partition_aware_channels( + num_input_partitions, + num_output_partitions, + ); + // Take transpose of senders and receivers. `state.channels` keeps track of entries per output partition + let txs = transpose(txs); + let rxs = transpose(rxs); + (txs, rxs) + } else { + // create one channel per *output* partition + // note we use a custom channel that ensures there is always data for each receiver + // but limits the amount of buffering if required. + let (txs, rxs) = tokio_channels(num_output_partitions); + // Clone sender for each input partitions + let txs = txs + .into_iter() + .map(|item| vec![item; num_input_partitions]) + .collect::>(); + let rxs = rxs.into_iter().map(|item| vec![item]).collect::>(); + (txs, rxs) + } +} + +/// Create a hashmap of channels for sending batches from input partitions to output partitions. +fn create_on_demand_partition_channels_hashmap( + txs: Vec, + rxs: Vec, + name: String, + context: Arc, +) -> HashMap< + usize, + ( + OnDemandInputPartitionsToCurrentPartitionSender, + OnDemandInputPartitionsToCurrentPartitionReceiver, + SharedMemoryReservation, + ), +> { + let mut channels = HashMap::with_capacity(txs.len()); + + for (partition, (tx, rx)) in txs.into_iter().zip(rxs).enumerate() { + let reservation = Arc::new(Mutex::new( + MemoryConsumer::new(format!("{}[{partition}]", name)) + .register(context.memory_pool()), + )); + channels.insert(partition, (tx, rx, reservation)); + } + + channels +} + +impl OnDemandRepartitionExecState { + #[allow(clippy::too_many_arguments)] + fn new( + input: Arc, + partitioning: Partitioning, + metrics: ExecutionPlanMetricsSet, + preserve_order: bool, + name: String, + context: Arc, + partition_receivers: Vec>, + ) -> Self { + let num_input_partitions = input.output_partitioning().partition_count(); + let num_output_partitions = partitioning.partition_count(); + + let (txs, rxs) = create_on_demand_repartition_channels( + preserve_order, + num_input_partitions, + num_output_partitions, + ); + + let channels = create_on_demand_partition_channels_hashmap( + txs, + rxs, + name, + Arc::clone(&context), + ); + + // launch one async task per *input* partition + let mut spawned_tasks = Vec::with_capacity(num_input_partitions); + for i in 0..num_input_partitions { + let txs: HashMap<_, _> = channels + .iter() + .map(|(partition, (tx, _rx, reservation))| { + (*partition, (tx[i].clone(), Arc::clone(reservation))) + }) + .collect(); + + let input_task = { + let partition_rx = if preserve_order { + partition_receivers.clone()[i].clone() + } else { + partition_receivers.clone()[0].clone() + }; + let r_metrics = + OnDemandRepartitionMetrics::new(i, num_output_partitions, &metrics); + + SpawnedTask::spawn(OnDemandRepartitionExec::pull_from_input( + Arc::clone(&input), + i, + txs.clone(), + partitioning.clone(), + partition_rx, + r_metrics, + Arc::clone(&context), + )) + }; + + // In a separate task, wait for each input to be done + // (and pass along any errors, including panic!s) + + let wait_for_task = + SpawnedTask::spawn(OnDemandRepartitionExec::wait_for_task( + input_task, + txs.into_iter() + .map(|(partition, (tx, _reservation))| (partition, tx)) + .collect(), + )); + + spawned_tasks.push(wait_for_task); + } + Self { + channels, + abort_helper: Arc::new(spawned_tasks), + } + } +} #[derive(Debug, Clone)] pub struct OnDemandRepartitionExec { base: RepartitionExecBase, /// Channel to send partition number to the downstream task partition_channels: Arc>>, + state: Arc>>, } impl OnDemandRepartitionExec { @@ -224,7 +384,7 @@ impl ExecutionPlan for OnDemandRepartitionExec { partition ); - let lazy_state = Arc::clone(&self.base.state); + let lazy_state = Arc::clone(&self.state); let partition_channels = Arc::clone(&self.partition_channels); let input = Arc::clone(&self.base.input); let partitioning = self.partitioning().clone(); @@ -263,19 +423,15 @@ impl ExecutionPlan for OnDemandRepartitionExec { let state = lazy_state .get_or_init(|| async move { - Mutex::new( - RepartitionExecStateBuilder::new() - .enable_pull_based(true) - .partition_receivers(partition_rxs.clone()) - .build( - input_captured, - partitioning.clone(), - metrics_captured, - preserve_order, - name_captured, - context_captured, - ), - ) + Mutex::new(OnDemandRepartitionExecState::new( + input_captured, + partitioning.clone(), + metrics_captured, + preserve_order, + name_captured, + context_captured, + partition_rxs.clone(), + )) }) .await; @@ -408,12 +564,12 @@ impl OnDemandRepartitionExec { Ok(OnDemandRepartitionExec { base: RepartitionExecBase { input, - state: Default::default(), metrics: ExecutionPlanMetricsSet::new(), preserve_order, cache, }, partition_channels: Default::default(), + state: Default::default(), }) } // Executes the input plan and poll stream into the buffer, records fetch_time and buffer_time metrics @@ -468,15 +624,17 @@ impl OnDemandRepartitionExec { input_partition: usize, mut output_channels: HashMap< usize, - (DistributionSender, SharedMemoryReservation), + (OnDemandDistributionSender, SharedMemoryReservation), >, partitioning: Partitioning, output_partition_rx: Receiver, metrics: OnDemandRepartitionMetrics, context: Arc, ) -> Result<()> { + let num_output_partition = partitioning.partition_count(); // initialize buffer channel so that we can pre-fetch from input - let (buffer_tx, mut buffer_rx) = tokio::sync::mpsc::channel(2); + let (buffer_tx, mut buffer_rx) = + tokio::sync::mpsc::channel(num_output_partition * 2); // execute the child operator in a separate task // that pushes batches into buffer channel with limited capacity let processing_task = SpawnedTask::spawn(Self::process_input( @@ -556,6 +714,52 @@ impl OnDemandRepartitionExec { })??; Ok(()) } + + /// Waits for `input_task` which is consuming one of the inputs to + /// complete. Upon each successful completion, sends a `None` to + /// each of the output tx channels to signal one of the inputs is + /// complete. Upon error, propagates the errors to all output tx + /// channels. + async fn wait_for_task( + input_task: SpawnedTask>, + txs: HashMap, + ) { + // wait for completion, and propagate error + // note we ignore errors on send (.ok) as that means the receiver has already shutdown. + + match input_task.join().await { + // Error in joining task + Err(e) => { + let e = Arc::new(e); + + for (_, tx) in txs { + let err = Err(DataFusionError::Context( + "Join Error".to_string(), + Box::new(DataFusionError::External(Box::new(Arc::clone(&e)))), + )); + tx.send(Some(err)).await.ok(); + } + } + // Error from running input task + Ok(Err(e)) => { + // send the same Arc'd error to all output partitions + let e = Arc::new(e); + + for (_, tx) in txs { + // wrap it because need to send error to all output partitions + let err = Err(DataFusionError::from(&e)); + tx.send(Some(err)).await.ok(); + } + } + // Input task completed successfully + Ok(Ok(())) => { + // notify each output partition that this input partition has no more data + for (_, tx) in txs { + tx.send(None).await.ok(); + } + } + } + } } #[derive(Debug, Clone)] @@ -609,7 +813,7 @@ struct OnDemandPerPartitionStream { schema: SchemaRef, /// channel containing the repartitioned batches - receiver: DistributionReceiver, + receiver: OnDemandDistributionReceiver, /// Handle to ensure background tasks are killed when no longer needed. _drop_helper: Arc>>, @@ -647,7 +851,7 @@ impl Stream for OnDemandPerPartitionStream { } } - let result = ready!(self.receiver.recv().poll_unpin(cx)); + let result = ready!(self.receiver.poll_recv(cx)); // set is_requested to false, when receiving a batch self.is_requested = false; @@ -683,7 +887,7 @@ struct OnDemandRepartitionStream { schema: SchemaRef, /// channel containing the repartitioned batches - input: DistributionReceiver, + input: OnDemandDistributionReceiver, /// Handle to ensure background tasks are killed when no longer needed. _drop_helper: Arc>>, @@ -726,7 +930,7 @@ impl Stream for OnDemandRepartitionStream { } } - let result = ready!(self.input.recv().poll_unpin(cx)); + let result = ready!(self.input.poll_recv(cx)); // set is_requested to false, when receiving a batch self.is_requested = false; @@ -787,6 +991,7 @@ mod tests { use arrow::datatypes::{DataType, Field, Schema}; use datafusion_common::cast::as_string_array; use datafusion_common::{assert_batches_sorted_eq, exec_err}; + use futures::FutureExt; use tokio::task::JoinSet; use arrow_schema::SortOptions; From d68141919044c3ecbbf6f978202349da3b86a321 Mon Sep 17 00:00:00 2001 From: Weijun-H Date: Mon, 10 Feb 2025 16:28:38 +0800 Subject: [PATCH 22/25] chore: Used unbounded channel --- .../src/repartition/distributor_channels.rs | 14 ++++++++------ .../src/repartition/on_demand_repartition.rs | 16 +++++++--------- 2 files changed, 15 insertions(+), 15 deletions(-) diff --git a/datafusion/physical-plan/src/repartition/distributor_channels.rs b/datafusion/physical-plan/src/repartition/distributor_channels.rs index b12343940a2e2..b6dae8cd74154 100644 --- a/datafusion/physical-plan/src/repartition/distributor_channels.rs +++ b/datafusion/physical-plan/src/repartition/distributor_channels.rs @@ -82,11 +82,12 @@ pub fn channels( pub fn tokio_channels( n: usize, ) -> ( - Vec>, - Vec>, + Vec>, + Vec>, ) { - // only used for OnDemandRepartitionExec, so no need for unbounded capacity - let (senders, receivers) = (0..n).map(|_| tokio::sync::mpsc::channel(2)).unzip(); + let (senders, receivers) = (0..n) + .map(|_| tokio::sync::mpsc::unbounded_channel()) + .unzip(); (senders, receivers) } @@ -103,8 +104,9 @@ pub fn partition_aware_channels( (0..n_in).map(|_| channels(n_out)).unzip() } -type OnDemandPartitionAwareSenders = Vec>>; -type OnDemandPartitionAwareReceivers = Vec>>; +type OnDemandPartitionAwareSenders = Vec>>; +type OnDemandPartitionAwareReceivers = + Vec>>; pub fn on_demand_partition_aware_channels( n_in: usize, diff --git a/datafusion/physical-plan/src/repartition/on_demand_repartition.rs b/datafusion/physical-plan/src/repartition/on_demand_repartition.rs index 055c1d0537e00..ba0a033adc99f 100644 --- a/datafusion/physical-plan/src/repartition/on_demand_repartition.rs +++ b/datafusion/physical-plan/src/repartition/on_demand_repartition.rs @@ -97,8 +97,8 @@ type PartitionChannels = (Vec>, Vec>); /// └─────────────────┘ Distribute data to the output partitions /// /// ``` -type OnDemandDistributionSender = tokio::sync::mpsc::Sender; -type OnDemandDistributionReceiver = tokio::sync::mpsc::Receiver; +type OnDemandDistributionSender = tokio::sync::mpsc::UnboundedSender; +type OnDemandDistributionReceiver = tokio::sync::mpsc::UnboundedReceiver; type OnDemandInputPartitionsToCurrentPartitionSender = Vec; type OnDemandInputPartitionsToCurrentPartitionReceiver = @@ -631,10 +631,8 @@ impl OnDemandRepartitionExec { metrics: OnDemandRepartitionMetrics, context: Arc, ) -> Result<()> { - let num_output_partition = partitioning.partition_count(); // initialize buffer channel so that we can pre-fetch from input - let (buffer_tx, mut buffer_rx) = - tokio::sync::mpsc::channel(num_output_partition * 2); + let (buffer_tx, mut buffer_rx) = tokio::sync::mpsc::channel(2); // execute the child operator in a separate task // that pushes batches into buffer channel with limited capacity let processing_task = SpawnedTask::spawn(Self::process_input( @@ -683,7 +681,7 @@ impl OnDemandRepartitionExec { if let Some((tx, reservation)) = output_channels.get_mut(&partition) { reservation.lock().try_grow(size)?; - if tx.send(Some(Ok(batch))).await.is_err() { + if tx.send(Some(Ok(batch))).is_err() { // If the other end has hung up, it was an early shutdown (e.g. LIMIT) reservation.lock().shrink(size); output_channels.remove(&partition); @@ -737,7 +735,7 @@ impl OnDemandRepartitionExec { "Join Error".to_string(), Box::new(DataFusionError::External(Box::new(Arc::clone(&e)))), )); - tx.send(Some(err)).await.ok(); + tx.send(Some(err)).ok(); } } // Error from running input task @@ -748,14 +746,14 @@ impl OnDemandRepartitionExec { for (_, tx) in txs { // wrap it because need to send error to all output partitions let err = Err(DataFusionError::from(&e)); - tx.send(Some(err)).await.ok(); + tx.send(Some(err)).ok(); } } // Input task completed successfully Ok(Ok(())) => { // notify each output partition that this input partition has no more data for (_, tx) in txs { - tx.send(None).await.ok(); + tx.send(None).ok(); } } } From 4bcd51401403c72e9729380a5c7cfa155f562204 Mon Sep 17 00:00:00 2001 From: Weijun-H Date: Sat, 15 Feb 2025 13:45:19 +0800 Subject: [PATCH 23/25] refactor --- .../src/repartition/on_demand_repartition.rs | 19 +++---------------- 1 file changed, 3 insertions(+), 16 deletions(-) diff --git a/datafusion/physical-plan/src/repartition/on_demand_repartition.rs b/datafusion/physical-plan/src/repartition/on_demand_repartition.rs index ba0a033adc99f..f5e757ef13eff 100644 --- a/datafusion/physical-plan/src/repartition/on_demand_repartition.rs +++ b/datafusion/physical-plan/src/repartition/on_demand_repartition.rs @@ -656,22 +656,9 @@ impl OnDemandRepartitionExec { })?; // Fetch the batch from the buffer, ideally this should reduce the time gap between the requester and the input stream - let batch_opt = loop { - match buffer_rx.try_recv() { - Ok(batch) => break Some(batch), - Err(tokio::sync::mpsc::error::TryRecvError::Empty) => { - tokio::task::yield_now().await; - } - Err(tokio::sync::mpsc::error::TryRecvError::Disconnected) => { - break None - } - } - }; - - let batch = if let Some(batch) = batch_opt { - batch - } else { - break; + let batch = match buffer_rx.recv().await { + Some(batch) => batch, + None => break, }; let size = batch.get_array_memory_size(); From 714668414a6a6749d2090f7a692781ad424d515e Mon Sep 17 00:00:00 2001 From: Weijun-H Date: Sat, 15 Feb 2025 14:12:54 +0800 Subject: [PATCH 24/25] fix conflict --- Cargo.lock | 64 +++++++++---------- benchmarks/src/clickbench.rs | 4 ++ benchmarks/src/tpch/run.rs | 4 ++ .../physical-plan/src/repartition/mod.rs | 9 ++- 4 files changed, 46 insertions(+), 35 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 78e6ece14330c..896e924fe67f5 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -199,9 +199,9 @@ dependencies = [ [[package]] name = "anyhow" -version = "1.0.95" +version = "1.0.96" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "34ac096ce696dc2fcabef30516bb13c0a68a11d30131d3df6f04711467681b04" +checksum = "6b964d184e89d9b6b67dd2715bc8e74cf3107fb2b529990c90cf517326150bf4" [[package]] name = "apache-avro" @@ -1261,9 +1261,9 @@ checksum = "37b2a672a2cb129a2e41c10b1224bb368f9f37a2b16b612598138befd7b37eb5" [[package]] name = "cc" -version = "1.2.14" +version = "1.2.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "0c3d1b2e905a3a7b00a6141adb0e4c0bb941d11caf55349d863942a1cc44e3c9" +checksum = "c736e259eea577f443d5c86c304f9f4ae0295c43f3ba05c21f1d66b5f06001af" dependencies = [ "jobserver", "libc", @@ -3712,7 +3712,7 @@ checksum = "c0ff37bd590ca25063e35af745c343cb7a0271906fb7b37e4813e8f79f00268d" dependencies = [ "bitflags 2.8.0", "libc", - "redox_syscall 0.5.8", + "redox_syscall 0.5.9", ] [[package]] @@ -3751,9 +3751,9 @@ dependencies = [ [[package]] name = "log" -version = "0.4.25" +version = "0.4.26" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "04cbf5b083de1c7e0222a7a51dbfdba1cbe1c6ab0b15e29fff3f6c077fd9cd9f" +checksum = "30bde2b3dc3671ae49d8e2e9f044c7c005836e7a023ee57cffa25ab82764bb9e" [[package]] name = "lz4_flex" @@ -3842,9 +3842,9 @@ dependencies = [ [[package]] name = "miniz_oxide" -version = "0.8.4" +version = "0.8.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b3b1c9bd4fe1f0f8b387f6eb9eb3b4a1aa26185e5750efb9140301703f62cd1b" +checksum = "8e3e04debbb59698c15bacbb6d93584a8c0ca9cc3213cb423d31f760d8843ce5" dependencies = [ "adler2", ] @@ -4098,7 +4098,7 @@ checksum = "1e401f977ab385c9e4e3ab30627d6f26d00e2c73eef317493c4ec6d468726cf8" dependencies = [ "cfg-if", "libc", - "redox_syscall 0.5.8", + "redox_syscall 0.5.9", "smallvec", "windows-targets 0.52.6", ] @@ -4748,7 +4748,7 @@ checksum = "3779b94aeb87e8bd4e834cee3650289ee9e0d5677f976ecdb6d219e5f4f6cd94" dependencies = [ "rand_chacha 0.9.0", "rand_core 0.9.1", - "zerocopy 0.8.18", + "zerocopy 0.8.20", ] [[package]] @@ -4787,7 +4787,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "a88e0da7a2c97baa202165137c158d0a2e824ac465d13d81046727b34cb247d3" dependencies = [ "getrandom 0.3.1", - "zerocopy 0.8.18", + "zerocopy 0.8.20", ] [[package]] @@ -4851,9 +4851,9 @@ dependencies = [ [[package]] name = "redox_syscall" -version = "0.5.8" +version = "0.5.9" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "03a862b389f93e68874fbf580b9de08dd02facb9a788ebadaf4a3fd33cf58834" +checksum = "82b568323e98e49e2a0899dcee453dd679fae22d69adf9b11dd508d1549b7e2f" dependencies = [ "bitflags 2.8.0", ] @@ -4986,9 +4986,9 @@ dependencies = [ [[package]] name = "ring" -version = "0.17.9" +version = "0.17.10" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e75ec5e92c4d8aede845126adc388046234541629e76029599ed35a003c7ed24" +checksum = "d34b5020fcdea098ef7d95e9f89ec15952123a4a039badd09fabebe9e963e839" dependencies = [ "cc", "cfg-if", @@ -5671,9 +5671,9 @@ checksum = "a8f112729512f8e442d81f95a8a7ddf2b7c6b8a1a6f509a95864142b30cab2d3" [[package]] name = "stacker" -version = "0.1.18" +version = "0.1.19" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1d08feb8f695b465baed819b03c128dc23f57a694510ab1f06c77f763975685e" +checksum = "d9156ebd5870ef293bfb43f91c7a74528d363ec0d424afe24160ed5a4343d08a" dependencies = [ "cc", "cfg-if", @@ -6464,9 +6464,9 @@ checksum = "5c1cb5db39152898a79168971543b1cb5020dff7fe43c8dc468b0885f5e29df5" [[package]] name = "unicode-ident" -version = "1.0.16" +version = "1.0.17" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a210d160f08b701c8721ba1c726c11662f877ea6b7094007e1ca9a1041945034" +checksum = "00e2473a93778eb0bad35909dff6a10d28e63f792f16ed15e404fca9d5eeedbe" [[package]] name = "unicode-normalization" @@ -6557,9 +6557,9 @@ checksum = "06abde3611657adf66d383f00b093d7faecc7fa57071cce2578660c9f1010821" [[package]] name = "uuid" -version = "1.13.2" +version = "1.14.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8c1f41ffb7cf259f1ecc2876861a17e7142e63ead296f671f81f6ae85903e0d6" +checksum = "93d59ca99a559661b96bf898d8fce28ed87935fd2bea9f05983c1464dd6c71b1" dependencies = [ "getrandom 0.3.1", "js-sys", @@ -6762,7 +6762,7 @@ version = "1.5.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "372d5b87f58ec45c384ba03563b03544dc5fadc3983e434b286913f5b4a9bb6d" dependencies = [ - "redox_syscall 0.5.8", + "redox_syscall 0.5.9", "wasite", "web-sys", ] @@ -7040,9 +7040,9 @@ checksum = "589f6da84c646204747d1270a2a5661ea66ed1cced2631d546fdfb155959f9ec" [[package]] name = "winnow" -version = "0.7.2" +version = "0.7.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "59690dea168f2198d1a3b0cac23b8063efcd11012f10ae4698f284808c8ef603" +checksum = "0e7f4ea97f6f78012141bcdb6a216b2609f0979ada50b20ca5b52dde2eac2bb1" dependencies = [ "memchr", ] @@ -7139,11 +7139,11 @@ dependencies = [ [[package]] name = "zerocopy" -version = "0.8.18" +version = "0.8.20" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "79386d31a42a4996e3336b0919ddb90f81112af416270cff95b5f5af22b839c2" +checksum = "dde3bb8c68a8f3f1ed4ac9221aad6b10cece3e60a8e2ea54a6a2dec806d0084c" dependencies = [ - "zerocopy-derive 0.8.18", + "zerocopy-derive 0.8.20", ] [[package]] @@ -7159,9 +7159,9 @@ dependencies = [ [[package]] name = "zerocopy-derive" -version = "0.8.18" +version = "0.8.20" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "76331675d372f91bf8d17e13afbd5fe639200b73d01f0fc748bb059f9cca2db7" +checksum = "eea57037071898bf96a6da35fd626f4f27e9cee3ead2a6c703cf09d472b2e700" dependencies = [ "proc-macro2", "quote", @@ -7219,9 +7219,9 @@ dependencies = [ [[package]] name = "zstd" -version = "0.13.2" +version = "0.13.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "fcf2b778a664581e31e389454a7072dab1647606d44f7feea22cd5abb9c9f3f9" +checksum = "e91ee311a569c327171651566e07972200e76fcfe2242a4fa446149a3881c08a" dependencies = [ "zstd-safe", ] diff --git a/benchmarks/src/clickbench.rs b/benchmarks/src/clickbench.rs index 4a9ec7714fe48..5ffda65fd7928 100644 --- a/benchmarks/src/clickbench.rs +++ b/benchmarks/src/clickbench.rs @@ -132,6 +132,10 @@ impl RunOpt { parquet_options.binary_as_string = true; } + config + .options_mut() + .optimizer + .prefer_round_robin_repartition = self.prefer_round_robin; let rt_builder = self.common.runtime_env_builder()?; let ctx = SessionContext::new_with_config_rt(config, rt_builder.build_arc()?); self.register_hits(&ctx).await?; diff --git a/benchmarks/src/tpch/run.rs b/benchmarks/src/tpch/run.rs index 48a54fb93618b..40dfab8d05258 100644 --- a/benchmarks/src/tpch/run.rs +++ b/benchmarks/src/tpch/run.rs @@ -126,6 +126,10 @@ impl RunOpt { .config() .with_collect_statistics(!self.disable_statistics); config.options_mut().optimizer.prefer_hash_join = self.prefer_hash_join; + config + .options_mut() + .optimizer + .prefer_round_robin_repartition = self.prefer_round_robin; let rt_builder = self.common.runtime_env_builder()?; let ctx = SessionContext::new_with_config_rt(config, rt_builder.build_arc()?); diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 63c4e4bf01e1b..c4aa3da9bb537 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -1129,7 +1129,6 @@ mod tests { use crate::{ collect, expressions::col, - memory::MemorySourceConfig, test::{ assert_is_pending, exec::{ @@ -1137,7 +1136,6 @@ mod tests { ErrorExec, MockExec, }, }, - {collect, expressions::col}, }; use arrow::array::{ArrayRef, StringArray, UInt32Array}; @@ -1672,12 +1670,13 @@ mod tests { #[cfg(test)] mod test { use arrow::compute::SortOptions; - use arrow::datatypes::Schema; + use arrow::datatypes::{DataType, Field, Schema}; use super::*; use crate::test::TestMemoryExec; use crate::union::UnionExec; + use datafusion_physical_expr::expressions::col; use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; /// Asserts that the plan is as expected @@ -1767,6 +1766,10 @@ mod test { Ok(()) } + fn test_schema() -> Arc { + Arc::new(Schema::new(vec![Field::new("c0", DataType::UInt32, false)])) + } + fn sort_exprs(schema: &Schema) -> LexOrdering { let options = SortOptions::default(); LexOrdering::new(vec![PhysicalSortExpr { From 3fa3e8b4da066d4cda60d3fbbd541a46b6d67b8b Mon Sep 17 00:00:00 2001 From: Weijun-H Date: Sat, 22 Feb 2025 18:22:30 +0800 Subject: [PATCH 25/25] chore: Clean up --- .../enforce_distribution.rs | 1 - .../src/enforce_distribution.rs | 3 +- .../src/repartition/distributor_channels.rs | 5 +- .../src/repartition/on_demand_repartition.rs | 118 +++++++++--------- datafusion/proto/src/logical_plan/mod.rs | 1 - .../test_files/on_demand_repartition.slt | 2 +- 6 files changed, 63 insertions(+), 67 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index 46daf54c0af02..66d1380e09c38 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -58,7 +58,6 @@ use datafusion_physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; use datafusion_physical_plan::projection::ProjectionExec; use datafusion_physical_plan::sorts::sort::SortExec; use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; -use datafusion_physical_plan::source::DataSourceExec; use datafusion_physical_plan::union::UnionExec; use datafusion_physical_plan::ExecutionPlanProperties; use datafusion_physical_plan::PlanProperties; diff --git a/datafusion/physical-optimizer/src/enforce_distribution.rs b/datafusion/physical-optimizer/src/enforce_distribution.rs index 8d4ed73081b5a..b8dda232b4c0c 100644 --- a/datafusion/physical-optimizer/src/enforce_distribution.rs +++ b/datafusion/physical-optimizer/src/enforce_distribution.rs @@ -1096,6 +1096,7 @@ fn replace_order_preserving_variants( context.update_plan_from_children() } +/// Replace the round robin repartition with on-demand repartition when prefer_round_robin_repartition is set to false. fn replace_round_robin_repartition_with_on_demand( mut context: DistributionContext, ) -> Result { @@ -1413,6 +1414,7 @@ pub fn ensure_distribution( } } } + // when prefer_round_robin_repartition is set to false, replace round robin repartition with on-demand repartition if !prefer_round_robin_repartition { child = replace_round_robin_repartition_with_on_demand(child)?; } @@ -1501,6 +1503,5 @@ fn update_children(mut dist_context: DistributionContext) -> Result( (senders, receivers) } -pub fn tokio_channels( +/// Create `n` empty mpsc channels with unbounded capacity. +pub fn unbounded_channels( n: usize, ) -> ( Vec>, @@ -115,7 +116,7 @@ pub fn on_demand_partition_aware_channels( OnDemandPartitionAwareSenders, OnDemandPartitionAwareReceivers, ) { - (0..n_in).map(|_| tokio_channels(n_out)).unzip() + (0..n_in).map(|_| unbounded_channels(n_out)).unzip() } /// Erroring during [send](DistributionSender::send). diff --git a/datafusion/physical-plan/src/repartition/on_demand_repartition.rs b/datafusion/physical-plan/src/repartition/on_demand_repartition.rs index f5e757ef13eff..03f73965da898 100644 --- a/datafusion/physical-plan/src/repartition/on_demand_repartition.rs +++ b/datafusion/physical-plan/src/repartition/on_demand_repartition.rs @@ -27,7 +27,9 @@ use std::sync::Arc; use std::task::{Context, Poll}; use std::{any::Any, vec}; -use super::distributor_channels::{on_demand_partition_aware_channels, tokio_channels}; +use super::distributor_channels::{ + on_demand_partition_aware_channels, unbounded_channels, +}; use super::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use super::{ DisplayAs, ExecutionPlanProperties, MaybeBatch, RecordBatchStream, @@ -58,51 +60,17 @@ use futures::{ready, StreamExt, TryStreamExt}; use log::trace; use parking_lot::Mutex; +/// Channels for sending output partition number to the operator. type PartitionChannels = (Vec>, Vec>); -/// The OnDemandRepartitionExec operator repartitions the input data based on a push-based model. -/// It is similar to the RepartitionExec operator, but it doesn't distribute the data to the output -/// partitions until the output partitions request the data. -/// -/// When polling, the operator sends the output partition number to the one partition channel, then the prefetch buffer will distribute the data based on the order of the partition number. -/// Each input steams has a prefetch buffer(channel) to distribute the data to the output partitions. -/// -/// The following diagram illustrates the data flow of the OnDemandRepartitionExec operator with 3 output partitions for the input stream 1: -/// ```text -/// /\ /\ /\ -/// ││ ││ ││ -/// ││ ││ ││ -/// ││ ││ ││ -/// ┌───────┴┴────────┐ ┌───────┴┴────────┐ ┌───────┴┴────────┐ -/// │ Stream │ │ Stream │ │ Stream │ -/// │ (1) │ │ (2) │ │ (3) │ -/// └────────┬────────┘ └───────┬─────────┘ └────────┬────────┘ -/// │ │ │ / \ -/// │ │ │ | | -/// │ │ │ | | -/// └────────────────┐ │ ┌──────────────────┘ | | -/// │ │ │ | | -/// ▼ ▼ ▼ | | -/// ┌─────────────────┐ | | -/// Send the partition │ partion channel │ | | -/// number when polling │ │ | | -/// └────────┬────────┘ | | -/// │ | | -/// │ | | -/// │ Get the partition number | | -/// ▼ then send data | | -/// ┌─────────────────┐ | | -/// │ Prefetch Buffer │───────────────────┘ | -/// │ (1) │─────────────────────┘ -/// └─────────────────┘ Distribute data to the output partitions -/// -/// ``` +/// The OnDemandRepartitionExec operator cannot use the custom DistributionSender and DistributionReceiver because it can prevent channels from filling up endlessly. type OnDemandDistributionSender = tokio::sync::mpsc::UnboundedSender; type OnDemandDistributionReceiver = tokio::sync::mpsc::UnboundedReceiver; type OnDemandInputPartitionsToCurrentPartitionSender = Vec; type OnDemandInputPartitionsToCurrentPartitionReceiver = Vec; + /// Inner state of [`OnDemandRepartitionExec`]. #[derive(Debug)] struct OnDemandRepartitionExecState { @@ -143,7 +111,7 @@ fn create_on_demand_repartition_channels( // create one channel per *output* partition // note we use a custom channel that ensures there is always data for each receiver // but limits the amount of buffering if required. - let (txs, rxs) = tokio_channels(num_output_partitions); + let (txs, rxs) = unbounded_channels(num_output_partitions); // Clone sender for each input partitions let txs = txs .into_iter() @@ -258,6 +226,43 @@ impl OnDemandRepartitionExecState { } } +/// The OnDemandRepartitionExec operator repartitions the input data based on a push-based model. +/// It is similar to the RepartitionExec operator, but it doesn't distribute the data to the output +/// partitions until the output partitions request the data. +/// +/// When polling, the operator sends the output partition number to the one partition channel, then the prefetch buffer will distribute the data based on the order of the partition number. +/// Each input steams has a prefetch buffer(channel) to distribute the data to the output partitions. +/// +/// The following diagram illustrates the data flow of the OnDemandRepartitionExec operator with 3 output partitions for the input stream 1: +/// ```text +/// /\ /\ /\ +/// ││ ││ ││ +/// ││ ││ ││ +/// ││ ││ ││ +/// ┌───────┴┴────────┐ ┌───────┴┴────────┐ ┌───────┴┴────────┐ +/// │ Stream │ │ Stream │ │ Stream │ +/// │ (1) │ │ (2) │ │ (3) │ +/// └────────┬────────┘ └───────┬─────────┘ └────────┬────────┘ +/// │ │ │ / \ +/// │ │ │ | | +/// │ │ │ | | +/// └────────────────┐ │ ┌──────────────────┘ | | +/// │ │ │ | | +/// ▼ ▼ ▼ | | +/// ┌─────────────────┐ | | +/// Send the partition │ partion channel │ | | +/// number when polling │ │ | | +/// └────────┬────────┘ | | +/// │ | | +/// │ | | +/// │ Get the partition number | | +/// ▼ then send data | | +/// ┌─────────────────┐ | | +/// │ Prefetch Buffer │───────────────────┘ | +/// │ (1) │─────────────────────┘ +/// └─────────────────┘ Distribute data to the output partitions +/// +/// ``` #[derive(Debug, Clone)] pub struct OnDemandRepartitionExec { base: RepartitionExecBase, @@ -961,14 +966,13 @@ mod tests { use super::*; use crate::{ collect, - memory::MemorySourceConfig, - source::DataSourceExec, test::{ assert_is_pending, exec::{ assert_strong_count_converges_to_zero, BarrierExec, BlockingExec, ErrorExec, MockExec, }, + TestMemoryExec, }, }; @@ -1018,11 +1022,8 @@ mod tests { ) -> Result>> { let task_ctx = Arc::new(TaskContext::default()); // create physical plan - let exec = MemorySourceConfig::try_new_exec( - &input_partitions, - Arc::clone(schema), - None, - )?; + let exec = + TestMemoryExec::try_new_exec(&input_partitions, Arc::clone(schema), None)?; let exec = OnDemandRepartitionExec::try_new(exec, partitioning)?; // execute and collect results @@ -1082,8 +1083,7 @@ mod tests { let schema = test_schema(); let partition: Vec = create_vec_batches(2); let partitions = vec![partition.clone(), partition.clone()]; - let input = - MemorySourceConfig::try_new_exec(&partitions, Arc::clone(&schema), None)?; + let input = TestMemoryExec::try_new_exec(&partitions, Arc::clone(&schema), None)?; let exec = OnDemandRepartitionExec::try_new(input, Partitioning::OnDemand(3)).unwrap(); @@ -1452,19 +1452,15 @@ mod tests { Arc::new(UInt32Array::from(vec![1, 2, 3, 4])) as ArrayRef, )])?; - let source = Arc::new(DataSourceExec::new(Arc::new( - MemorySourceConfig::try_new( - &[vec![batch.clone()]], - Arc::clone(&schema), - None, - ) - .unwrap() - .try_with_sort_information(vec![sort_exprs]) - .unwrap(), + let source = Arc::new(TestMemoryExec::update_cache(Arc::new( + TestMemoryExec::try_new(&[vec![batch.clone()]], Arc::clone(&schema), None) + .unwrap() + .try_with_sort_information(vec![sort_exprs]) + .unwrap(), ))); // output has multiple partitions, and is sorted - let union = UnionExec::new(vec![Arc::::clone(&source), source]); + let union = UnionExec::new(vec![Arc::::clone(&source), source]); let repartition_exec = OnDemandRepartitionExec::try_new(Arc::new(union), Partitioning::OnDemand(5)) .unwrap() @@ -1559,15 +1555,15 @@ mod tests { } fn memory_exec(schema: &SchemaRef) -> Arc { - MemorySourceConfig::try_new_exec(&[vec![]], Arc::clone(schema), None).unwrap() + TestMemoryExec::try_new_exec(&[vec![]], Arc::clone(schema), None).unwrap() } fn sorted_memory_exec( schema: &SchemaRef, sort_exprs: LexOrdering, ) -> Arc { - Arc::new(DataSourceExec::new(Arc::new( - MemorySourceConfig::try_new(&[vec![]], Arc::clone(schema), None) + Arc::new(TestMemoryExec::update_cache(Arc::new( + TestMemoryExec::try_new(&[vec![]], Arc::clone(schema), None) .unwrap() .try_with_sort_information(vec![sort_exprs]) .unwrap(), diff --git a/datafusion/proto/src/logical_plan/mod.rs b/datafusion/proto/src/logical_plan/mod.rs index a0217cc95612f..641dfe7b5fb84 100644 --- a/datafusion/proto/src/logical_plan/mod.rs +++ b/datafusion/proto/src/logical_plan/mod.rs @@ -1417,7 +1417,6 @@ impl AsLogicalPlan for LogicalPlanNode { Partitioning::RoundRobinBatch(partition_count) => { PartitionMethod::RoundRobin(*partition_count as u64) } - Partitioning::DistributeBy(_) => { return not_impl_err!("DistributeBy") } diff --git a/datafusion/sqllogictest/test_files/on_demand_repartition.slt b/datafusion/sqllogictest/test_files/on_demand_repartition.slt index afc92eac83d1a..ece13aaaba6a4 100644 --- a/datafusion/sqllogictest/test_files/on_demand_repartition.slt +++ b/datafusion/sqllogictest/test_files/on_demand_repartition.slt @@ -426,7 +426,7 @@ EXPLAIN SELECT ---- logical_plan 01)Projection: sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING, count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING -02)--WindowAggr: windowExpr=[[count(Int64(1)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING AS count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING]] +02)--WindowAggr: windowExpr=[[count(*) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING]] 03)----Projection: aggregate_test_100.c1, aggregate_test_100.c2, sum(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING 04)------WindowAggr: windowExpr=[[sum(CAST(aggregate_test_100.c4 AS Int64)) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING]] 05)--------TableScan: aggregate_test_100 projection=[c1, c2, c4]