From af38dad5b347fbb7d0c7ebd9353df0e9c94f780c Mon Sep 17 00:00:00 2001 From: Weijun-H Date: Sat, 14 Dec 2024 16:31:37 +0800 Subject: [PATCH 01/20] draft --- datafusion/physical-plan/src/sorts/merge.rs | 4 ++ .../src/sorts/sort_preserving_merge.rs | 41 ++++++++++++++----- .../src/sorts/streaming_merge.rs | 25 ++++++++--- 3 files changed, 54 insertions(+), 16 deletions(-) diff --git a/datafusion/physical-plan/src/sorts/merge.rs b/datafusion/physical-plan/src/sorts/merge.rs index 458c1c29c0cf..19b387ea6e22 100644 --- a/datafusion/physical-plan/src/sorts/merge.rs +++ b/datafusion/physical-plan/src/sorts/merge.rs @@ -142,6 +142,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 { @@ -153,6 +155,7 @@ impl SortPreservingMergeStream { fetch: Option, reservation: MemoryReservation, enable_round_robin_tie_breaker: bool, + enable_pull_based_execution: bool, ) -> Self { let stream_count = streams.partitions(); @@ -174,6 +177,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 906164f21b8c..383733bc2f35 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -35,6 +35,7 @@ use datafusion_execution::TaskContext; use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; use log::{debug, trace}; +use tokio::stream; /// Sort preserving merge execution plan /// @@ -82,6 +83,7 @@ pub struct SortPreservingMergeExec { cache: PlanProperties, /// Configuration parameter to enable round-robin selection of tied winners of loser tree. enable_round_robin_repartition: bool, + enable_pull_based_execution: bool, } impl SortPreservingMergeExec { @@ -95,6 +97,7 @@ impl SortPreservingMergeExec { fetch: None, cache, enable_round_robin_repartition: true, + enable_pull_based_execution: true, } } @@ -190,6 +193,7 @@ impl ExecutionPlan for SortPreservingMergeExec { fetch: limit, cache: self.cache.clone(), enable_round_robin_repartition: true, + enable_pull_based_execution: true, })) } @@ -271,18 +275,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)) @@ -290,6 +310,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 2178cc012a10..b17c2bf28618 100644 --- a/datafusion/physical-plan/src/sorts/streaming_merge.rs +++ b/datafusion/physical-plan/src/sorts/streaming_merge.rs @@ -37,7 +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) => {{ + ($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) => {{ let streams = FieldCursorStream::<$t>::new($sort, $streams); return Ok(Box::pin(SortPreservingMergeStream::new( Box::new(streams), @@ -47,6 +47,7 @@ macro_rules! merge_helper { $fetch, $reservation, $enable_round_robin_tie_breaker, + $enable_pull_based_execution, ))); }}; } @@ -60,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<'_> { @@ -73,6 +75,7 @@ impl Default for StreamingMergeBuilder<'_> { fetch: None, reservation: None, enable_round_robin_tie_breaker: false, + enable_pull_based_execution: false, } } } @@ -128,6 +131,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, @@ -138,6 +149,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 @@ -170,11 +182,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) _ => {} } } @@ -193,6 +205,7 @@ impl<'a> StreamingMergeBuilder<'a> { fetch, reservation, enable_round_robin_tie_breaker, + enable_pull_based_execution, ))) } } From 5d33aa586d58f96dec40946800e2bdbe11dc765e Mon Sep 17 00:00:00 2001 From: Weijun-H Date: Sat, 14 Dec 2024 17:07:45 +0800 Subject: [PATCH 02/20] clean up --- datafusion/physical-plan/src/sorts/merge.rs | 4 --- .../src/sorts/sort_preserving_merge.rs | 2 -- .../src/sorts/streaming_merge.rs | 25 +++++-------------- 3 files changed, 6 insertions(+), 25 deletions(-) diff --git a/datafusion/physical-plan/src/sorts/merge.rs b/datafusion/physical-plan/src/sorts/merge.rs index 19b387ea6e22..458c1c29c0cf 100644 --- a/datafusion/physical-plan/src/sorts/merge.rs +++ b/datafusion/physical-plan/src/sorts/merge.rs @@ -142,8 +142,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 { @@ -155,7 +153,6 @@ impl SortPreservingMergeStream { fetch: Option, reservation: MemoryReservation, enable_round_robin_tie_breaker: bool, - enable_pull_based_execution: bool, ) -> Self { let stream_count = streams.partitions(); @@ -177,7 +174,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 383733bc2f35..5639f0ccf2a8 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -35,7 +35,6 @@ use datafusion_execution::TaskContext; use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; use log::{debug, trace}; -use tokio::stream; /// Sort preserving merge execution plan /// @@ -310,7 +309,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 b17c2bf28618..2b36a3aec5fd 100644 --- a/datafusion/physical-plan/src/sorts/streaming_merge.rs +++ b/datafusion/physical-plan/src/sorts/streaming_merge.rs @@ -37,7 +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, $enable_pull_based_execution:ident) => {{ + ($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 = FieldCursorStream::<$t>::new($sort, $streams); return Ok(Box::pin(SortPreservingMergeStream::new( Box::new(streams), @@ -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, } } } @@ -131,14 +128,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, @@ -149,7 +138,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 @@ -182,11 +170,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) _ => {} } } @@ -205,7 +193,6 @@ impl<'a> StreamingMergeBuilder<'a> { fetch, reservation, enable_round_robin_tie_breaker, - enable_pull_based_execution, ))) } } From d4e830e5c6993597bf3a2b99aa7d89a284fab09e Mon Sep 17 00:00:00 2001 From: Weijun-H Date: Sun, 15 Dec 2024 16:55:11 +0800 Subject: [PATCH 03/20] 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 2b36a3aec5fd..2178cc012a10 100644 --- a/datafusion/physical-plan/src/sorts/streaming_merge.rs +++ b/datafusion/physical-plan/src/sorts/streaming_merge.rs @@ -171,7 +171,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 ca61c3a81c8e50ee59251df8ab21c2fef7e3fbd3 Mon Sep 17 00:00:00 2001 From: Weijun-H Date: Tue, 24 Dec 2024 10:45:38 +0200 Subject: [PATCH 04/20] 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 0a80dcd34e05..730875109a93 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 000000000000..354e82d92601 --- /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 fb86ebec8e19af264e6b5b8a759b29797498988f Mon Sep 17 00:00:00 2001 From: Weijun-H Date: Thu, 26 Dec 2024 21:40:31 +0200 Subject: [PATCH 05/20] 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 98c0c864b9f7..f5d0817234d3 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 bb0e21fdfd15..b6a8a4a95b51 100644 --- a/datafusion/physical-plan/Cargo.toml +++ b/datafusion/physical-plan/Cargo.toml @@ -61,6 +61,7 @@ indexmap = { workspace = true } itertools = { workspace = true, features = ["use_std"] } log = { workspace = true } once_cell = "1.18.0" +async-channel = "2.3.1" parking_lot = { workspace = true } pin-project-lite = "^0.2.7" rand = { workspace = true } diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 730875109a93..f4b3e228c6fc 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 354e82d92601..7595facdb36f 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 b3bd61c1a19e7b20a30bf07b89cbe0935a9f2029 Mon Sep 17 00:00:00 2001 From: Weijun-H Date: Sun, 29 Dec 2024 16:03:05 +0100 Subject: [PATCH 06/20] bench: add config to avoid OnDemandRepartitionExec instead of Round-Robin RepartitionExec --- benchmarks/src/tpch/run.rs | 2 + datafusion/common/src/config.rs | 2 + .../enforce_distribution.rs | 59 +++++++++++++++++++ .../physical-plan/src/repartition/mod.rs | 2 +- 4 files changed, 64 insertions(+), 1 deletion(-) diff --git a/benchmarks/src/tpch/run.rs b/benchmarks/src/tpch/run.rs index de3ee3d67db2..f2e4e70e87aa 100644 --- a/benchmarks/src/tpch/run.rs +++ b/benchmarks/src/tpch/run.rs @@ -121,6 +121,8 @@ impl RunOpt { .config() .with_collect_statistics(!self.disable_statistics); config.options_mut().optimizer.prefer_hash_join = self.prefer_hash_join; + config.options_mut().optimizer.enable_on_demand_repartition = true; + config.options_mut().optimizer.enable_round_robin_repartition = false; let ctx = SessionContext::new_with_config(config); // register tables diff --git a/datafusion/common/src/config.rs b/datafusion/common/src/config.rs index 3a07a238a4c9..6449e600f1ff 100644 --- a/datafusion/common/src/config.rs +++ b/datafusion/common/src/config.rs @@ -532,6 +532,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/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index 27323eaedccc..d44c17e426ac 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -54,6 +54,7 @@ use datafusion_physical_expr::{ }; use datafusion_physical_optimizer::output_requirements::OutputRequirementExec; use datafusion_physical_optimizer::PhysicalOptimizerRule; +use datafusion_physical_plan::repartition::on_demand_repartition::OnDemandRepartitionExec; use datafusion_physical_plan::windows::{get_best_fitting_window, BoundedWindowAggExec}; use datafusion_physical_plan::ExecutionPlanProperties; @@ -404,6 +405,10 @@ 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::() @@ -857,6 +862,32 @@ fn add_roundrobin_on_top( } } +fn add_on_demand_repartition_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. @@ -1035,6 +1066,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() @@ -1156,6 +1199,7 @@ 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; + let enable_on_demand_repartition = config.optimizer.enable_on_demand_repartition; let repartition_file_scans = config.optimizer.repartition_file_scans; let batch_size = config.execution.batch_size; let should_use_estimates = config @@ -1251,6 +1295,10 @@ fn ensure_distribution( child = add_hash_on_top(child, exprs.to_vec(), target_partitions)?; } + if enable_on_demand_repartition { + child = + add_on_demand_repartition_on_top(child, target_partitions)?; + } } Distribution::UnspecifiedDistribution => { if add_roundrobin { @@ -1258,6 +1306,10 @@ fn ensure_distribution( // to increase parallelism. child = add_roundrobin_on_top(child, target_partitions)?; } + if enable_on_demand_repartition { + child = + add_on_demand_repartition_on_top(child, target_partitions)?; + } } }; @@ -1365,6 +1417,13 @@ fn update_children(mut dist_context: DistributionContext) -> Result() + { + !matches!( + repartition.partitioning(), + Partitioning::UnknownPartitioning(_) + ) } else { child_plan_any.is::() || child_plan_any.is::() diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index f4b3e228c6fc..97a30d6ebf0d 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 c950bb53c13bdfca653ce0f0c159107a86a76327 Mon Sep 17 00:00:00 2001 From: Weijun-H Date: Mon, 30 Dec 2024 23:58:04 +0100 Subject: [PATCH 07/20] refactor: Use common structure to avoid duplication --- .../physical-plan/src/repartition/mod.rs | 434 ++++++++++++------ .../src/repartition/on_demand_repartition.rs | 321 +++---------- 2 files changed, 357 insertions(+), 398 deletions(-) diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 97a30d6ebf0d..45c5058b3c08 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -43,6 +43,7 @@ use arrow::compute::take_arrays; use arrow::datatypes::{SchemaRef, UInt32Type}; use arrow::record_batch::RecordBatch; use arrow_array::{PrimitiveArray, RecordBatchOptions}; +use async_channel::Receiver; use datafusion_common::utils::transpose; use datafusion_common::{not_impl_err, DataFusionError, Result}; use datafusion_common_runtime::SpawnedTask; @@ -55,6 +56,7 @@ use datafusion_common::HashMap; 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,93 @@ 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 { + // 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`. + 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. /// @@ -412,21 +605,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 @@ -473,18 +657,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 @@ -506,14 +702,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(()) @@ -533,11 +729,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( @@ -548,7 +744,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)) @@ -559,7 +755,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( @@ -573,17 +772,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(); @@ -594,9 +793,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, @@ -673,11 +872,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 { @@ -694,87 +893,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 { - // 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 /// diff --git a/datafusion/physical-plan/src/repartition/on_demand_repartition.rs b/datafusion/physical-plan/src/repartition/on_demand_repartition.rs index 7595facdb36f..a2637a035ad2 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 011cff367bb2e986d6155bdd3da961b9175899a9 Mon Sep 17 00:00:00 2001 From: Weijun-H Date: Tue, 31 Dec 2024 00:09:57 +0100 Subject: [PATCH 08/20] 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 45c5058b3c08..86bf26445475 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_common::sort_expr::LexOrdering; use datafusion_common::HashMap; 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; @@ -1065,8 +1065,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() @@ -1075,7 +1075,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 { @@ -1086,12 +1086,9 @@ impl Stream for RepartitionStream { continue; } } - Poll::Ready(None) => { + None => { return Poll::Ready(None); } - Poll::Pending => { - return Poll::Pending; - } } } } @@ -1127,21 +1124,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 a2637a035ad2..10540a93d5ac 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 8cd65c7b3f0e7ea57ad9205f3c92ef749b9e9923 Mon Sep 17 00:00:00 2001 From: Weijun-H Date: Mon, 13 Jan 2025 22:10:44 +0800 Subject: [PATCH 09/20] fix: Avoid comsuing all data in one poll --- datafusion/physical-plan/src/memory.rs | 5 ++ .../physical-plan/src/repartition/mod.rs | 43 ++++++++++---- .../src/repartition/on_demand_repartition.rs | 59 ++++++++++++++----- datafusion/physical-plan/src/stream.rs | 4 ++ datafusion/proto/proto/datafusion.proto | 1 + datafusion/proto/src/generated/pbjson.rs | 15 +++++ datafusion/proto/src/generated/prost.rs | 4 +- .../proto/src/physical_plan/from_proto.rs | 3 + .../proto/src/physical_plan/to_proto.rs | 5 ++ 9 files changed, 112 insertions(+), 27 deletions(-) diff --git a/datafusion/physical-plan/src/memory.rs b/datafusion/physical-plan/src/memory.rs index bf6294f5a55b..a7d3137d1ed6 100644 --- a/datafusion/physical-plan/src/memory.rs +++ b/datafusion/physical-plan/src/memory.rs @@ -17,6 +17,7 @@ //! Execution plan for reading in-memory batches of data +use log::debug; use parking_lot::RwLock; use std::any::Any; use std::fmt; @@ -338,6 +339,8 @@ impl Stream for MemoryStream { mut self: std::pin::Pin<&mut Self>, _: &mut Context<'_>, ) -> Poll> { + // TODO: To be removed + debug!("Memory Stream poll"); Poll::Ready(if self.index < self.data.len() { self.index += 1; let batch = &self.data[self.index - 1]; @@ -350,6 +353,8 @@ impl Stream for MemoryStream { Some(Ok(batch)) } else { + // TODO: To be removed + debug!("Memory stream exhausted!"); None }) } diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 86bf26445475..328f9948767e 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -1692,19 +1692,13 @@ mod tests { ) .unwrap() } -} -#[cfg(test)] -mod test { - use arrow_schema::{DataType, Field, Schema, SortOptions}; + use arrow_schema::SortOptions; - use crate::memory::MemoryExec; + use crate::coalesce_partitions::CoalescePartitionsExec; 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 @@ -1769,6 +1763,36 @@ mod test { Ok(()) } + #[tokio::test] + async fn test_coalesce_partition() -> Result<()> { + let schema = test_schema(); + let partition = create_vec_batches(2); + let partitions = vec![partition.clone()]; + let input = Arc::new( + MemoryExec::try_new(&partitions, Arc::clone(&schema), None).unwrap(), + ); + let exec = + OnDemandRepartitionExec::try_new(input, Partitioning::RoundRobinBatch(2)).unwrap(); + + let coalesce_exec = + CoalescePartitionsExec::new(Arc::new(exec) as Arc); + + // CoalescePartitionExec should not change the plan + let expected_plan = [ + "CoalescePartitionsExec", + " OnDemandRepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", + " MemoryExec: partitions=1, partition_sizes=[2]", + ]; + assert_plan!(expected_plan, coalesce_exec.clone()); + + // execute the plan + let task_ctx = Arc::new(TaskContext::default()); + let stream = coalesce_exec.execute(0, task_ctx)?; + let batches = crate::common::collect(stream).await?; + + Ok(()) + } + #[tokio::test] async fn test_preserve_order_input_not_sorted() -> Result<()> { let schema = test_schema(); @@ -1792,9 +1816,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(); diff --git a/datafusion/physical-plan/src/repartition/on_demand_repartition.rs b/datafusion/physical-plan/src/repartition/on_demand_repartition.rs index 10540a93d5ac..8907c15fec4e 100644 --- a/datafusion/physical-plan/src/repartition/on_demand_repartition.rs +++ b/datafusion/physical-plan/src/repartition/on_demand_repartition.rs @@ -52,7 +52,7 @@ use datafusion_execution::TaskContext; use datafusion_common::HashMap; use futures::stream::Stream; use futures::{ready, FutureExt, StreamExt, TryStreamExt}; -use log::trace; +use log::{debug, trace}; use parking_lot::Mutex; #[derive(Debug, Clone)] @@ -249,6 +249,7 @@ impl ExecutionPlan for OnDemandRepartitionExec { reservation: Arc::clone(&reservation), sender: partition_tx.clone(), partition, + is_requested: false, }) as SendableRecordBatchStream }) .collect::>(); @@ -279,6 +280,7 @@ impl ExecutionPlan for OnDemandRepartitionExec { reservation, sender: partition_tx.clone(), partition, + is_requested: false, }) as SendableRecordBatchStream) } }) @@ -361,6 +363,13 @@ impl OnDemandRepartitionExec { // fetch the next batch let timer = metrics.fetch_time.timer(); + // TODO: To be removed + debug!( + "On demand pull from input Part: {} \n Are Output channels empty? {}", + partition, + output_channels.is_empty() + ); + let result = stream.next().await; timer.done(); @@ -481,6 +490,9 @@ struct OnDemandPerPartitionStream { /// Partition number partition: usize, + + /// Sender State + is_requested: bool, } impl Stream for OnDemandPerPartitionStream { @@ -490,17 +502,24 @@ impl Stream for OnDemandPerPartitionStream { mut self: Pin<&mut Self>, cx: &mut Context<'_>, ) -> Poll> { - if !self.sender.is_closed() { + // TODO: To be removed + debug!( + "On Demand Repartition per partition poll {}", + self.partition + ); + if !self.sender.is_closed() && !self.is_requested { self.sender.send_blocking(self.partition).map_err(|e| { internal_datafusion_err!( "Error sending partition number to input partitions: {}", e ) })?; + self.is_requested = true; } match ready!(self.receiver.recv().poll_unpin(cx)) { Some(Some(v)) => { + self.is_requested = false; if let Ok(batch) = &v { self.reservation .lock() @@ -510,10 +529,14 @@ impl Stream for OnDemandPerPartitionStream { Poll::Ready(Some(v)) } Some(None) => { + self.is_requested = false; // Input partition has finished sending batches Poll::Ready(None) } - None => Poll::Ready(None), + None => { + self.is_requested = false; + Poll::Ready(None) + } } } } @@ -549,6 +572,9 @@ struct OnDemandRepartitionStream { /// Partition number partition: usize, + + /// Sender state + is_requested: bool, } impl Stream for OnDemandRepartitionStream { @@ -558,19 +584,28 @@ impl Stream for OnDemandRepartitionStream { mut self: Pin<&mut Self>, cx: &mut Context<'_>, ) -> Poll> { + // TODO: To be removed + debug!("On Demand Repartition poll {}", self.partition); loop { // Send partition number to input partitions - if !self.sender.is_closed() { + if !self.sender.is_closed() && !self.is_requested { + debug!( + "On Demand Repartition poll {} sending partition", + self.partition + ); self.sender.send_blocking(self.partition).map_err(|e| { internal_datafusion_err!( "Error sending partition number to input partitions: {}", e ) })?; + self.is_requested = true; } match ready!(self.input.recv().poll_unpin(cx)) { Some(Some(v)) => { + self.is_requested = false; + if let Ok(batch) = &v { self.reservation .lock() @@ -581,6 +616,7 @@ impl Stream for OnDemandRepartitionStream { } Some(None) => { self.num_input_partitions_processed += 1; + self.is_requested = false; if self.num_input_partitions == self.num_input_partitions_processed { // all input partitions have finished sending batches @@ -591,6 +627,8 @@ impl Stream for OnDemandRepartitionStream { } } None => { + self.is_requested = false; + return Poll::Ready(None); } } @@ -1023,19 +1061,14 @@ mod tests { ) .unwrap() } -} -#[cfg(test)] -mod test { - use arrow_schema::{DataType, Field, Schema, SortOptions}; + use arrow_schema::SortOptions; - use crate::memory::MemoryExec; + use crate::coalesce_partitions::CoalescePartitionsExec; 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 @@ -1128,10 +1161,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/stream.rs b/datafusion/physical-plan/src/stream.rs index a05b46d22840..c299f4acec12 100644 --- a/datafusion/physical-plan/src/stream.rs +++ b/datafusion/physical-plan/src/stream.rs @@ -471,7 +471,11 @@ impl Stream for ObservedStream { mut self: Pin<&mut Self>, cx: &mut Context<'_>, ) -> Poll> { + // TODO: To be removed + debug!("Coalesce partitions poll ObservedStream"); let poll = self.inner.poll_next_unpin(cx); + // TODO: To be removed + debug!("Coalesce partitions poll result {:?}", poll); self.baseline_metrics.record_poll(poll) } } diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index cb0235f1f20a..23d25719a4de 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -1209,6 +1209,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 f920e16d0a71..147b6469a6f9 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -12936,6 +12936,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() @@ -12952,6 +12957,8 @@ impl<'de> serde::Deserialize<'de> for Partitioning { "roundRobin", "hash", "unknown", + "on_demand", + "onDemand", ]; #[allow(clippy::enum_variant_names)] @@ -12959,6 +12966,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 @@ -12983,6 +12991,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)), } } @@ -13024,6 +13033,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 a2be3207acab..5f380bcd1089 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -1756,7 +1756,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`. @@ -1769,6 +1769,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/physical_plan/from_proto.rs b/datafusion/proto/src/physical_plan/from_proto.rs index 5bf6218cb90e..9d26985ee699 100644 --- a/datafusion/proto/src/physical_plan/from_proto.rs +++ b/datafusion/proto/src/physical_plan/from_proto.rs @@ -447,6 +447,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 3805b970591d..8ca37dab1251 100644 --- a/datafusion/proto/src/physical_plan/to_proto.rs +++ b/datafusion/proto/src/physical_plan/to_proto.rs @@ -396,6 +396,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 { From f51176e83cf9c9b79058e0828f9ccabd4c2147bf Mon Sep 17 00:00:00 2001 From: Weijun-H Date: Tue, 14 Jan 2025 23:58:27 +0800 Subject: [PATCH 10/20] fix: Share mutex partition channel --- .../src/repartition/on_demand_repartition.rs | 40 +++++++++++++++---- 1 file changed, 33 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 8907c15fec4e..3f828a3b54b3 100644 --- a/datafusion/physical-plan/src/repartition/on_demand_repartition.rs +++ b/datafusion/physical-plan/src/repartition/on_demand_repartition.rs @@ -58,6 +58,9 @@ use parking_lot::Mutex; #[derive(Debug, Clone)] pub struct OnDemandRepartitionExec { base: RepartitionExecBase, + /// Channel to send partition number to the downstream task + partition_channel: + Arc, Receiver)>>>, } impl OnDemandRepartitionExec { @@ -179,6 +182,7 @@ impl ExecutionPlan for OnDemandRepartitionExec { ); let lazy_state = Arc::clone(&self.base.state); + let partition_channel = Arc::clone(&self.partition_channel); let input = Arc::clone(&self.base.input); let partitioning = self.partitioning().clone(); let metrics = self.base.metrics.clone(); @@ -197,7 +201,14 @@ impl ExecutionPlan for OnDemandRepartitionExec { 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_channel = partition_channel + .get_or_init(|| async move { Mutex::new(async_channel::unbounded()) }) + .await; + let (partition_tx, partition_rx) = { + let channel = partition_channel.lock(); + (channel.0.clone(), channel.1.clone()) + }; + let state = lazy_state .get_or_init(|| async move { Mutex::new( @@ -324,6 +335,7 @@ impl OnDemandRepartitionExec { preserve_order, cache, }, + partition_channel: Default::default(), }) } @@ -585,20 +597,23 @@ impl Stream for OnDemandRepartitionStream { cx: &mut Context<'_>, ) -> Poll> { // TODO: To be removed - debug!("On Demand Repartition poll {}", self.partition); + debug!( + "On Demand Repartition poll {}, start, is_requested {}", + self.partition, self.is_requested, + ); loop { // Send partition number to input partitions if !self.sender.is_closed() && !self.is_requested { - debug!( - "On Demand Repartition poll {} sending partition", - self.partition - ); self.sender.send_blocking(self.partition).map_err(|e| { internal_datafusion_err!( "Error sending partition number to input partitions: {}", e ) })?; + debug!( + "On Demand Repartition poll {}, send partition number", + self.partition + ); self.is_requested = true; } @@ -611,12 +626,19 @@ impl Stream for OnDemandRepartitionStream { .lock() .shrink(batch.get_array_memory_size()); } - + debug!( + "On Demand Repartition poll {}, return batch", + self.partition + ); return Poll::Ready(Some(v)); } Some(None) => { self.num_input_partitions_processed += 1; self.is_requested = false; + debug!( + "On Demand Repartition poll {}, input partitions processed: {}", + self.partition, self.num_input_partitions_processed + ); if self.num_input_partitions == self.num_input_partitions_processed { // all input partitions have finished sending batches @@ -627,6 +649,10 @@ impl Stream for OnDemandRepartitionStream { } } None => { + debug!( + "On Demand Repartition poll {}, input partitions None", + self.partition + ); self.is_requested = false; return Poll::Ready(None); From 2c95ca3c62a811c6cf88e666a72a1ad3d40e4972 Mon Sep 17 00:00:00 2001 From: Weijun-H Date: Thu, 16 Jan 2025 17:58:59 +0800 Subject: [PATCH 11/20] bench: replace all RoundRorbin with OnDemand Repartition --- datafusion/core/src/datasource/memory.rs | 10 +- .../src/physical_optimizer/enforce_sorting.rs | 5 + .../core/src/physical_optimizer/test_utils.rs | 12 +- datafusion/core/src/physical_planner.rs | 6 +- .../sort_preserving_repartition_fuzz.rs | 15 ++- datafusion/expr/src/logical_plan/display.rs | 7 ++ datafusion/expr/src/logical_plan/plan.rs | 12 ++ datafusion/expr/src/logical_plan/tree_node.rs | 8 +- .../physical-plan/src/repartition/mod.rs | 16 ++- .../src/repartition/on_demand_repartition.rs | 115 +++++++++++++++--- 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 | 11 +- .../substrait/src/logical_plan/producer.rs | 4 +- 15 files changed, 207 insertions(+), 34 deletions(-) diff --git a/datafusion/core/src/datasource/memory.rs b/datafusion/core/src/datasource/memory.rs index c1e0bea0b3ff..99535e042d5e 100644 --- a/datafusion/core/src/datasource/memory.rs +++ b/datafusion/core/src/datasource/memory.rs @@ -45,6 +45,7 @@ use datafusion_expr::SortExpr; use datafusion_physical_plan::metrics::MetricsSet; use async_trait::async_trait; +use datafusion_physical_plan::repartition::on_demand_repartition::OnDemandRepartitionExec; use futures::StreamExt; use log::debug; use parking_lot::Mutex; @@ -165,9 +166,14 @@ impl MemTable { let exec = MemoryExec::try_new(&data, Arc::clone(&schema), None)?; if let Some(num_partitions) = output_partitions { - let exec = RepartitionExec::try_new( + // TODO: replaced with OnDemandRepartitionExec + // let exec = RepartitionExec::try_new( + // Arc::new(exec), + // Partitioning::RoundRobinBatch(num_partitions), + // )?; + let exec = OnDemandRepartitionExec::try_new( Arc::new(exec), - Partitioning::RoundRobinBatch(num_partitions), + Partitioning::OnDemand(num_partitions), )?; // execute and collect results diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index cfc08562f7d7..533845a0ae2e 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -548,6 +548,11 @@ fn remove_bottleneck_in_subplan( if let Partitioning::RoundRobinBatch(n_out) = repartition.partitioning() { can_remove |= *n_out == input_partitioning.partition_count(); } + // TODO: replaced with OnDemand + 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) } diff --git a/datafusion/core/src/physical_optimizer/test_utils.rs b/datafusion/core/src/physical_optimizer/test_utils.rs index 9156301393c0..4dd7d60bdda7 100644 --- a/datafusion/core/src/physical_optimizer/test_utils.rs +++ b/datafusion/core/src/physical_optimizer/test_utils.rs @@ -323,12 +323,20 @@ pub fn global_limit_exec(input: Arc) -> Arc) -> Arc { - Arc::new(RepartitionExec::try_new(input, Partitioning::RoundRobinBatch(10)).unwrap()) + // TODO: replace with OnDemand + Arc::new(RepartitionExec::try_new(input, Partitioning::OnDemand(10)).unwrap()) + // Arc::new(RepartitionExec::try_new(input, Partitioning::RoundRobinBatch(10)).unwrap()) } pub fn spr_repartition_exec(input: Arc) -> Arc { + // TODO: replaced with OnDemand + // Arc::new( + // RepartitionExec::try_new(input, Partitioning::RoundRobinBatch(10)) + // .unwrap() + // .with_preserve_order(), + // ) Arc::new( - RepartitionExec::try_new(input, Partitioning::RoundRobinBatch(10)) + RepartitionExec::try_new(input, Partitioning::OnDemand(10)) .unwrap() .with_preserve_order(), ) diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 44537c951f94..c4734a78dc61 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -788,8 +788,10 @@ impl DefaultPhysicalPlanner { let physical_input = children.one()?; let input_dfschema = input.as_ref().schema(); let physical_partitioning = match partitioning_scheme { - LogicalPartitioning::RoundRobinBatch(n) => { - Partitioning::RoundRobinBatch(*n) + LogicalPartitioning::RoundRobinBatch(n) + | LogicalPartitioning::OnDemand(n) => { + // TODO: replaced by OnDemand + Partitioning::OnDemand(*n) } LogicalPartitioning::Hash(expr, n) => { let runtime_expr = expr 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 73f4a569954e..c4943668cb32 100644 --- a/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/sort_preserving_repartition_fuzz.rs @@ -43,6 +43,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_physical_expr_common::sort_expr::LexOrdering; @@ -390,8 +391,14 @@ mod sp_repartition_fuzz_tests { fn sort_preserving_repartition_exec_round_robin( input: Arc, ) -> Arc { + // TODO: replaced with OnDemandRepartitionExec + // Arc::new( + // RepartitionExec::try_new(input, Partitioning::RoundRobinBatch(2)) + // .unwrap() + // .with_preserve_order(), + // ) Arc::new( - RepartitionExec::try_new(input, Partitioning::RoundRobinBatch(2)) + OnDemandRepartitionExec::try_new(input, Partitioning::OnDemand(2)) .unwrap() .with_preserve_order(), ) @@ -400,8 +407,12 @@ mod sp_repartition_fuzz_tests { fn repartition_exec_round_robin( input: Arc, ) -> Arc { + // TODO: replaced with OnDemandRepartitionExec + // Arc::new( + // RepartitionExec::try_new(input, Partitioning::RoundRobinBatch(2)).unwrap(), + // ) Arc::new( - RepartitionExec::try_new(input, Partitioning::RoundRobinBatch(2)).unwrap(), + OnDemandRepartitionExec::try_new(input, Partitioning::OnDemand(2)).unwrap(), ) } diff --git a/datafusion/expr/src/logical_plan/display.rs b/datafusion/expr/src/logical_plan/display.rs index 14758b61e859..ae5950239d68 100644 --- a/datafusion/expr/src/logical_plan/display.rs +++ b/datafusion/expr/src/logical_plan/display.rs @@ -516,6 +516,13 @@ impl<'a, 'b> PgJsonVisitor<'a, 'b> { "Partition Count": n }) } + Partitioning::OnDemand(n) => { + json!({ + "Node Type": "Repartition", + "Partitioning Scheme": "OnDemand", + "Partition Count": n + }) + } Partitioning::Hash(expr, n) => { let hash_expr: Vec = expr.iter().map(|e| format!("{e}")).collect(); diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index 31bf4c573444..e39e6a30babc 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -835,6 +835,14 @@ impl LogicalPlan { input: Arc::new(input), })) } + Partitioning::OnDemand(n) => { + self.assert_no_expressions(expr)?; + let input = self.only_input(inputs)?; + Ok(LogicalPlan::Repartition(Repartition { + partitioning_scheme: Partitioning::OnDemand(*n), + input: Arc::new(input), + })) + } Partitioning::Hash(_, n) => { let input = self.only_input(inputs)?; Ok(LogicalPlan::Repartition(Repartition { @@ -1913,6 +1921,9 @@ impl LogicalPlan { Partitioning::RoundRobinBatch(n) => { write!(f, "Repartition: RoundRobinBatch partition_count={n}") } + Partitioning::OnDemand(n) => { + write!(f, "Repartition: OnDemand partition_count={n}") + } Partitioning::Hash(expr, n) => { let hash_expr: Vec = expr.iter().map(|e| format!("{e}")).collect(); @@ -3391,6 +3402,7 @@ pub enum Partitioning { Hash(Vec, usize), /// The DISTRIBUTE BY clause is used to repartition the data based on the input expressions DistributeBy(Vec), + OnDemand(usize), } /// Represent the unnesting operation on a list column, such as the recursion depth and diff --git a/datafusion/expr/src/logical_plan/tree_node.rs b/datafusion/expr/src/logical_plan/tree_node.rs index 1539b69b4007..d7f0e2acef6c 100644 --- a/datafusion/expr/src/logical_plan/tree_node.rs +++ b/datafusion/expr/src/logical_plan/tree_node.rs @@ -415,7 +415,9 @@ impl LogicalPlan { Partitioning::Hash(expr, _) | Partitioning::DistributeBy(expr) => { expr.apply_elements(f) } - Partitioning::RoundRobinBatch(_) => Ok(TreeNodeRecursion::Continue), + Partitioning::RoundRobinBatch(_) | Partitioning::OnDemand(_) => { + Ok(TreeNodeRecursion::Continue) + } }, LogicalPlan::Window(Window { window_expr, .. }) => { window_expr.apply_elements(f) @@ -527,7 +529,9 @@ impl LogicalPlan { Partitioning::DistributeBy(expr) => expr .map_elements(f)? .update_data(Partitioning::DistributeBy), - Partitioning::RoundRobinBatch(_) => Transformed::no(partitioning_scheme), + Partitioning::RoundRobinBatch(_) | Partitioning::OnDemand(_) => { + Transformed::no(partitioning_scheme) + } } .update_data(|partitioning_scheme| { LogicalPlan::Repartition(Repartition { diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 328f9948767e..82db377ad118 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -766,6 +766,18 @@ impl ExecutionPlan for RepartitionExec { partition: usize, context: Arc, ) -> Result { + // TODO: make sure that this is only called for hash partitioning + // match self.partitioning() { + // Partitioning::Hash(_, _) => {} + // _ => { + // panic!( + // "RepartitionExec::execute should never be called directly. \ + // Partition type: {:?}", + // self.partitioning() + // ); + // } + // } + trace!( "Start {}::execute for partition: {}", self.name(), @@ -1772,7 +1784,8 @@ mod tests { MemoryExec::try_new(&partitions, Arc::clone(&schema), None).unwrap(), ); let exec = - OnDemandRepartitionExec::try_new(input, Partitioning::RoundRobinBatch(2)).unwrap(); + OnDemandRepartitionExec::try_new(input, Partitioning::RoundRobinBatch(2)) + .unwrap(); let coalesce_exec = CoalescePartitionsExec::new(Arc::new(exec) as Arc); @@ -1816,7 +1829,6 @@ mod tests { Ok(()) } - 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 3f828a3b54b3..d57040252c08 100644 --- a/datafusion/physical-plan/src/repartition/on_demand_repartition.rs +++ b/datafusion/physical-plan/src/repartition/on_demand_repartition.rs @@ -516,8 +516,8 @@ impl Stream for OnDemandPerPartitionStream { ) -> Poll> { // TODO: To be removed debug!( - "On Demand Repartition per partition poll {}", - self.partition + "On Demand Repartition per partition poll {}, start, is_requested {}", + self.partition, self.is_requested, ); if !self.sender.is_closed() && !self.is_requested { self.sender.send_blocking(self.partition).map_err(|e| { @@ -526,6 +526,10 @@ impl Stream for OnDemandPerPartitionStream { e ) })?; + debug!( + "On Demand Repartition per partition poll {}, send partition number", + self.partition + ); self.is_requested = true; } @@ -537,15 +541,27 @@ impl Stream for OnDemandPerPartitionStream { .lock() .shrink(batch.get_array_memory_size()); } + debug!( + "On Demand Repartition per partition poll {}, return batch", + self.partition + ); Poll::Ready(Some(v)) } Some(None) => { self.is_requested = false; // Input partition has finished sending batches + debug!( + "On Demand Repartition per partition poll {}, input partitions finished", + self.partition + ); Poll::Ready(None) } None => { + debug!( + "On Demand Repartition per partition poll {}, input partitions None", + self.partition + ); self.is_requested = false; Poll::Ready(None) } @@ -675,6 +691,8 @@ mod tests { use super::*; use crate::{ + collect, + memory::MemoryExec, test::{ assert_is_pending, exec::{ @@ -682,7 +700,6 @@ mod tests { ErrorExec, MockExec, }, }, - {collect, memory::MemoryExec}, }; use arrow::array::{ArrayRef, StringArray, UInt32Array}; @@ -1133,7 +1150,7 @@ mod tests { // 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", @@ -1142,20 +1159,88 @@ mod tests { 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( + MemoryExec::try_new(&[vec![batch.clone()]], Arc::clone(&schema), None) + .unwrap() + .try_with_sort_information(vec![sort_exprs.clone()]) + .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", + " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=my_awesome_field@0 ASC", + " MemoryExec: 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); @@ -1169,16 +1254,14 @@ mod tests { 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]", diff --git a/datafusion/proto/proto/datafusion.proto b/datafusion/proto/proto/datafusion.proto index 23d25719a4de..3b08048bb3f4 100644 --- a/datafusion/proto/proto/datafusion.proto +++ b/datafusion/proto/proto/datafusion.proto @@ -145,6 +145,7 @@ message RepartitionNode { oneof partition_method { uint64 round_robin = 2; HashRepartition hash = 3; + uint64 on_demand = 4; } } diff --git a/datafusion/proto/src/generated/pbjson.rs b/datafusion/proto/src/generated/pbjson.rs index 147b6469a6f9..2d6fb64328d4 100644 --- a/datafusion/proto/src/generated/pbjson.rs +++ b/datafusion/proto/src/generated/pbjson.rs @@ -17853,6 +17853,11 @@ impl serde::Serialize for RepartitionNode { repartition_node::PartitionMethod::Hash(v) => { struct_ser.serialize_field("hash", v)?; } + repartition_node::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() @@ -17869,6 +17874,8 @@ impl<'de> serde::Deserialize<'de> for RepartitionNode { "round_robin", "roundRobin", "hash", + "on_demand", + "onDemand", ]; #[allow(clippy::enum_variant_names)] @@ -17876,6 +17883,7 @@ impl<'de> serde::Deserialize<'de> for RepartitionNode { Input, RoundRobin, Hash, + OnDemand, } impl<'de> serde::Deserialize<'de> for GeneratedField { fn deserialize(deserializer: D) -> std::result::Result @@ -17900,6 +17908,7 @@ impl<'de> serde::Deserialize<'de> for RepartitionNode { "input" => Ok(GeneratedField::Input), "roundRobin" | "round_robin" => Ok(GeneratedField::RoundRobin), "hash" => Ok(GeneratedField::Hash), + "onDemand" | "on_demand" => Ok(GeneratedField::OnDemand), _ => Err(serde::de::Error::unknown_field(value, FIELDS)), } } @@ -17942,6 +17951,12 @@ impl<'de> serde::Deserialize<'de> for RepartitionNode { partition_method__ = map_.next_value::<::std::option::Option<_>>()?.map(repartition_node::PartitionMethod::Hash) ; } + 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| repartition_node::PartitionMethod::OnDemand(x.0)); + } } } Ok(RepartitionNode { diff --git a/datafusion/proto/src/generated/prost.rs b/datafusion/proto/src/generated/prost.rs index 5f380bcd1089..079271ac57f8 100644 --- a/datafusion/proto/src/generated/prost.rs +++ b/datafusion/proto/src/generated/prost.rs @@ -205,7 +205,7 @@ pub struct SortNode { pub struct RepartitionNode { #[prost(message, optional, boxed, tag = "1")] pub input: ::core::option::Option<::prost::alloc::boxed::Box>, - #[prost(oneof = "repartition_node::PartitionMethod", tags = "2, 3")] + #[prost(oneof = "repartition_node::PartitionMethod", tags = "2, 3, 4")] pub partition_method: ::core::option::Option, } /// Nested message and enum types in `RepartitionNode`. @@ -216,6 +216,8 @@ pub mod repartition_node { RoundRobin(u64), #[prost(message, tag = "3")] Hash(super::HashRepartition), + #[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 50636048ebc9..c7edd0978c7d 100644 --- a/datafusion/proto/src/logical_plan/mod.rs +++ b/datafusion/proto/src/logical_plan/mod.rs @@ -33,6 +33,7 @@ use crate::{ }; use crate::protobuf::{proto_error, ToProtoError}; +use arrow::compute::kernels::partition; use arrow::datatypes::{DataType, Schema, SchemaRef}; use datafusion::datasource::cte_worktable::CteWorkTable; #[cfg(feature = "parquet")] @@ -516,8 +517,9 @@ impl AsLogicalPlan for LogicalPlanNode { from_proto::parse_exprs(pb_hash_expr, ctx, extension_codec)?, *partition_count as usize, ), - PartitionMethod::RoundRobin(partition_count) => { - Partitioning::RoundRobinBatch(*partition_count as usize) + PartitionMethod::RoundRobin(partition_count) + | PartitionMethod::OnDemand(partition_count) => { + Partitioning::OnDemand(*partition_count as usize) } }; @@ -1371,8 +1373,9 @@ impl AsLogicalPlan for LogicalPlanNode { partition_count: *partition_count as u64, }) } - Partitioning::RoundRobinBatch(partition_count) => { - PartitionMethod::RoundRobin(*partition_count as u64) + Partitioning::RoundRobinBatch(partition_count) + | Partitioning::OnDemand(partition_count) => { + PartitionMethod::OnDemand(*partition_count as u64) } Partitioning::DistributeBy(_) => { return not_impl_err!("DistributeBy") diff --git a/datafusion/substrait/src/logical_plan/producer.rs b/datafusion/substrait/src/logical_plan/producer.rs index 26d71c7fd3e2..6e2f06aedb95 100644 --- a/datafusion/substrait/src/logical_plan/producer.rs +++ b/datafusion/substrait/src/logical_plan/producer.rs @@ -551,6 +551,7 @@ pub fn to_substrait_rel( let partition_count = match repartition.partitioning_scheme { Partitioning::RoundRobinBatch(num) => num, Partitioning::Hash(_, num) => num, + Partitioning::OnDemand(num) => num, Partitioning::DistributeBy(_) => { return not_impl_err!( "Physical plan does not support DistributeBy partitioning" @@ -559,9 +560,10 @@ pub fn to_substrait_rel( }; // ref: https://substrait.io/relations/physical_relations/#exchange-types let exchange_kind = match &repartition.partitioning_scheme { - Partitioning::RoundRobinBatch(_) => { + Partitioning::RoundRobinBatch(_) | Partitioning::OnDemand(_) => { ExchangeKind::RoundRobin(RoundRobin::default()) } + Partitioning::Hash(exprs, _) => { let fields = exprs .iter() From 87b3d1b5e9a30dbb7855b3e638ad1e5a7c19f949 Mon Sep 17 00:00:00 2001 From: Weijun-H Date: Thu, 16 Jan 2025 18:47:02 +0800 Subject: [PATCH 12/20] test: Added tests for repartitions from n to m --- .../src/repartition/on_demand_repartition.rs | 111 +++++++++++++----- 1 file changed, 83 insertions(+), 28 deletions(-) diff --git a/datafusion/physical-plan/src/repartition/on_demand_repartition.rs b/datafusion/physical-plan/src/repartition/on_demand_repartition.rs index d57040252c08..3a9c592bf309 100644 --- a/datafusion/physical-plan/src/repartition/on_demand_repartition.rs +++ b/datafusion/physical-plan/src/repartition/on_demand_repartition.rs @@ -709,6 +709,34 @@ mod tests { use datafusion_execution::runtime_env::RuntimeEnvBuilder; 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)])) } @@ -775,6 +803,61 @@ mod tests { Ok(()) } + #[tokio::test] + async fn many_to_many_on_demand_with_coalesce() -> Result<()> { + let schema = test_schema(); + let partition: Vec = create_vec_batches(1); + let partitions = vec![partition.clone(), partition.clone()]; + let input = Arc::new( + MemoryExec::try_new(&partitions, Arc::clone(&schema), None).unwrap(), + ); + 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=[1, 1]", + ]; + assert_plan!(expected_plan, coalesce_exec.clone()); + + // execute the plan + let task_ctx = Arc::new(TaskContext::default()); + let stream = coalesce_exec.execute(0, task_ctx)?; + let batches = crate::common::collect(stream).await?; + + #[rustfmt::skip] + let expected = vec![ + "+----+", + "| c0 |", + "+----+", + "| 1 |", + "| 1 |", + "| 2 |", + "| 2 |", + "| 3 |", + "| 3 |", + "| 4 |", + "| 4 |", + "| 5 |", + "| 5 |", + "| 6 |", + "| 6 |", + "| 7 |", + "| 7 |", + "| 8 |", + "| 8 |", + "+----+", + ]; + + assert_batches_sorted_eq!(&expected, &batches); + + Ok(()) + } + #[tokio::test] async fn unsupported_partitioning() { let task_ctx = Arc::new(TaskContext::default()); @@ -1105,34 +1188,6 @@ mod tests { .unwrap() } - 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" - ); - }; - } - #[tokio::test] async fn test_preserve_order() -> Result<()> { let schema = test_schema(); From 7dbd8e314c6624579d313000869221de4ce201fd Mon Sep 17 00:00:00 2001 From: Weijun-H Date: Thu, 16 Jan 2025 22:02:52 +0800 Subject: [PATCH 13/20] chore --- .../src/repartition/on_demand_repartition.rs | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/datafusion/physical-plan/src/repartition/on_demand_repartition.rs b/datafusion/physical-plan/src/repartition/on_demand_repartition.rs index 3a9c592bf309..ee1f3f3407d3 100644 --- a/datafusion/physical-plan/src/repartition/on_demand_repartition.rs +++ b/datafusion/physical-plan/src/repartition/on_demand_repartition.rs @@ -806,7 +806,7 @@ mod tests { #[tokio::test] async fn many_to_many_on_demand_with_coalesce() -> Result<()> { let schema = test_schema(); - let partition: Vec = create_vec_batches(1); + let partition: Vec = create_vec_batches(2); let partitions = vec![partition.clone(), partition.clone()]; let input = Arc::new( MemoryExec::try_new(&partitions, Arc::clone(&schema), None).unwrap(), @@ -820,7 +820,7 @@ mod tests { let expected_plan = [ "CoalescePartitionsExec", " OnDemandRepartitionExec: partitioning=OnDemand(3), input_partitions=2", - " MemoryExec: partitions=2, partition_sizes=[1, 1]", + " MemoryExec: partitions=2, partition_sizes=[2, 2]", ]; assert_plan!(expected_plan, coalesce_exec.clone()); @@ -970,7 +970,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(); @@ -1143,7 +1143,7 @@ mod tests { let schema = test_schema(); let partition = create_vec_batches(50); let input_partitions = vec![partition]; - let partitioning = Partitioning::RoundRobinBatch(4); + let partitioning = Partitioning::OnDemand(4); // setup up context let runtime = RuntimeEnvBuilder::default() @@ -1198,7 +1198,7 @@ mod tests { let union = UnionExec::new(vec![source1, source2]); let exec = OnDemandRepartitionExec::try_new( Arc::new(union), - Partitioning::RoundRobinBatch(10), + Partitioning::OnDemand(10), ) .unwrap() .with_preserve_order(); From 0b9feffb53b6e01d56ee2499e56b0782a38afd41 Mon Sep 17 00:00:00 2001 From: Weijun-H Date: Fri, 17 Jan 2025 23:42:22 +0800 Subject: [PATCH 14/20] fix all tests --- benchmarks/src/tpch/convert.rs | 2 +- benchmarks/src/tpch/run.rs | 2 - .../enforce_distribution.rs | 296 ++++++++---------- .../src/physical_optimizer/enforce_sorting.rs | 99 +++--- .../physical_optimizer/projection_pushdown.rs | 33 ++ .../replace_with_order_preserving_variants.rs | 167 +++++----- .../src/physical_optimizer/sanity_checker.rs | 13 +- .../src/physical_optimizer/sort_pushdown.rs | 2 + .../core/src/physical_optimizer/test_utils.rs | 5 +- .../core/src/physical_optimizer/utils.rs | 6 + datafusion/core/src/physical_planner.rs | 7 +- .../combine_partial_final_agg.rs | 6 +- .../physical_optimizer/limit_pushdown.rs | 21 +- datafusion/core/tests/sql/explain_analyze.rs | 6 +- datafusion/core/tests/sql/joins.rs | 12 +- datafusion/expr/src/logical_plan/plan.rs | 11 +- datafusion/physical-expr/src/partitioning.rs | 8 +- .../src/coalesce_batches.rs | 10 +- .../physical-plan/src/execution_plan.rs | 8 + .../src/joins/nested_loop_join.rs | 10 +- datafusion/physical-plan/src/memory.rs | 7 +- .../physical-plan/src/repartition/mod.rs | 31 -- .../src/repartition/on_demand_repartition.rs | 32 +- .../src/sorts/sort_preserving_merge.rs | 1 + .../test_files/agg_func_substitute.slt | 6 +- .../sqllogictest/test_files/aggregate.slt | 30 +- .../test_files/aggregates_topk.slt | 10 +- .../test_files/count_star_rule.slt | 4 +- datafusion/sqllogictest/test_files/cte.slt | 8 +- .../sqllogictest/test_files/distinct_on.slt | 2 +- .../sqllogictest/test_files/explain.slt | 2 +- .../test_files/filter_without_sort_exec.slt | 12 +- .../sqllogictest/test_files/group_by.slt | 50 +-- .../test_files/information_schema.slt | 2 + datafusion/sqllogictest/test_files/insert.slt | 6 +- .../test_files/insert_to_external.slt | 4 +- datafusion/sqllogictest/test_files/join.slt | 8 +- .../join_disable_repartition_joins.slt | 4 +- datafusion/sqllogictest/test_files/joins.slt | 124 ++++---- datafusion/sqllogictest/test_files/json.slt | 2 +- datafusion/sqllogictest/test_files/limit.slt | 10 +- .../test_files/monotonic_projection_test.slt | 10 +- datafusion/sqllogictest/test_files/order.slt | 32 +- .../sqllogictest/test_files/parquet.slt | 6 +- .../test_files/parquet_filter_pushdown.slt | 2 +- .../sqllogictest/test_files/predicates.slt | 8 +- .../sqllogictest/test_files/repartition.slt | 4 +- datafusion/sqllogictest/test_files/select.slt | 16 +- .../sqllogictest/test_files/subquery.slt | 20 +- .../sqllogictest/test_files/subquery_sort.slt | 2 +- datafusion/sqllogictest/test_files/union.slt | 28 +- datafusion/sqllogictest/test_files/unnest.slt | 4 +- datafusion/sqllogictest/test_files/window.slt | 26 +- 53 files changed, 663 insertions(+), 574 deletions(-) diff --git a/benchmarks/src/tpch/convert.rs b/benchmarks/src/tpch/convert.rs index 30178d17aa54..03ac92618e52 100644 --- a/benchmarks/src/tpch/convert.rs +++ b/benchmarks/src/tpch/convert.rs @@ -95,7 +95,7 @@ impl ConvertOpt { // optionally, repartition the file let partitions = self.partitions; if partitions > 1 { - csv = csv.repartition(Partitioning::RoundRobinBatch(partitions))? + csv = csv.repartition(Partitioning::OnDemand(partitions))? } // create the physical plan diff --git a/benchmarks/src/tpch/run.rs b/benchmarks/src/tpch/run.rs index f2e4e70e87aa..de3ee3d67db2 100644 --- a/benchmarks/src/tpch/run.rs +++ b/benchmarks/src/tpch/run.rs @@ -121,8 +121,6 @@ impl RunOpt { .config() .with_collect_statistics(!self.disable_statistics); config.options_mut().optimizer.prefer_hash_join = self.prefer_hash_join; - config.options_mut().optimizer.enable_on_demand_repartition = true; - config.options_mut().optimizer.enable_round_robin_repartition = false; let ctx = SessionContext::new_with_config(config); // register tables diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index d44c17e426ac..10cf48b65c63 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -61,6 +61,8 @@ use datafusion_physical_plan::ExecutionPlanProperties; use datafusion_physical_expr_common::sort_expr::LexOrdering; use itertools::izip; +use super::utils::is_on_demand_repartition; + /// The `EnforceDistribution` rule ensures that distribution requirements are /// met. In doing so, this rule will increase the parallelism in the plan by /// introducing repartitioning operators to the physical plan. @@ -848,36 +850,15 @@ fn add_roundrobin_on_top( // - 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::RoundRobinBatch(n_target); - let repartition = - RepartitionExec::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) - } -} - -fn add_on_demand_repartition_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`) + // TODO: replaced with OnDemandRepartitionExec let partitioning = Partitioning::OnDemand(n_target); let repartition = OnDemandRepartitionExec::try_new(Arc::clone(&input.plan), partitioning)? .with_preserve_order(); + // let partitioning = Partitioning::RoundRobinBatch(n_target); + // let repartition = + // RepartitionExec::try_new(Arc::clone(&input.plan), partitioning)? + // .with_preserve_order(); let new_plan = Arc::new(repartition) as _; @@ -1007,6 +988,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) { @@ -1060,10 +1042,21 @@ fn replace_order_preserving_variants( context.plan.as_any().downcast_ref::() { if repartition.preserve_order() { - context.plan = Arc::new(RepartitionExec::try_new( - Arc::clone(&context.children[0].plan), - repartition.partitioning().clone(), - )?); + match repartition.partitioning() { + Partitioning::RoundRobinBatch(n) => { + context.plan = Arc::new(OnDemandRepartitionExec::try_new( + Arc::clone(&context.children[0].plan), + Partitioning::OnDemand(*n), + )?); + } + _ => { + context.plan = Arc::new(RepartitionExec::try_new( + Arc::clone(&context.children[0].plan), + repartition.partitioning().clone(), + )?); + } + } + return Ok(context); } } else if let Some(repartition) = context @@ -1290,15 +1283,10 @@ fn ensure_distribution( // to increase parallelism. child = add_roundrobin_on_top(child, target_partitions)?; } - // When inserting hash is necessary to satisy hash requirement, insert hash repartition. if hash_necessary { child = add_hash_on_top(child, exprs.to_vec(), target_partitions)?; } - if enable_on_demand_repartition { - child = - add_on_demand_repartition_on_top(child, target_partitions)?; - } } Distribution::UnspecifiedDistribution => { if add_roundrobin { @@ -1306,10 +1294,6 @@ fn ensure_distribution( // to increase parallelism. child = add_roundrobin_on_top(child, target_partitions)?; } - if enable_on_demand_repartition { - child = - add_on_demand_repartition_on_top(child, target_partitions)?; - } } }; @@ -1417,13 +1401,11 @@ fn update_children(mut dist_context: DistributionContext) -> Result() + } else if child_plan_any + .downcast_ref::() + .is_some() { - !matches!( - repartition.partitioning(), - Partitioning::UnknownPartitioning(_) - ) + true } else { child_plan_any.is::() || child_plan_any.is::() @@ -2068,14 +2050,14 @@ pub(crate) mod tests { top_join_plan.as_str(), join_plan.as_str(), "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "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", + "OnDemandRepartitionExec: partitioning=OnDemand(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]", "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ], // Should include 4 RepartitionExecs @@ -2084,14 +2066,14 @@ pub(crate) mod tests { "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", join_plan.as_str(), "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "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", + "OnDemandRepartitionExec: partitioning=OnDemand(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]", "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ], }; @@ -2132,14 +2114,14 @@ pub(crate) mod tests { top_join_plan.as_str(), join_plan.as_str(), "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "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", + "OnDemandRepartitionExec: partitioning=OnDemand(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]", "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ], // Should include 4 RepartitionExecs @@ -2149,14 +2131,14 @@ pub(crate) mod tests { "RepartitionExec: partitioning=Hash([b1@6], 10), input_partitions=10", join_plan.as_str(), "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "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", + "OnDemandRepartitionExec: partitioning=OnDemand(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]", "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ], }; @@ -2208,13 +2190,13 @@ pub(crate) mod tests { "ProjectionExec: expr=[a@0 as a1, a@0 as a2]", "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", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "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", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "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", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ]; assert_optimized!(expected, top_join.clone(), true); @@ -2234,13 +2216,13 @@ pub(crate) mod tests { "ProjectionExec: expr=[a@0 as a1, a@0 as a2]", "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", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "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", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "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", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ]; assert_optimized!(expected, top_join.clone(), true); @@ -2289,13 +2271,13 @@ pub(crate) mod tests { "ProjectionExec: expr=[c@2 as c1]", "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", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "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", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "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", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ]; @@ -2331,12 +2313,12 @@ pub(crate) mod tests { "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", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "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", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ]; assert_optimized!(expected, join.clone(), true); @@ -2384,12 +2366,12 @@ pub(crate) mod tests { "AggregateExec: mode=FinalPartitioned, gby=[b1@0 as b1, a1@1 as a1], aggr=[]", "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", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "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", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ]; assert_optimized!(expected, join.clone(), true); @@ -2509,18 +2491,18 @@ pub(crate) mod tests { "ProjectionExec: expr=[a@0 as A, a@0 as AA, b@1 as B, c@2 as C]", "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", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "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", + "OnDemandRepartitionExec: partitioning=OnDemand(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]", "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", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "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", + "OnDemandRepartitionExec: partitioning=OnDemand(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]", ]; @@ -2653,18 +2635,18 @@ pub(crate) mod tests { "ProjectionExec: expr=[a@0 as A, a@0 as AA, b@1 as B, c@2 as C]", "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", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "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", + "OnDemandRepartitionExec: partitioning=OnDemand(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]", "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", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "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", + "OnDemandRepartitionExec: partitioning=OnDemand(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]", ]; @@ -2793,18 +2775,18 @@ pub(crate) mod tests { "ProjectionExec: expr=[a@0 as A, a@0 as AA, b@1 as B, c@2 as C]", "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", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "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", + "OnDemandRepartitionExec: partitioning=OnDemand(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]", "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", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "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", + "OnDemandRepartitionExec: partitioning=OnDemand(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]", ]; @@ -2871,16 +2853,16 @@ pub(crate) mod tests { join_plan.as_str(), "SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "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", + "OnDemandRepartitionExec: partitioning=OnDemand(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]", "SortExec: expr=[c@2 ASC], preserve_partitioning=[true]", "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ], // Should include 7 RepartitionExecs (4 hash, 3 round-robin), 4 SortExecs @@ -2900,16 +2882,16 @@ pub(crate) mod tests { join_plan.as_str(), "SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "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", + "OnDemandRepartitionExec: partitioning=OnDemand(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]", "SortExec: expr=[c@2 ASC], preserve_partitioning=[true]", "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ], }; @@ -2922,16 +2904,16 @@ pub(crate) mod tests { top_join_plan.as_str(), join_plan.as_str(), "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", "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", + "OnDemandRepartitionExec: partitioning=OnDemand(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]", "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10, preserve_order=true, sort_exprs=c@2 ASC", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ], @@ -2948,21 +2930,21 @@ pub(crate) mod tests { top_join_plan.as_str(), // Below 4 operators are differences introduced, when join mode is changed "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", "CoalescePartitionsExec", join_plan.as_str(), "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", "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", + "OnDemandRepartitionExec: partitioning=OnDemand(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]", "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10, preserve_order=true, sort_exprs=c@2 ASC", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ], @@ -2994,16 +2976,16 @@ pub(crate) mod tests { join_plan.as_str(), "SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "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", + "OnDemandRepartitionExec: partitioning=OnDemand(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]", "SortExec: expr=[c@2 ASC], preserve_partitioning=[true]", "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ], // Should include 7 RepartitionExecs (4 hash, 3 round-robin) and 4 SortExecs @@ -3014,16 +2996,16 @@ pub(crate) mod tests { join_plan.as_str(), "SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "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", + "OnDemandRepartitionExec: partitioning=OnDemand(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]", "SortExec: expr=[c@2 ASC], preserve_partitioning=[true]", "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ], // this match arm cannot be reached @@ -3037,16 +3019,16 @@ pub(crate) mod tests { top_join_plan.as_str(), join_plan.as_str(), "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", "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", + "OnDemandRepartitionExec: partitioning=OnDemand(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]", "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10, preserve_order=true, sort_exprs=c@2 ASC", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ], @@ -3054,21 +3036,21 @@ pub(crate) mod tests { JoinType::Left | JoinType::Full => vec![ top_join_plan.as_str(), "RepartitionExec: partitioning=Hash([b1@6], 10), input_partitions=10, preserve_order=true, sort_exprs=b1@6 ASC", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "SortExec: expr=[b1@6 ASC], preserve_partitioning=[false]", "CoalescePartitionsExec", join_plan.as_str(), "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", "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", + "OnDemandRepartitionExec: partitioning=OnDemand(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]", "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10, preserve_order=true, sort_exprs=c@2 ASC", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ], @@ -3144,14 +3126,14 @@ pub(crate) mod tests { "AggregateExec: mode=FinalPartitioned, gby=[b1@0 as b1, a1@1 as a1], aggr=[]", "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", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "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", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ]; assert_optimized!(expected, join.clone(), true, true); @@ -3159,7 +3141,7 @@ pub(crate) mod tests { let expected_first_sort_enforcement = &[ "SortMergeJoin: join_type=Inner, on=[(b3@1, b2@1), (a3@0, a2@0)]", "RepartitionExec: partitioning=Hash([b3@1, a3@0], 10), input_partitions=10, preserve_order=true, sort_exprs=b3@1 ASC, a3@0 ASC", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "SortExec: expr=[b3@1 ASC, a3@0 ASC], preserve_partitioning=[false]", "CoalescePartitionsExec", "ProjectionExec: expr=[a1@0 as a3, b1@1 as b3]", @@ -3167,17 +3149,17 @@ pub(crate) mod tests { "AggregateExec: mode=FinalPartitioned, gby=[b1@0 as b1, a1@1 as a1], aggr=[]", "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", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "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", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "SortExec: expr=[b2@1 ASC, a2@0 ASC], preserve_partitioning=[false]", "CoalescePartitionsExec", "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", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ]; assert_optimized!(expected_first_sort_enforcement, join, false, true); @@ -3256,12 +3238,12 @@ pub(crate) mod tests { "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", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "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", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ]; assert_optimized!(expected, plan.clone(), true); @@ -3299,12 +3281,12 @@ pub(crate) mod tests { "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", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "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", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "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 @@ -3339,7 +3321,7 @@ pub(crate) mod tests { "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", "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", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ]; assert_optimized!(expected, plan.clone(), true); @@ -3358,7 +3340,7 @@ pub(crate) mod tests { "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", "FilterExec: c@2 = 0", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ]; assert_optimized!(expected, plan.clone(), true); @@ -3378,7 +3360,7 @@ pub(crate) mod tests { "LocalLimitExec: fetch=100", "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", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ]; @@ -3427,7 +3409,7 @@ pub(crate) mod tests { "FilterExec: c@2 = 0", // We can use repartition here, ordering requirement by SortRequiredExec // is still satisfied. - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ]; @@ -3450,13 +3432,13 @@ pub(crate) mod tests { "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", "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", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "GlobalLimitExec: skip=0, fetch=100", "CoalescePartitionsExec", "LocalLimitExec: fetch=100", "FilterExec: c@2 = 0", // repartition should happen prior to the filter to maximize parallelism - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "GlobalLimitExec: skip=0, fetch=100", "LocalLimitExec: fetch=100", // Expect no repartition to happen for local limit @@ -3592,7 +3574,7 @@ pub(crate) mod tests { let expected = &[ "SortRequiredExec: [d@3 ASC]", "FilterExec: c@2 = 0", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[d@3 ASC]", ]; @@ -3634,7 +3616,7 @@ pub(crate) mod tests { "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", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ]; assert_optimized!(expected, plan.clone(), true); @@ -3667,7 +3649,7 @@ pub(crate) mod tests { "SortExec: expr=[sum@0 ASC], preserve_partitioning=[true]", // 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", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ]; @@ -3678,7 +3660,7 @@ pub(crate) mod tests { "CoalescePartitionsExec", // 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", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ]; assert_optimized!(expected_first_sort_enforcement, plan, false); @@ -3766,7 +3748,7 @@ pub(crate) mod tests { "SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", // 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", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ]; @@ -3777,7 +3759,7 @@ pub(crate) mod tests { "CoalescePartitionsExec", "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", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ]; assert_optimized!(expected_first_sort_enforcement, plan, false); @@ -3813,7 +3795,7 @@ pub(crate) mod tests { "ProjectionExec: expr=[a@0 as a, b@1 as b, c@2 as c]", "FilterExec: c@2 = 0", // repartition is lowest down - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ]; @@ -3824,7 +3806,7 @@ pub(crate) mod tests { "CoalescePartitionsExec", "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", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ]; assert_optimized!(expected_first_sort_enforcement, plan, false); @@ -3924,7 +3906,7 @@ pub(crate) mod tests { "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=[]", - "RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", + "OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1", "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", ]; @@ -4073,7 +4055,7 @@ pub(crate) mod tests { "FilterExec: c@2 = 0", // even though data is sorted, we can use repartition here. Since // ordering is not used in subsequent stages anyway. - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "OnDemandRepartitionExec: partitioning=OnDemand(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]", @@ -4085,7 +4067,7 @@ pub(crate) mod tests { "FilterExec: c@2 = 0", // even though data is sorted, we can use repartition here. Since // ordering is not used in subsequent stages anyway. - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "OnDemandRepartitionExec: partitioning=OnDemand(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", @@ -4112,13 +4094,13 @@ pub(crate) mod tests { "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", "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", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "GlobalLimitExec: skip=0, fetch=100", "CoalescePartitionsExec", "LocalLimitExec: fetch=100", "FilterExec: c@2 = 0", // repartition should happen prior to the filter to maximize parallelism - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "GlobalLimitExec: skip=0, fetch=100", // Limit doesn't benefit from input partitioning - no parallelism "LocalLimitExec: fetch=100", @@ -4128,13 +4110,13 @@ pub(crate) mod tests { "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", "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", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "GlobalLimitExec: skip=0, fetch=100", "CoalescePartitionsExec", "LocalLimitExec: fetch=100", "FilterExec: c@2 = 0", // repartition should happen prior to the filter to maximize parallelism - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "GlobalLimitExec: skip=0, fetch=100", // Limit doesn't benefit from input partitioning - no parallelism "LocalLimitExec: fetch=100", @@ -4358,17 +4340,17 @@ pub(crate) mod tests { let repartition = repartition_exec(repartition_exec(input)); let physical_plan = repartition_exec(filter_exec(repartition)); let expected = &[ - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=10", " FilterExec: c@2 = 0", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=10", + " OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", " 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", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ]; assert_optimized!(expected, physical_plan.clone(), true); @@ -4392,7 +4374,7 @@ pub(crate) mod tests { let expected = &[ "CoalescePartitionsExec", "FilterExec: c@2 = 0", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2, preserve_order=true, sort_exprs=c@2 ASC", + "OnDemandRepartitionExec: partitioning=OnDemand(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]", ]; // last flag sets config.optimizer.PREFER_EXISTING_SORT @@ -4415,7 +4397,7 @@ pub(crate) mod tests { let expected = &[ "SortPreservingMergeExec: [d@3 ASC]", "FilterExec: c@2 = 0", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2, preserve_order=true, sort_exprs=d@3 ASC", + "OnDemandRepartitionExec: partitioning=OnDemand(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]", ]; // last flag sets config.optimizer.PREFER_EXISTING_SORT @@ -4439,7 +4421,7 @@ pub(crate) mod tests { "SortPreservingMergeExec: [a@0 ASC]", "SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", "FilterExec: c@2 = 0", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=2", "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", ]; @@ -4449,7 +4431,7 @@ pub(crate) mod tests { "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", "CoalescePartitionsExec", "FilterExec: c@2 = 0", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=2", "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", ]; assert_optimized!(expected, physical_plan, false); @@ -4472,7 +4454,7 @@ pub(crate) mod tests { "CoalescePartitionsExec", // 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", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=2", "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); @@ -4500,7 +4482,7 @@ pub(crate) mod tests { "SortPreservingMergeExec: [a@0 ASC]", "SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", "FilterExec: c@2 = 0", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=2", "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", ]; @@ -4511,7 +4493,7 @@ pub(crate) mod tests { "CoalescePartitionsExec", "SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", "FilterExec: c@2 = 0", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=2", "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", ]; assert_optimized!(expected, physical_plan, false); @@ -4531,7 +4513,7 @@ pub(crate) mod tests { let expected = &[ "FilterExec: c@2 = 0", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=2", "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); @@ -4563,7 +4545,7 @@ pub(crate) mod tests { // Since at the start of the rule ordering requirement is not satisfied // EnforceDistribution rule doesn't satisfy this requirement either. "FilterExec: c@2 = 0", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ]; @@ -4657,7 +4639,7 @@ pub(crate) mod tests { "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=[]", - "RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2", + "OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=2", "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). @@ -4672,7 +4654,7 @@ pub(crate) mod tests { let physical_plan = coalesce_partitions_exec(repartition_exec(parquet_exec())); let expected = &[ "CoalescePartitionsExec", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", " ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ]; plans_matches_expected!(expected, physical_plan.clone()); @@ -4692,10 +4674,10 @@ pub(crate) mod tests { ))); let expected = &[ "FilterExec: c@2 = 0", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", " CoalescePartitionsExec", " FilterExec: c@2 = 0", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", " ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ]; plans_matches_expected!(expected, physical_plan.clone()); @@ -4703,7 +4685,7 @@ pub(crate) mod tests { let expected = &[ "FilterExec: c@2 = 0", "FilterExec: c@2 = 0", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", ]; assert_optimized!(expected, physical_plan.clone(), true); diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index 533845a0ae2e..327381c8a556 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -37,7 +37,7 @@ use std::sync::Arc; -use super::utils::{add_sort_above, add_sort_above_with_check}; +use super::utils::{add_sort_above, add_sort_above_with_check, is_on_demand_repartition}; use crate::config::ConfigOptions; use crate::error::Result; use crate::physical_optimizer::replace_with_order_preserving_variants::{ @@ -65,6 +65,7 @@ use datafusion_physical_expr::Partitioning; use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; use datafusion_physical_optimizer::PhysicalOptimizerRule; 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::ExecutionPlanProperties; @@ -520,8 +521,9 @@ fn remove_bottleneck_in_subplan( // We can safely use the 0th index since we have a `CoalescePartitionsExec`. let mut new_child_node = children[0].children.swap_remove(0); while new_child_node.plan.output_partitioning() == plan.output_partitioning() - && is_repartition(&new_child_node.plan) - && is_repartition(plan) + && (is_repartition(&new_child_node.plan) + || is_on_demand_repartition(&new_child_node.plan)) + && (is_repartition(plan) || is_on_demand_repartition(plan)) { new_child_node = new_child_node.children.swap_remove(0) } @@ -548,11 +550,21 @@ fn remove_bottleneck_in_subplan( if let Partitioning::RoundRobinBatch(n_out) = repartition.partitioning() { can_remove |= *n_out == input_partitioning.partition_count(); } - // TODO: replaced with OnDemand + 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) } @@ -615,6 +627,13 @@ fn remove_corresponding_sort_from_sub_plan( if is_sort_preserving_merge(&node.plan) { node.children = node.children.swap_remove(0).children; node.plan = Arc::clone(node.plan.children().swap_remove(0)); + } else if let Some(repartition) = + node.plan.as_any().downcast_ref::() + { + node.plan = Arc::new(OnDemandRepartitionExec::try_new( + Arc::clone(&node.children[0].plan), + repartition.properties().output_partitioning().clone(), + )?) as _; } else if let Some(repartition) = node.plan.as_any().downcast_ref::() { @@ -948,8 +967,8 @@ mod tests { let physical_plan = repartition_exec(repartition_exec(sort3)); let expected_input = [ - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=10", + " OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", " SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", @@ -959,8 +978,8 @@ mod tests { ]; let expected_optimized = [ - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=10", + " OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", " MemoryExec: partitions=1, partition_sizes=[0]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -996,7 +1015,7 @@ mod tests { "AggregateExec: mode=Final, gby=[], aggr=[]", " 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=1", + " OnDemandRepartitionExec: partitioning=OnDemand(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]", @@ -1005,7 +1024,7 @@ mod tests { let expected_optimized = [ "AggregateExec: mode=Final, gby=[], aggr=[]", " CoalescePartitionsExec", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", " MemoryExec: partitions=1, partition_sizes=[0]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -1049,18 +1068,18 @@ mod tests { " SortPreservingMergeExec: [non_nullable_col@1 ASC]", " SortExec: expr=[non_nullable_col@1 ASC], preserve_partitioning=[true]", " UnionExec", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", " MemoryExec: partitions=1, partition_sizes=[0]", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", " 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", + " OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", " MemoryExec: partitions=1, partition_sizes=[0]", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", " MemoryExec: partitions=1, partition_sizes=[0]"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -1275,7 +1294,7 @@ mod tests { let physical_plan = sort_preserving_merge_exec(sort_exprs, repartition); let expected_input = ["SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", + " OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=2", " UnionExec", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", " GlobalLimitExec: skip=0, fetch=100", @@ -1286,7 +1305,7 @@ mod tests { // 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", + " OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=2", " UnionExec", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", " GlobalLimitExec: skip=0, fetch=100", @@ -1610,7 +1629,7 @@ mod tests { " 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", + " OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", " 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. @@ -1620,7 +1639,7 @@ mod tests { " 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", + " OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -2120,12 +2139,12 @@ mod tests { let expected_input = ["SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", " FilterExec: NOT non_nullable_col@1", " CoalescePartitionsExec", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", " 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", + " OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -2154,7 +2173,7 @@ mod tests { let actual = get_plan_string(&orig_plan); let expected_input = vec![ "SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(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]", ]; @@ -2210,14 +2229,14 @@ mod tests { let expected_input = ["SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", " SortPreservingMergeExec: [nullable_col@0 ASC]", " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[true]", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", " CoalescePartitionsExec", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", " 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", + " OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", " MemoryExec: partitions=1, partition_sizes=[0]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -2241,12 +2260,12 @@ mod tests { let expected_input = ["SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", " CoalescePartitionsExec", " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", " 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", + " OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", " 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); @@ -2279,14 +2298,14 @@ mod tests { "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", " CoalescePartitionsExec", " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", " StreamingTableExec: partition_sizes=1, projection=[a, b, c, d, e], infinite_source=true, output_ordering=[a@0 ASC]", ]; let expected_input_bounded = vec![ "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", " CoalescePartitionsExec", " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(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", ]; @@ -2294,7 +2313,7 @@ mod tests { let expected_optimized_unbounded = vec![ "SortPreservingMergeExec: [a@0 ASC]", " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", " StreamingTableExec: partition_sizes=1, projection=[a, b, c, d, e], infinite_source=true, output_ordering=[a@0 ASC]", ]; @@ -2303,14 +2322,14 @@ mod tests { "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", " CoalescePartitionsExec", " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(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", ]; 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", + " OnDemandRepartitionExec: partitioning=OnDemand(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", ]; let (expected_input, expected_optimized, expected_optimized_sort_parallelize) = @@ -2354,11 +2373,11 @@ mod tests { 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", + " OnDemandRepartitionExec: partitioning=OnDemand(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",]; 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", + " OnDemandRepartitionExec: partitioning=OnDemand(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",]; assert_optimized!(expected_input, expected_optimized, physical_plan, false); @@ -2383,11 +2402,11 @@ mod tests { 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", + " OnDemandRepartitionExec: partitioning=OnDemand(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",]; 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", + " OnDemandRepartitionExec: partitioning=OnDemand(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",]; assert_optimized!(expected_input, expected_optimized, physical_plan, false); @@ -2409,8 +2428,8 @@ mod tests { let expected_input = [ "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: [a@0 ASC, b@1 ASC]", - " 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", + " OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC, b@1 ASC", + " OnDemandRepartitionExec: partitioning=OnDemand(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", ]; @@ -2418,8 +2437,8 @@ mod tests { "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=[a@0 ASC, b@1 ASC], preserve_partitioning=[false]", " CoalescePartitionsExec", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=10", + " OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", ]; assert_optimized!(expected_input, expected_optimized, physical_plan, false); diff --git a/datafusion/core/src/physical_optimizer/projection_pushdown.rs b/datafusion/core/src/physical_optimizer/projection_pushdown.rs index 3ac40bfb62ea..642e88728cf2 100644 --- a/datafusion/core/src/physical_optimizer/projection_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/projection_pushdown.rs @@ -51,6 +51,7 @@ use datafusion_physical_expr::{ utils::collect_columns, Partitioning, PhysicalExpr, PhysicalExprRef, PhysicalSortExpr, PhysicalSortRequirement, }; +use datafusion_physical_plan::repartition::on_demand_repartition::OnDemandRepartitionExec; use datafusion_physical_plan::streaming::StreamingTableExec; use datafusion_physical_plan::union::UnionExec; @@ -130,6 +131,10 @@ pub fn remove_unnecessary_projections( )? } else if let Some(repartition) = input.downcast_ref::() { try_swapping_with_repartition(projection, repartition)? + } else if let Some(on_demand_repartition) = + input.downcast_ref::() + { + try_swapping_with_on_demand_repartition(projection, on_demand_repartition)? } else if let Some(sort) = input.downcast_ref::() { try_swapping_with_sort(projection, sort)? } else if let Some(spm) = input.downcast_ref::() { @@ -413,6 +418,34 @@ fn try_swapping_with_filter( .map(|e| Some(Arc::new(e) as _)) } +/// Tries to swap the projection with its input [`RepartitionExec`]. If it can be done, +/// it returns the new swapped version having the [`RepartitionExec`] as the top plan. +/// Otherwise, it returns None. +fn try_swapping_with_on_demand_repartition( + projection: &ProjectionExec, + repartition: &OnDemandRepartitionExec, +) -> 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, repartition.input())?; + + let new_partitioning = repartition.partitioning().clone(); + + Ok(Some(Arc::new(OnDemandRepartitionExec::try_new( + new_projection, + new_partitioning, + )?))) +} + /// Tries to swap the projection with its input [`RepartitionExec`]. If it can be done, /// it returns the new swapped version having the [`RepartitionExec`] as the top plan. /// Otherwise, it returns None. diff --git a/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs b/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs index 2f6b7a51ee75..2d2331b6d3fe 100644 --- a/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs +++ b/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs @@ -21,7 +21,7 @@ use std::sync::Arc; -use super::utils::{is_repartition, is_sort_preserving_merge}; +use super::utils::{is_on_demand_repartition, is_repartition, is_sort_preserving_merge}; use crate::error::Result; use crate::physical_optimizer::utils::{is_coalesce_partitions, is_sort}; use crate::physical_plan::repartition::RepartitionExec; @@ -29,7 +29,9 @@ use crate::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::Transformed; +use datafusion_physical_expr::Partitioning; use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; +use datafusion_physical_plan::repartition::on_demand_repartition::OnDemandRepartitionExec; use datafusion_physical_plan::tree_node::PlanContext; use datafusion_physical_plan::ExecutionPlanProperties; @@ -55,6 +57,7 @@ pub fn update_children(opc: &mut OrderPreservationContext) { 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. @@ -66,7 +69,8 @@ pub fn update_children(opc: &mut OrderPreservationContext) { *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())) { @@ -114,7 +118,7 @@ fn plan_with_order_preserving_variants( .collect::>()?; sort_input.data = false; - if is_repartition(&sort_input.plan) + if (is_repartition(&sort_input.plan) || is_on_demand_repartition(&sort_input.plan)) && !sort_input.plan.maintains_input_order()[0] && is_spr_better { @@ -181,7 +185,25 @@ fn plan_with_order_breaking_variants( // non-sort-preserving variant: 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 _; + + // TODO: replaced with OnDemandRepartitionExec for RoundRobinBatch + match partitioning { + Partitioning::RoundRobinBatch(n) => { + sort_input.plan = Arc::new(OnDemandRepartitionExec::try_new( + child, + Partitioning::OnDemand(n), + )?) as _; + } + _ => { + sort_input.plan = + Arc::new(RepartitionExec::try_new(child, partitioning)?) as _; + } + } + } else if is_on_demand_repartition(plan) && plan.maintains_input_order()[0] { + 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); @@ -437,14 +459,14 @@ mod tests { "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", " SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true]", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", ]; let expected_input_bounded = [ "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", " SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true]", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", ]; @@ -452,7 +474,7 @@ mod tests { let expected_optimized_unbounded = [ "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", ]; @@ -461,13 +483,13 @@ mod tests { "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", " SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true]", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", ]; let expected_optimized_bounded_sort_preserve = [ "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", ]; assert_optimized_in_all_boundedness_situations!( @@ -519,11 +541,11 @@ mod tests { " SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", " FilterExec: c@1 > 3", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", " SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", " CoalescePartitionsExec", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC]", ]; let expected_input_bounded = [ @@ -531,11 +553,11 @@ mod tests { " SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", " FilterExec: c@1 > 3", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", " SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", " CoalescePartitionsExec", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC], has_header=true", ]; @@ -544,10 +566,10 @@ mod tests { "SortPreservingMergeExec: [a@0 ASC]", " FilterExec: c@1 > 3", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", " SortPreservingMergeExec: [a@0 ASC]", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC]", ]; @@ -557,21 +579,21 @@ mod tests { " SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", " FilterExec: c@1 > 3", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", " SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", " CoalescePartitionsExec", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC], has_header=true", ]; let expected_optimized_bounded_sort_preserve = [ "SortPreservingMergeExec: [a@0 ASC]", " FilterExec: c@1 > 3", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", " SortPreservingMergeExec: [a@0 ASC]", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC], has_header=true", ]; assert_optimized_in_all_boundedness_situations!( @@ -612,7 +634,7 @@ mod tests { " SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true]", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " FilterExec: c@1 > 3", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", ]; let expected_input_bounded = [ @@ -620,7 +642,7 @@ mod tests { " SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true]", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " FilterExec: c@1 > 3", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", ]; @@ -629,7 +651,7 @@ mod tests { "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", " FilterExec: c@1 > 3", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", ]; @@ -639,14 +661,14 @@ mod tests { " SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true]", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " FilterExec: c@1 > 3", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", ]; let expected_optimized_bounded_sort_preserve = [ "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", " FilterExec: c@1 > 3", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", ]; assert_optimized_in_all_boundedness_situations!( @@ -689,7 +711,7 @@ mod tests { " CoalesceBatchesExec: target_batch_size=8192", " FilterExec: c@1 > 3", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", ]; let expected_input_bounded = [ @@ -698,7 +720,7 @@ mod tests { " CoalesceBatchesExec: target_batch_size=8192", " FilterExec: c@1 > 3", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", ]; @@ -708,7 +730,7 @@ mod tests { " CoalesceBatchesExec: target_batch_size=8192", " FilterExec: c@1 > 3", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", ]; @@ -719,7 +741,7 @@ mod tests { " CoalesceBatchesExec: target_batch_size=8192", " FilterExec: c@1 > 3", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", ]; let expected_optimized_bounded_sort_preserve = [ @@ -727,7 +749,7 @@ mod tests { " CoalesceBatchesExec: target_batch_size=8192", " FilterExec: c@1 > 3", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", ]; assert_optimized_in_all_boundedness_situations!( @@ -773,7 +795,7 @@ mod tests { " FilterExec: c@1 > 3", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " CoalesceBatchesExec: target_batch_size=8192", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", ]; let expected_input_bounded = [ @@ -783,7 +805,7 @@ mod tests { " FilterExec: c@1 > 3", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " CoalesceBatchesExec: target_batch_size=8192", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", ]; @@ -794,7 +816,7 @@ mod tests { " FilterExec: c@1 > 3", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", " CoalesceBatchesExec: target_batch_size=8192", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", ]; @@ -806,7 +828,7 @@ mod tests { " FilterExec: c@1 > 3", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", " CoalesceBatchesExec: target_batch_size=8192", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", ]; let expected_optimized_bounded_sort_preserve = [ @@ -815,7 +837,7 @@ mod tests { " FilterExec: c@1 > 3", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", " CoalesceBatchesExec: target_batch_size=8192", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", ]; assert_optimized_in_all_boundedness_situations!( @@ -856,7 +878,7 @@ mod tests { " CoalesceBatchesExec: target_batch_size=8192", " FilterExec: c@1 > 3", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", ]; let expected_input_bounded = [ @@ -864,7 +886,7 @@ mod tests { " CoalesceBatchesExec: target_batch_size=8192", " FilterExec: c@1 > 3", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", ]; @@ -874,7 +896,7 @@ mod tests { " CoalesceBatchesExec: target_batch_size=8192", " FilterExec: c@1 > 3", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", ]; @@ -884,7 +906,7 @@ mod tests { " CoalesceBatchesExec: target_batch_size=8192", " FilterExec: c@1 > 3", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", ]; let expected_optimized_bounded_sort_preserve = expected_optimized_bounded; @@ -931,7 +953,7 @@ mod tests { " CoalesceBatchesExec: target_batch_size=8192", " FilterExec: c@1 > 3", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", ]; let expected_input_bounded = [ @@ -941,7 +963,7 @@ mod tests { " CoalesceBatchesExec: target_batch_size=8192", " FilterExec: c@1 > 3", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", ]; @@ -952,7 +974,7 @@ mod tests { " CoalesceBatchesExec: target_batch_size=8192", " FilterExec: c@1 > 3", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", ]; @@ -964,7 +986,7 @@ mod tests { " CoalesceBatchesExec: target_batch_size=8192", " FilterExec: c@1 > 3", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", ]; let expected_optimized_bounded_sort_preserve = [ @@ -973,7 +995,7 @@ mod tests { " CoalesceBatchesExec: target_batch_size=8192", " FilterExec: c@1 > 3", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", ]; assert_optimized_in_all_boundedness_situations!( @@ -1018,14 +1040,14 @@ mod tests { "SortPreservingMergeExec: [c@1 ASC]", " SortExec: expr=[c@1 ASC], preserve_partitioning=[true]", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", ]; let expected_input_bounded = [ "SortPreservingMergeExec: [c@1 ASC]", " SortExec: expr=[c@1 ASC], preserve_partitioning=[true]", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", ]; @@ -1034,7 +1056,7 @@ mod tests { "SortPreservingMergeExec: [c@1 ASC]", " SortExec: expr=[c@1 ASC], preserve_partitioning=[true]", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", ]; @@ -1043,7 +1065,7 @@ mod tests { "SortPreservingMergeExec: [c@1 ASC]", " SortExec: expr=[c@1 ASC], preserve_partitioning=[true]", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", ]; let expected_optimized_bounded_sort_preserve = expected_optimized_bounded; @@ -1083,14 +1105,14 @@ mod tests { "SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[false]", " CoalescePartitionsExec", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", ]; let expected_input_bounded = [ "SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[false]", " CoalescePartitionsExec", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", ]; @@ -1098,7 +1120,7 @@ mod tests { let expected_optimized_unbounded = [ "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", ]; @@ -1107,13 +1129,13 @@ mod tests { "SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[false]", " CoalescePartitionsExec", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", ]; let expected_optimized_bounded_sort_preserve = [ "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", ]; assert_optimized_in_all_boundedness_situations!( @@ -1165,11 +1187,11 @@ mod tests { " SortExec: expr=[c@1 ASC], preserve_partitioning=[true]", " FilterExec: c@1 > 3", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", " SortExec: expr=[c@1 ASC], preserve_partitioning=[false]", " CoalescePartitionsExec", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", ]; let expected_input_bounded = [ @@ -1177,11 +1199,11 @@ mod tests { " SortExec: expr=[c@1 ASC], preserve_partitioning=[true]", " FilterExec: c@1 > 3", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", " SortExec: expr=[c@1 ASC], preserve_partitioning=[false]", " CoalescePartitionsExec", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", ]; @@ -1190,11 +1212,11 @@ mod tests { "SortPreservingMergeExec: [c@1 ASC]", " FilterExec: c@1 > 3", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=c@1 ASC", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", " SortExec: expr=[c@1 ASC], preserve_partitioning=[false]", " CoalescePartitionsExec", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", ]; @@ -1204,22 +1226,22 @@ mod tests { " SortExec: expr=[c@1 ASC], preserve_partitioning=[true]", " FilterExec: c@1 > 3", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", " SortExec: expr=[c@1 ASC], preserve_partitioning=[false]", " CoalescePartitionsExec", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", ]; let expected_optimized_bounded_sort_preserve = [ "SortPreservingMergeExec: [c@1 ASC]", " FilterExec: c@1 > 3", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=c@1 ASC", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", " SortExec: expr=[c@1 ASC], preserve_partitioning=[false]", " CoalescePartitionsExec", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", ]; assert_optimized_in_all_boundedness_situations!( @@ -1283,11 +1305,11 @@ mod tests { " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@1, c@1)]", " CoalesceBatchesExec: target_batch_size=4096", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", " CoalesceBatchesExec: target_batch_size=4096", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", ]; let expected_input_bounded = [ @@ -1296,11 +1318,11 @@ mod tests { " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@1, c@1)]", " CoalesceBatchesExec: target_batch_size=4096", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", " CoalesceBatchesExec: target_batch_size=4096", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", ]; @@ -1311,11 +1333,11 @@ mod tests { " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@1, c@1)]", " CoalesceBatchesExec: target_batch_size=4096", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", " CoalesceBatchesExec: target_batch_size=4096", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", ]; @@ -1327,11 +1349,11 @@ mod tests { " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@1, c@1)]", " CoalesceBatchesExec: target_batch_size=4096", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", " CoalesceBatchesExec: target_batch_size=4096", " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", ]; let expected_optimized_bounded_sort_preserve = expected_optimized_bounded; @@ -1398,8 +1420,11 @@ mod tests { fn repartition_exec_round_robin( input: Arc, ) -> Arc { + // Arc::new( + // RepartitionExec::try_new(input, Partitioning::RoundRobinBatch(8)).unwrap(), + // ) Arc::new( - RepartitionExec::try_new(input, Partitioning::RoundRobinBatch(8)).unwrap(), + OnDemandRepartitionExec::try_new(input, Partitioning::OnDemand(8)).unwrap(), ) } diff --git a/datafusion/core/src/physical_optimizer/sanity_checker.rs b/datafusion/core/src/physical_optimizer/sanity_checker.rs index 99bd1cab3ed4..7561956cd5fd 100644 --- a/datafusion/core/src/physical_optimizer/sanity_checker.rs +++ b/datafusion/core/src/physical_optimizer/sanity_checker.rs @@ -175,6 +175,7 @@ mod tests { use datafusion_physical_expr::expressions::col; use datafusion_physical_expr::Partitioning; use datafusion_physical_plan::displayable; + use datafusion_physical_plan::repartition::on_demand_repartition::OnDemandRepartitionExec; use datafusion_physical_plan::repartition::RepartitionExec; fn create_test_schema() -> SchemaRef { @@ -504,7 +505,7 @@ mod tests { limit.as_ref(), vec![ "GlobalLimitExec: skip=0, fetch=100", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", " MemoryExec: partitions=1, partition_sizes=[0]", ], ); @@ -635,9 +636,13 @@ mod tests { let sort_exprs2 = vec![sort_expr_options("a", &source2.schema(), sort_opts)]; let left = sort_exec(sort_exprs1, source1); let right = sort_exec(sort_exprs2, source2); - let right = Arc::new(RepartitionExec::try_new( + // let right = Arc::new(RepartitionExec::try_new( + // right, + // Partitioning::RoundRobinBatch(10), + // )?); + let right = Arc::new(OnDemandRepartitionExec::try_new( right, - Partitioning::RoundRobinBatch(10), + Partitioning::OnDemand(10), )?); let left_jcol = col("c9", &left.schema()).unwrap(); let right_jcol = col("a", &right.schema()).unwrap(); @@ -659,7 +664,7 @@ mod tests { " RepartitionExec: partitioning=Hash([c9@0], 10), input_partitions=1", " SortExec: expr=[c9@0 ASC], preserve_partitioning=[false]", " MemoryExec: partitions=1, partition_sizes=[0]", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", " SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", " MemoryExec: partitions=1, partition_sizes=[0]", ], diff --git a/datafusion/core/src/physical_optimizer/sort_pushdown.rs b/datafusion/core/src/physical_optimizer/sort_pushdown.rs index 6c761f674b3b..7b93fd72a495 100644 --- a/datafusion/core/src/physical_optimizer/sort_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/sort_pushdown.rs @@ -41,6 +41,7 @@ use datafusion_physical_expr::PhysicalSortRequirement; use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; use datafusion_physical_plan::joins::utils::ColumnIndex; use datafusion_physical_plan::joins::HashJoinExec; +use datafusion_physical_plan::repartition::on_demand_repartition::OnDemandRepartitionExec; /// This is a "data class" we use within the [`EnforceSorting`] rule to push /// down [`SortExec`] in the plan. In some cases, we can reduce the total @@ -271,6 +272,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/core/src/physical_optimizer/test_utils.rs b/datafusion/core/src/physical_optimizer/test_utils.rs index 4dd7d60bdda7..2b83fdd70d86 100644 --- a/datafusion/core/src/physical_optimizer/test_utils.rs +++ b/datafusion/core/src/physical_optimizer/test_utils.rs @@ -51,6 +51,7 @@ use datafusion_expr::{WindowFrame, WindowFunctionDefinition}; use datafusion_functions_aggregate::count::count_udaf; use datafusion_physical_expr::expressions::col; use datafusion_physical_expr::{PhysicalExpr, PhysicalSortExpr}; +use datafusion_physical_plan::repartition::on_demand_repartition::OnDemandRepartitionExec; use datafusion_physical_plan::tree_node::PlanContext; use datafusion_physical_plan::{ displayable, DisplayAs, DisplayFormatType, PlanProperties, @@ -324,7 +325,7 @@ pub fn global_limit_exec(input: Arc) -> Arc) -> Arc { // TODO: replace with OnDemand - Arc::new(RepartitionExec::try_new(input, Partitioning::OnDemand(10)).unwrap()) + Arc::new(OnDemandRepartitionExec::try_new(input, Partitioning::OnDemand(10)).unwrap()) // Arc::new(RepartitionExec::try_new(input, Partitioning::RoundRobinBatch(10)).unwrap()) } @@ -336,7 +337,7 @@ pub fn spr_repartition_exec(input: Arc) -> Arc) -> bool { 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::() +} diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index c4734a78dc61..a11426f5efb0 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -84,6 +84,7 @@ use datafusion_physical_expr::aggregate::{AggregateExprBuilder, AggregateFunctio use datafusion_physical_expr::expressions::Literal; use datafusion_physical_expr::LexOrdering; use datafusion_physical_plan::placeholder_row::PlaceholderRowExec; +use datafusion_physical_plan::repartition::on_demand_repartition::OnDemandRepartitionExec; use datafusion_physical_plan::unnest::ListUnnest; use datafusion_sql::utils::window_expr_common_partition_keys; @@ -791,7 +792,11 @@ impl DefaultPhysicalPlanner { LogicalPartitioning::RoundRobinBatch(n) | LogicalPartitioning::OnDemand(n) => { // TODO: replaced by OnDemand - Partitioning::OnDemand(*n) + + return Ok(Arc::new(OnDemandRepartitionExec::try_new( + physical_input, + Partitioning::OnDemand(*n), + )?)); } LogicalPartitioning::Hash(expr, n) => { let runtime_expr = expr diff --git a/datafusion/core/tests/physical_optimizer/combine_partial_final_agg.rs b/datafusion/core/tests/physical_optimizer/combine_partial_final_agg.rs index 85076abdaf29..d8b508be8cbc 100644 --- a/datafusion/core/tests/physical_optimizer/combine_partial_final_agg.rs +++ b/datafusion/core/tests/physical_optimizer/combine_partial_final_agg.rs @@ -34,6 +34,7 @@ use datafusion_physical_plan::aggregates::{ AggregateExec, AggregateMode, PhysicalGroupBy, }; use datafusion_physical_plan::displayable; +use datafusion_physical_plan::repartition::on_demand_repartition::OnDemandRepartitionExec; use datafusion_physical_plan::repartition::RepartitionExec; use datafusion_physical_plan::ExecutionPlan; @@ -122,7 +123,8 @@ fn final_aggregate_exec( } fn repartition_exec(input: Arc) -> Arc { - Arc::new(RepartitionExec::try_new(input, Partitioning::RoundRobinBatch(10)).unwrap()) + // Arc::new(RepartitionExec::try_new(input, Partitioning::RoundRobinBatch(10)).unwrap()) + Arc::new(OnDemandRepartitionExec::try_new(input, Partitioning::OnDemand(10)).unwrap()) } // Return appropriate expr depending if COUNT is for col or table (*) @@ -157,7 +159,7 @@ fn aggregations_not_combined() -> datafusion_common::Result<()> { // should not combine the Partial/Final AggregateExecs let expected = &[ "AggregateExec: mode=Final, gby=[], aggr=[COUNT(1)]", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "OnDemandRepartitionExec: partitioning=OnDemand(10), input_partitions=1", "AggregateExec: mode=Partial, gby=[], aggr=[COUNT(1)]", "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c]", ]; diff --git a/datafusion/core/tests/physical_optimizer/limit_pushdown.rs b/datafusion/core/tests/physical_optimizer/limit_pushdown.rs index 1b4c28d41d19..c1a206a9f024 100644 --- a/datafusion/core/tests/physical_optimizer/limit_pushdown.rs +++ b/datafusion/core/tests/physical_optimizer/limit_pushdown.rs @@ -31,6 +31,7 @@ use datafusion_physical_plan::empty::EmptyExec; use datafusion_physical_plan::filter::FilterExec; use datafusion_physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; 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::sorts::sort_preserving_merge::SortPreservingMergeExec; @@ -121,7 +122,7 @@ fn transforms_coalesce_batches_exec_into_fetching_version_and_removes_local_limi " LocalLimitExec: fetch=5", " CoalesceBatchesExec: target_batch_size=8192", " FilterExec: c3@2 > 0", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", " StreamingTableExec: partition_sizes=1, projection=[c1, c2, c3], infinite_source=true" ]; assert_eq!(initial, expected_initial); @@ -134,7 +135,7 @@ fn transforms_coalesce_batches_exec_into_fetching_version_and_removes_local_limi " CoalescePartitionsExec", " CoalesceBatchesExec: target_batch_size=8192, fetch=5", " FilterExec: c3@2 > 0", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", " StreamingTableExec: partition_sizes=1, projection=[c1, c2, c3], infinite_source=true" ]; assert_eq!(get_plan_string(&after_optimize), expected); @@ -227,7 +228,7 @@ fn pushes_global_limit_into_multiple_fetch_plans() -> datafusion_common::Result< "GlobalLimitExec: skip=0, fetch=5", " SortPreservingMergeExec: [c1@0 ASC]", " SortExec: expr=[c1@0 ASC], preserve_partitioning=[false]", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", " ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, c3@2 as c3]", " CoalesceBatchesExec: target_batch_size=8192", " StreamingTableExec: partition_sizes=1, projection=[c1, c2, c3], infinite_source=true" @@ -241,7 +242,7 @@ fn pushes_global_limit_into_multiple_fetch_plans() -> datafusion_common::Result< let expected = [ "SortPreservingMergeExec: [c1@0 ASC], fetch=5", " SortExec: TopK(fetch=5), expr=[c1@0 ASC], preserve_partitioning=[false]", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", " ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, c3@2 as c3]", " CoalesceBatchesExec: target_batch_size=8192", " StreamingTableExec: partition_sizes=1, projection=[c1, c2, c3], infinite_source=true" @@ -266,7 +267,7 @@ fn keeps_pushed_local_limit_exec_when_there_are_multiple_input_partitions( "GlobalLimitExec: skip=0, fetch=5", " CoalescePartitionsExec", " FilterExec: c3@2 > 0", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", " StreamingTableExec: partition_sizes=1, projection=[c1, c2, c3], infinite_source=true" ]; assert_eq!(initial, expected_initial); @@ -278,7 +279,7 @@ fn keeps_pushed_local_limit_exec_when_there_are_multiple_input_partitions( "GlobalLimitExec: skip=0, fetch=5", " CoalescePartitionsExec", " FilterExec: c3@2 > 0", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", " StreamingTableExec: partition_sizes=1, projection=[c1, c2, c3], infinite_source=true" ]; assert_eq!(get_plan_string(&after_optimize), expected); @@ -479,9 +480,13 @@ fn coalesce_partitions_exec( fn repartition_exec( streaming_table: Arc, ) -> datafusion_common::Result> { - Ok(Arc::new(RepartitionExec::try_new( + // Ok(Arc::new(RepartitionExec::try_new( + // streaming_table, + // Partitioning::RoundRobinBatch(8), + // )?)) + Ok(Arc::new(OnDemandRepartitionExec::try_new( streaming_table, - Partitioning::RoundRobinBatch(8), + Partitioning::OnDemand(8), )?)) } diff --git a/datafusion/core/tests/sql/explain_analyze.rs b/datafusion/core/tests/sql/explain_analyze.rs index 39fd492786bc..c3f62d172f8d 100644 --- a/datafusion/core/tests/sql/explain_analyze.rs +++ b/datafusion/core/tests/sql/explain_analyze.rs @@ -616,7 +616,7 @@ async fn test_physical_plan_display_indent() { " AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[max(aggregate_test_100.c12), min(aggregate_test_100.c12)]", " CoalesceBatchesExec: target_batch_size=4096", " FilterExec: c12@1 < 10", - " RepartitionExec: partitioning=RoundRobinBatch(9000), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(9000), input_partitions=1", " CsvExec: file_groups={1 group: [[ARROW_TEST_DATA/csv/aggregate_test_100.csv]]}, projection=[c1, c12], has_header=true", ]; @@ -656,11 +656,11 @@ async fn test_physical_plan_display_indent_multi_children() { " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c1@0, c2@0)], projection=[c1@0]", " CoalesceBatchesExec: target_batch_size=4096", " RepartitionExec: partitioning=Hash([c1@0], 9000), input_partitions=9000", - " RepartitionExec: partitioning=RoundRobinBatch(9000), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(9000), input_partitions=1", " CsvExec: file_groups={1 group: [[ARROW_TEST_DATA/csv/aggregate_test_100.csv]]}, projection=[c1], has_header=true", " CoalesceBatchesExec: target_batch_size=4096", " RepartitionExec: partitioning=Hash([c2@0], 9000), input_partitions=9000", - " RepartitionExec: partitioning=RoundRobinBatch(9000), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(9000), input_partitions=1", " ProjectionExec: expr=[c1@0 as c2]", " CsvExec: file_groups={1 group: [[ARROW_TEST_DATA/csv/aggregate_test_100.csv]]}, projection=[c1], has_header=true", ]; diff --git a/datafusion/core/tests/sql/joins.rs b/datafusion/core/tests/sql/joins.rs index fab92c0f9c2b..ea2a0ff1a94f 100644 --- a/datafusion/core/tests/sql/joins.rs +++ b/datafusion/core/tests/sql/joins.rs @@ -66,11 +66,11 @@ async fn join_change_in_planner() -> Result<()> { "SymmetricHashJoinExec: mode=Partitioned, join_type=Full, on=[(a2@1, a2@1)], filter=CAST(a1@0 AS Int64) > CAST(a1@1 AS Int64) + 3 AND CAST(a1@0 AS Int64) < CAST(a1@1 AS Int64) + 10", " CoalesceBatchesExec: target_batch_size=8192", " RepartitionExec: partitioning=Hash([a2@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a1@0 ASC NULLS LAST", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", // " CsvExec: file_groups={1 group: [[tempdir/left.csv]]}, projection=[a1, a2], has_header=false", " CoalesceBatchesExec: target_batch_size=8192", " RepartitionExec: partitioning=Hash([a2@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a1@0 ASC NULLS LAST", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", // " CsvExec: file_groups={1 group: [[tempdir/right.csv]]}, projection=[a1, a2], has_header=false" ] }; @@ -134,11 +134,11 @@ async fn join_no_order_on_filter() -> Result<()> { "SymmetricHashJoinExec: mode=Partitioned, join_type=Full, on=[(a2@1, a2@1)], filter=CAST(a3@0 AS Int64) > CAST(a3@1 AS Int64) + 3 AND CAST(a3@0 AS Int64) < CAST(a3@1 AS Int64) + 10", " CoalesceBatchesExec: target_batch_size=8192", " RepartitionExec: partitioning=Hash([a2@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", // " CsvExec: file_groups={1 group: [[tempdir/left.csv]]}, projection=[a1, a2], has_header=false", " CoalesceBatchesExec: target_batch_size=8192", " RepartitionExec: partitioning=Hash([a2@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", // " CsvExec: file_groups={1 group: [[tempdir/right.csv]]}, projection=[a1, a2], has_header=false" ] }; @@ -184,11 +184,11 @@ async fn join_change_in_planner_without_sort() -> Result<()> { "SymmetricHashJoinExec: mode=Partitioned, join_type=Full, on=[(a2@1, a2@1)], filter=CAST(a1@0 AS Int64) > CAST(a1@1 AS Int64) + 3 AND CAST(a1@0 AS Int64) < CAST(a1@1 AS Int64) + 10", " CoalesceBatchesExec: target_batch_size=8192", " RepartitionExec: partitioning=Hash([a2@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", // " CsvExec: file_groups={1 group: [[tempdir/left.csv]]}, projection=[a1, a2], has_header=false", " CoalesceBatchesExec: target_batch_size=8192", " RepartitionExec: partitioning=Hash([a2@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1", // " CsvExec: file_groups={1 group: [[tempdir/right.csv]]}, projection=[a1, a2], has_header=false" ] }; diff --git a/datafusion/expr/src/logical_plan/plan.rs b/datafusion/expr/src/logical_plan/plan.rs index e39e6a30babc..8c041a893eff 100644 --- a/datafusion/expr/src/logical_plan/plan.rs +++ b/datafusion/expr/src/logical_plan/plan.rs @@ -55,6 +55,7 @@ use datafusion_common::{ UnnestOptions, }; use indexmap::IndexSet; +use sqlparser::ast::Partition; // backwards compatibility use crate::display::PgJsonVisitor; @@ -827,15 +828,7 @@ impl LogicalPlan { partitioning_scheme, .. }) => match partitioning_scheme { - Partitioning::RoundRobinBatch(n) => { - self.assert_no_expressions(expr)?; - let input = self.only_input(inputs)?; - Ok(LogicalPlan::Repartition(Repartition { - partitioning_scheme: Partitioning::RoundRobinBatch(*n), - input: Arc::new(input), - })) - } - Partitioning::OnDemand(n) => { + Partitioning::RoundRobinBatch(n) | Partitioning::OnDemand(n) => { self.assert_no_expressions(expr)?; let input = self.only_input(inputs)?; Ok(LogicalPlan::Repartition(Repartition { diff --git a/datafusion/physical-expr/src/partitioning.rs b/datafusion/physical-expr/src/partitioning.rs index f5d0817234d3..d383572ea9fe 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, } } @@ -314,7 +319,8 @@ mod tests { let single_partition = Partitioning::UnknownPartitioning(1); let unspecified_partition = Partitioning::UnknownPartitioning(10); - let round_robin_partition = Partitioning::RoundRobinBatch(10); + // let round_robin_partition = Partitioning::RoundRobinBatch(10); + let round_robin_partition = Partitioning::OnDemand(10); let hash_partition1 = Partitioning::Hash(partition_exprs1, 10); let hash_partition2 = Partitioning::Hash(partition_exprs2, 10); let eq_properties = EquivalenceProperties::new(schema); diff --git a/datafusion/physical-optimizer/src/coalesce_batches.rs b/datafusion/physical-optimizer/src/coalesce_batches.rs index 5cf2c877c61a..a3661342ddd1 100644 --- a/datafusion/physical-optimizer/src/coalesce_batches.rs +++ b/datafusion/physical-optimizer/src/coalesce_batches.rs @@ -26,8 +26,11 @@ use datafusion_common::config::ConfigOptions; use datafusion_common::error::Result; use datafusion_physical_expr::Partitioning; use datafusion_physical_plan::{ - coalesce_batches::CoalesceBatchesExec, filter::FilterExec, joins::HashJoinExec, - repartition::RepartitionExec, ExecutionPlan, + coalesce_batches::CoalesceBatchesExec, + filter::FilterExec, + joins::HashJoinExec, + repartition::{on_demand_repartition::OnDemandRepartitionExec, RepartitionExec}, + ExecutionPlan, }; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; @@ -70,8 +73,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-plan/src/execution_plan.rs b/datafusion/physical-plan/src/execution_plan.rs index ba9e4b0697c1..c98859cf35af 100644 --- a/datafusion/physical-plan/src/execution_plan.rs +++ b/datafusion/physical-plan/src/execution_plan.rs @@ -46,6 +46,7 @@ pub use crate::display::{DefaultDisplay, DisplayAs, DisplayFormatType, VerboseDi pub use crate::metrics::Metric; use crate::metrics::MetricsSet; pub use crate::ordering::InputOrderMode; +use crate::repartition::on_demand_repartition::OnDemandRepartitionExec; use crate::repartition::RepartitionExec; use crate::sorts::sort_preserving_merge::SortPreservingMergeExec; pub use crate::stream::EmptyRecordBatchStream; @@ -669,6 +670,13 @@ pub fn need_data_exchange(plan: Arc) -> bool { repartition.properties().output_partitioning(), Partitioning::RoundRobinBatch(_) ) + } else if let Some(repartition) = + plan.as_any().downcast_ref::() + { + !matches!( + repartition.properties().output_partitioning(), + Partitioning::OnDemand(_) + ) } else if let Some(coalesce) = plan.as_any().downcast_ref::() { coalesce.input().output_partitioning().partition_count() > 1 diff --git a/datafusion/physical-plan/src/joins/nested_loop_join.rs b/datafusion/physical-plan/src/joins/nested_loop_join.rs index d174564178df..4d85aba87732 100644 --- a/datafusion/physical-plan/src/joins/nested_loop_join.rs +++ b/datafusion/physical-plan/src/joins/nested_loop_join.rs @@ -851,6 +851,7 @@ impl RecordBatchStream for NestedLoopJoinStr #[cfg(test)] pub(crate) mod tests { use super::*; + use crate::repartition::on_demand_repartition::OnDemandRepartitionExec; use crate::{ common, expressions::Column, memory::MemoryExec, repartition::RepartitionExec, test::build_table_i32, @@ -986,9 +987,14 @@ pub(crate) mod tests { let partition_count = 4; // Redistributing right input - let right = Arc::new(RepartitionExec::try_new( + // TODO: replaced with OnDemandRepartitionExec + // let right = Arc::new(RepartitionExec::try_new( + // right, + // Partitioning::RoundRobinBatch(partition_count), + // )?) as Arc; + let right = Arc::new(OnDemandRepartitionExec::try_new( right, - Partitioning::RoundRobinBatch(partition_count), + Partitioning::OnDemand(partition_count), )?) as Arc; // Use the required distribution for nested loop join to test partition data diff --git a/datafusion/physical-plan/src/memory.rs b/datafusion/physical-plan/src/memory.rs index a7d3137d1ed6..ce16cccae23c 100644 --- a/datafusion/physical-plan/src/memory.rs +++ b/datafusion/physical-plan/src/memory.rs @@ -395,9 +395,14 @@ impl LazyMemoryExec { schema: SchemaRef, generators: Vec>>, ) -> Result { + // let cache = PlanProperties::new( + // EquivalenceProperties::new(Arc::clone(&schema)), + // Partitioning::RoundRobinBatch(generators.len()), + // ExecutionMode::Bounded, + // ); let cache = PlanProperties::new( EquivalenceProperties::new(Arc::clone(&schema)), - Partitioning::RoundRobinBatch(generators.len()), + Partitioning::OnDemand(generators.len()), ExecutionMode::Bounded, ); Ok(Self { diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 82db377ad118..f9090e4bbeea 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -1775,37 +1775,6 @@ mod tests { Ok(()) } - #[tokio::test] - async fn test_coalesce_partition() -> Result<()> { - let schema = test_schema(); - let partition = create_vec_batches(2); - let partitions = vec![partition.clone()]; - let input = Arc::new( - MemoryExec::try_new(&partitions, Arc::clone(&schema), None).unwrap(), - ); - let exec = - OnDemandRepartitionExec::try_new(input, Partitioning::RoundRobinBatch(2)) - .unwrap(); - - let coalesce_exec = - CoalescePartitionsExec::new(Arc::new(exec) as Arc); - - // CoalescePartitionExec should not change the plan - let expected_plan = [ - "CoalescePartitionsExec", - " OnDemandRepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[2]", - ]; - assert_plan!(expected_plan, coalesce_exec.clone()); - - // execute the plan - let task_ctx = Arc::new(TaskContext::default()); - let stream = coalesce_exec.execute(0, task_ctx)?; - let batches = crate::common::collect(stream).await?; - - Ok(()) - } - #[tokio::test] async fn test_preserve_order_input_not_sorted() -> Result<()> { let schema = test_schema(); diff --git a/datafusion/physical-plan/src/repartition/on_demand_repartition.rs b/datafusion/physical-plan/src/repartition/on_demand_repartition.rs index ee1f3f3407d3..7f59c2f15b63 100644 --- a/datafusion/physical-plan/src/repartition/on_demand_repartition.rs +++ b/datafusion/physical-plan/src/repartition/on_demand_repartition.rs @@ -530,7 +530,7 @@ impl Stream for OnDemandPerPartitionStream { "On Demand Repartition per partition poll {}, send partition number", self.partition ); - self.is_requested = true; + // self.is_requested = true; } match ready!(self.receiver.recv().poll_unpin(cx)) { @@ -836,18 +836,34 @@ mod tests { "+----+", "| 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 |", "+----+", @@ -897,7 +913,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(); @@ -931,7 +947,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(); @@ -1196,12 +1212,10 @@ mod tests { 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::OnDemand(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 = [ diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index 5639f0ccf2a8..58b4958145bb 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -345,6 +345,7 @@ mod tests { use crate::expressions::col; use crate::memory::MemoryExec; use crate::metrics::{MetricValue, Timestamp}; + use crate::repartition::on_demand_repartition::OnDemandRepartitionExec; use crate::repartition::RepartitionExec; use crate::sorts::sort::SortExec; use crate::stream::RecordBatchReceiverStream; diff --git a/datafusion/sqllogictest/test_files/agg_func_substitute.slt b/datafusion/sqllogictest/test_files/agg_func_substitute.slt index 9a0a1d587433..c137310ee492 100644 --- a/datafusion/sqllogictest/test_files/agg_func_substitute.slt +++ b/datafusion/sqllogictest/test_files/agg_func_substitute.slt @@ -49,7 +49,7 @@ physical_plan 04)------CoalesceBatchesExec: target_batch_size=8192 05)--------RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 06)----------AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[nth_value(multiple_ordered_table.c,Int64(1)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]], ordering_mode=Sorted -07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +07)------------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 08)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], has_header=true @@ -69,7 +69,7 @@ physical_plan 04)------CoalesceBatchesExec: target_batch_size=8192 05)--------RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 06)----------AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[nth_value(multiple_ordered_table.c,Int64(1)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]], ordering_mode=Sorted -07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +07)------------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 08)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], has_header=true query TT @@ -88,7 +88,7 @@ physical_plan 04)------CoalesceBatchesExec: target_batch_size=8192 05)--------RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 06)----------AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[nth_value(multiple_ordered_table.c,Int64(1) + Int64(100)) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]], ordering_mode=Sorted -07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +07)------------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 08)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], has_header=true query II diff --git a/datafusion/sqllogictest/test_files/aggregate.slt b/datafusion/sqllogictest/test_files/aggregate.slt index e76c1466a547..0e1a6f278842 100644 --- a/datafusion/sqllogictest/test_files/aggregate.slt +++ b/datafusion/sqllogictest/test_files/aggregate.slt @@ -147,7 +147,7 @@ physical_plan 02)--CoalescePartitionsExec 03)----AggregateExec: mode=Partial, gby=[], aggr=[array_agg(agg_order.c1) ORDER BY [agg_order.c2 DESC NULLS FIRST, agg_order.c3 ASC NULLS LAST]] 04)------SortExec: expr=[c2@1 DESC, c3@2 ASC NULLS LAST], preserve_partitioning=[true] -05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +05)--------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/aggregate_agg_multi_order.csv]]}, projection=[c1, c2, c3], has_header=true # test array_agg_order with list data type @@ -1000,7 +1000,7 @@ physical_plan 05)--------AggregateExec: mode=FinalPartitioned, gby=[alias1@0 as alias1], aggr=[] 06)----------CoalesceBatchesExec: target_batch_size=8192 07)------------RepartitionExec: partitioning=Hash([alias1@0], 4), input_partitions=4 -08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)--------------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 09)----------------AggregateExec: mode=Partial, gby=[c@0 as alias1], aggr=[] 10)------------------MemoryExec: partitions=1, partition_sizes=[1] @@ -4935,7 +4935,7 @@ physical_plan 08)--------------CoalesceBatchesExec: target_batch_size=8192 09)----------------RepartitionExec: partitioning=Hash([c3@0], 4), input_partitions=4 10)------------------AggregateExec: mode=Partial, gby=[c3@1 as c3], aggr=[min(aggregate_test_100.c1)] -11)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +11)--------------------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 12)----------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c3], has_header=true @@ -4960,7 +4960,7 @@ physical_plan 02)--AggregateExec: mode=Final, gby=[c3@0 as c3], aggr=[], lim=[5] 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[c3@0 as c3], aggr=[], lim=[5] -05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +05)--------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c3], has_header=true query I @@ -4984,7 +4984,7 @@ physical_plan 02)--AggregateExec: mode=Final, gby=[c2@0 as c2, c3@1 as c3], aggr=[], lim=[9] 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[c2@0 as c2, c3@1 as c3], aggr=[], lim=[9] -05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +05)--------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3], has_header=true query II @@ -5012,14 +5012,14 @@ physical_plan 02)--AggregateExec: mode=Final, gby=[c3@0 as c3], aggr=[], lim=[4] 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[c3@0 as c3], aggr=[], lim=[4] -05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +05)--------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 06)----------ProjectionExec: expr=[c3@1 as c3] 07)------------AggregateExec: mode=Final, gby=[c2@0 as c2, c3@1 as c3], aggr=[] 08)--------------CoalescePartitionsExec 09)----------------AggregateExec: mode=Partial, gby=[c2@0 as c2, c3@1 as c3], aggr=[] 10)------------------CoalesceBatchesExec: target_batch_size=8192 11)--------------------FilterExec: c3@1 >= 10 AND c3@1 <= 20 -12)----------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +12)----------------------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 13)------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3], has_header=true query I @@ -5045,7 +5045,7 @@ physical_plan 03)----AggregateExec: mode=Final, gby=[c2@0 as c2, c3@1 as c3], aggr=[max(aggregate_test_100.c1)] 04)------CoalescePartitionsExec 05)--------AggregateExec: mode=Partial, gby=[c2@1 as c2, c3@2 as c3], aggr=[max(aggregate_test_100.c1)] -06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +06)----------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 07)------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3], has_header=true # TODO(msirek): Extend checking in LimitedDistinctAggregation equal groupings to ignore the order of columns @@ -5064,12 +5064,12 @@ physical_plan 02)--AggregateExec: mode=Final, gby=[c3@0 as c3, c2@1 as c2], aggr=[], lim=[13] 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[c3@0 as c3, c2@1 as c2], aggr=[], lim=[13] -05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +05)--------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 06)----------ProjectionExec: expr=[c3@1 as c3, c2@0 as c2] 07)------------AggregateExec: mode=Final, gby=[c2@0 as c2, c3@1 as c3], aggr=[] 08)--------------CoalescePartitionsExec 09)----------------AggregateExec: mode=Partial, gby=[c2@0 as c2, c3@1 as c3], aggr=[] -10)------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +10)------------------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 11)--------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3], has_header=true query II @@ -5093,7 +5093,7 @@ physical_plan 03)----AggregateExec: mode=Final, gby=[c2@0 as c2, c3@1 as c3, __grouping_id@2 as __grouping_id], aggr=[], lim=[3] 04)------CoalescePartitionsExec 05)--------AggregateExec: mode=Partial, gby=[(NULL as c2, NULL as c3), (c2@0 as c2, NULL as c3), (c2@0 as c2, c3@1 as c3)], aggr=[] -06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +06)----------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 07)------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3], has_header=true query II @@ -5120,7 +5120,7 @@ physical_plan 02)--AggregateExec: mode=Final, gby=[c3@0 as c3], aggr=[] 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[c3@0 as c3], aggr=[] -05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +05)--------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c3], has_header=true statement ok @@ -5905,7 +5905,7 @@ physical_plan 01)AggregateExec: mode=Final, gby=[], aggr=[first_value(convert_first_last_table.c1) ORDER BY [convert_first_last_table.c3 DESC NULLS FIRST]] 02)--CoalescePartitionsExec 03)----AggregateExec: mode=Partial, gby=[], aggr=[last_value(convert_first_last_table.c1) ORDER BY [convert_first_last_table.c3 ASC NULLS LAST]] -04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +04)------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/convert_first_last.csv]]}, projection=[c1, c3], output_orderings=[[c1@0 ASC NULLS LAST], [c3@1 ASC NULLS LAST]], has_header=true # test last to first @@ -5919,7 +5919,7 @@ physical_plan 01)AggregateExec: mode=Final, gby=[], aggr=[last_value(convert_first_last_table.c1) ORDER BY [convert_first_last_table.c2 ASC NULLS LAST]] 02)--CoalescePartitionsExec 03)----AggregateExec: mode=Partial, gby=[], aggr=[first_value(convert_first_last_table.c1) ORDER BY [convert_first_last_table.c2 DESC NULLS FIRST]] -04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +04)------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/convert_first_last.csv]]}, projection=[c1, c2], output_orderings=[[c1@0 ASC NULLS LAST], [c2@1 DESC]], has_header=true # test building plan with aggreagte sum @@ -5991,7 +5991,7 @@ physical_plan 03)----AggregateExec: mode=FinalPartitioned, gby=[v1@0 as v1, v2@1 as v2], aggr=[max(having_test.v1)] 04)------CoalesceBatchesExec: target_batch_size=8192 05)--------RepartitionExec: partitioning=Hash([v1@0, v2@1], 4), input_partitions=4 -06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +06)----------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 07)------------AggregateExec: mode=Partial, gby=[v1@0 as v1, v2@1 as v2], aggr=[max(having_test.v1)] 08)--------------MemoryExec: partitions=1, partition_sizes=[1] diff --git a/datafusion/sqllogictest/test_files/aggregates_topk.slt b/datafusion/sqllogictest/test_files/aggregates_topk.slt index a67fec695f6c..aaab7694e01a 100644 --- a/datafusion/sqllogictest/test_files/aggregates_topk.slt +++ b/datafusion/sqllogictest/test_files/aggregates_topk.slt @@ -49,7 +49,7 @@ physical_plan 03)----AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] 04)------CoalesceBatchesExec: target_batch_size=8192 05)--------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 -06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +06)----------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 07)------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] 08)--------------MemoryExec: partitions=1, partition_sizes=[1] @@ -113,7 +113,7 @@ physical_plan 03)----AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)], lim=[4] 04)------CoalesceBatchesExec: target_batch_size=8192 05)--------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 -06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +06)----------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 07)------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)], lim=[4] 08)--------------MemoryExec: partitions=1, partition_sizes=[1] @@ -130,7 +130,7 @@ physical_plan 03)----AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[min(traces.timestamp)] 04)------CoalesceBatchesExec: target_batch_size=8192 05)--------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 -06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +06)----------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 07)------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[min(traces.timestamp)] 08)--------------MemoryExec: partitions=1, partition_sizes=[1] @@ -147,7 +147,7 @@ physical_plan 03)----AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] 04)------CoalesceBatchesExec: target_batch_size=8192 05)--------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 -06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +06)----------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 07)------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] 08)--------------MemoryExec: partitions=1, partition_sizes=[1] @@ -164,7 +164,7 @@ physical_plan 03)----AggregateExec: mode=FinalPartitioned, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] 04)------CoalesceBatchesExec: target_batch_size=8192 05)--------RepartitionExec: partitioning=Hash([trace_id@0], 4), input_partitions=4 -06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +06)----------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 07)------------AggregateExec: mode=Partial, gby=[trace_id@0 as trace_id], aggr=[max(traces.timestamp)] 08)--------------MemoryExec: partitions=1, partition_sizes=[1] diff --git a/datafusion/sqllogictest/test_files/count_star_rule.slt b/datafusion/sqllogictest/test_files/count_star_rule.slt index 3625da68b39e..c2cd48ed009d 100644 --- a/datafusion/sqllogictest/test_files/count_star_rule.slt +++ b/datafusion/sqllogictest/test_files/count_star_rule.slt @@ -48,7 +48,7 @@ physical_plan 01)AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count()] 02)--CoalesceBatchesExec: target_batch_size=8192 03)----RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 -04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +04)------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 05)--------AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count()] 06)----------MemoryExec: partitions=1, partition_sizes=[1] @@ -67,7 +67,7 @@ physical_plan 04)------AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[count()] 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4 -07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +07)------------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 08)--------------AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[count()] 09)----------------MemoryExec: partitions=1, partition_sizes=[1] diff --git a/datafusion/sqllogictest/test_files/cte.slt b/datafusion/sqllogictest/test_files/cte.slt index 53ca8d81b9e4..8ac5e796943d 100644 --- a/datafusion/sqllogictest/test_files/cte.slt +++ b/datafusion/sqllogictest/test_files/cte.slt @@ -119,7 +119,7 @@ physical_plan 05)----ProjectionExec: expr=[id@0 + 1 as id] 06)------CoalesceBatchesExec: target_batch_size=8192 07)--------FilterExec: id@0 < 10 -08)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)----------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 09)------------WorkTableExec: name=nodes # setup @@ -165,7 +165,7 @@ physical_plan 05)------ProjectionExec: expr=[time@0 + 1 as time, name@1 as name, account_balance@2 + 10 as account_balance] 06)--------CoalesceBatchesExec: target_batch_size=2 07)----------FilterExec: time@0 < 10 -08)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)------------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 09)--------------WorkTableExec: name=balances # recursive CTE with static term derived from table works @@ -722,7 +722,7 @@ logical_plan 03)----Projection: Int64(1) AS val 04)------EmptyRelation 05)----Projection: Int64(2) AS val -06)------Cross Join: +06)------Cross Join: 07)--------Filter: recursive_cte.val < Int64(2) 08)----------TableScan: recursive_cte 09)--------SubqueryAlias: sub_cte @@ -737,7 +737,7 @@ physical_plan 06)------CoalescePartitionsExec 07)--------CoalesceBatchesExec: target_batch_size=8182 08)----------FilterExec: val@0 < 2 -09)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +09)------------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 10)--------------WorkTableExec: name=recursive_cte 11)------ProjectionExec: expr=[2 as val] 12)--------PlaceholderRowExec diff --git a/datafusion/sqllogictest/test_files/distinct_on.slt b/datafusion/sqllogictest/test_files/distinct_on.slt index cc0ebf83a843..e5de171c9804 100644 --- a/datafusion/sqllogictest/test_files/distinct_on.slt +++ b/datafusion/sqllogictest/test_files/distinct_on.slt @@ -101,7 +101,7 @@ physical_plan 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------RepartitionExec: partitioning=Hash([c1@0], 4), input_partitions=4 07)------------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[first_value(aggregate_test_100.c3) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c3 ASC NULLS LAST], first_value(aggregate_test_100.c2) ORDER BY [aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c3 ASC NULLS LAST]] -08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)--------------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 09)----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3], has_header=true # ON expressions are not a sub-set of the ORDER BY expressions diff --git a/datafusion/sqllogictest/test_files/explain.slt b/datafusion/sqllogictest/test_files/explain.slt index f3fee4f1fca6..ccaf51c30a67 100644 --- a/datafusion/sqllogictest/test_files/explain.slt +++ b/datafusion/sqllogictest/test_files/explain.slt @@ -45,7 +45,7 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: c2@1 > 10, projection=[c1@0] -03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +03)----OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2], has_header=true # explain_csv_exec_scan_config diff --git a/datafusion/sqllogictest/test_files/filter_without_sort_exec.slt b/datafusion/sqllogictest/test_files/filter_without_sort_exec.slt index d96044fda8c0..7d5b401fcf63 100644 --- a/datafusion/sqllogictest/test_files/filter_without_sort_exec.slt +++ b/datafusion/sqllogictest/test_files/filter_without_sort_exec.slt @@ -40,7 +40,7 @@ physical_plan 01)SortPreservingMergeExec: [date@0 ASC NULLS LAST, time@2 ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=8192 03)----FilterExec: ticker@1 = A -04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +04)------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 05)--------StreamingTableExec: partition_sizes=1, projection=[date, ticker, time], infinite_source=true, output_ordering=[date@0 ASC NULLS LAST, ticker@1 ASC NULLS LAST, time@2 ASC NULLS LAST] # constant ticker, CAST(time AS DATE) = time, order by time @@ -57,7 +57,7 @@ physical_plan 01)SortPreservingMergeExec: [time@2 ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=8192 03)----FilterExec: ticker@1 = A AND CAST(time@2 AS Date32) = date@0 -04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +04)------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 05)--------StreamingTableExec: partition_sizes=1, projection=[date, ticker, time], infinite_source=true, output_ordering=[date@0 ASC NULLS LAST, ticker@1 ASC NULLS LAST, time@2 ASC NULLS LAST] # same thing but order by date @@ -74,7 +74,7 @@ physical_plan 01)SortPreservingMergeExec: [date@0 ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=8192 03)----FilterExec: ticker@1 = A AND CAST(time@2 AS Date32) = date@0 -04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +04)------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 05)--------StreamingTableExec: partition_sizes=1, projection=[date, ticker, time], infinite_source=true, output_ordering=[date@0 ASC NULLS LAST, ticker@1 ASC NULLS LAST, time@2 ASC NULLS LAST] # same thing but order by ticker @@ -91,7 +91,7 @@ physical_plan 01)CoalescePartitionsExec 02)--CoalesceBatchesExec: target_batch_size=8192 03)----FilterExec: ticker@1 = A AND CAST(time@2 AS Date32) = date@0 -04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +04)------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 05)--------StreamingTableExec: partition_sizes=1, projection=[date, ticker, time], infinite_source=true, output_ordering=[date@0 ASC NULLS LAST, ticker@1 ASC NULLS LAST, time@2 ASC NULLS LAST] # same thing but order by time, date @@ -108,7 +108,7 @@ physical_plan 01)SortPreservingMergeExec: [time@2 ASC NULLS LAST, date@0 ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=8192 03)----FilterExec: ticker@1 = A AND CAST(time@2 AS Date32) = date@0 -04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +04)------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 05)--------StreamingTableExec: partition_sizes=1, projection=[date, ticker, time], infinite_source=true, output_ordering=[date@0 ASC NULLS LAST, ticker@1 ASC NULLS LAST, time@2 ASC NULLS LAST] # CAST(time AS DATE) <> date (should require a sort) @@ -149,5 +149,5 @@ physical_plan 01)SortPreservingMergeExec: [ticker@1 ASC NULLS LAST, time@2 ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=8192 03)----FilterExec: date@0 = 2006-01-02 -04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +04)------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 05)--------StreamingTableExec: partition_sizes=1, projection=[date, ticker, time], infinite_source=true, output_ordering=[date@0 ASC NULLS LAST, ticker@1 ASC NULLS LAST, time@2 ASC NULLS LAST] diff --git a/datafusion/sqllogictest/test_files/group_by.slt b/datafusion/sqllogictest/test_files/group_by.slt index df7e21c2da44..b4b57aee5fcf 100644 --- a/datafusion/sqllogictest/test_files/group_by.slt +++ b/datafusion/sqllogictest/test_files/group_by.slt @@ -2020,7 +2020,7 @@ physical_plan 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------RepartitionExec: partitioning=Hash([col0@0, col1@1, col2@2], 4), input_partitions=4 07)------------AggregateExec: mode=Partial, gby=[col0@0 as col0, col1@1 as col1, col2@2 as col2], aggr=[last_value(r.col1) ORDER BY [r.col0 ASC NULLS LAST]] -08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)--------------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 09)----------------ProjectionExec: expr=[col0@2 as col0, col1@3 as col1, col2@4 as col2, col0@0 as col0, col1@1 as col1] 10)------------------CoalesceBatchesExec: target_batch_size=8192 11)--------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(col0@0, col0@0)] @@ -2922,7 +2922,7 @@ physical_plan 04)------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[first_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], last_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]] 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=8 -07)------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +07)------------OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1 08)--------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[first_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], last_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]] 09)----------------MemoryExec: partitions=1, partition_sizes=[1] @@ -2958,7 +2958,7 @@ physical_plan 04)------AggregateExec: mode=FinalPartitioned, gby=[country@0 as country], aggr=[first_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], last_value(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]] 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=8 -07)------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +07)------------OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1 08)--------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[first_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], last_value(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]] 09)----------------MemoryExec: partitions=1, partition_sizes=[1] @@ -2995,7 +2995,7 @@ physical_plan 02)--AggregateExec: mode=Final, gby=[], aggr=[first_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], last_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]] 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[], aggr=[first_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], last_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]] -05)--------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +05)--------OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1 06)----------MemoryExec: partitions=1, partition_sizes=[1] query RR @@ -3021,7 +3021,7 @@ physical_plan 02)--AggregateExec: mode=Final, gby=[], aggr=[first_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], last_value(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]] 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[], aggr=[first_value(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST], last_value(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]] -05)--------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +05)--------OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1 06)----------MemoryExec: partitions=1, partition_sizes=[1] query RR @@ -3046,7 +3046,7 @@ physical_plan 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[], aggr=[array_agg(sales_global.amount) ORDER BY [sales_global.ts ASC NULLS LAST]] 05)--------SortExec: expr=[ts@0 ASC NULLS LAST], preserve_partitioning=[true] -06)----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +06)----------OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1 07)------------MemoryExec: partitions=1, partition_sizes=[1] query ? @@ -3070,7 +3070,7 @@ physical_plan 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[], aggr=[array_agg(sales_global.amount) ORDER BY [sales_global.ts DESC NULLS FIRST]] 05)--------SortExec: expr=[ts@0 DESC], preserve_partitioning=[true] -06)----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +06)----------OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1 07)------------MemoryExec: partitions=1, partition_sizes=[1] query ? @@ -3094,7 +3094,7 @@ physical_plan 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[], aggr=[array_agg(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]] 05)--------SortExec: expr=[amount@0 ASC NULLS LAST], preserve_partitioning=[true] -06)----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +06)----------OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1 07)------------MemoryExec: partitions=1, partition_sizes=[1] query ? @@ -3124,7 +3124,7 @@ physical_plan 06)----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=8 07)------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[array_agg(sales_global.amount) ORDER BY [sales_global.amount ASC NULLS LAST]] 08)--------------SortExec: expr=[amount@1 ASC NULLS LAST], preserve_partitioning=[true] -09)----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +09)----------------OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1 10)------------------MemoryExec: partitions=1, partition_sizes=[1] query T? @@ -3160,7 +3160,7 @@ physical_plan 06)----------RepartitionExec: partitioning=Hash([country@0], 8), input_partitions=8 07)------------AggregateExec: mode=Partial, gby=[country@0 as country], aggr=[array_agg(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST], last_value(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST], last_value(sales_global.amount) ORDER BY [sales_global.amount DESC NULLS FIRST]] 08)--------------SortExec: expr=[amount@1 DESC], preserve_partitioning=[true] -09)----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +09)----------------OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1 10)------------------MemoryExec: partitions=1, partition_sizes=[1] query T?RR @@ -3360,7 +3360,7 @@ physical_plan 05)--------CoalesceBatchesExec: target_batch_size=4 06)----------RepartitionExec: partitioning=Hash([sn@0, amount@1], 8), input_partitions=8 07)------------AggregateExec: mode=Partial, gby=[sn@0 as sn, amount@1 as amount], aggr=[] -08)--------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +08)--------------OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1 09)----------------MemoryExec: partitions=1, partition_sizes=[1] query IRI @@ -3432,7 +3432,7 @@ physical_plan 08)--------------ProjectionExec: expr=[amount@1 as amount, sn@2 as sn, amount@3 as amount] 09)----------------NestedLoopJoinExec: join_type=Inner, filter=sn@0 >= sn@1 10)------------------MemoryExec: partitions=1, partition_sizes=[1] -11)------------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +11)------------------OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1 12)--------------------MemoryExec: partitions=1, partition_sizes=[1] query IRR @@ -3577,7 +3577,7 @@ physical_plan 05)--------CoalesceBatchesExec: target_batch_size=4 06)----------RepartitionExec: partitioning=Hash([sn@0, zip_code@1, country@2, ts@3, currency@4, amount@5, sum_amount@6], 8), input_partitions=8 07)------------AggregateExec: mode=Partial, gby=[sn@2 as sn, zip_code@0 as zip_code, country@1 as country, ts@3 as ts, currency@4 as currency, amount@5 as amount, sum_amount@6 as sum_amount], aggr=[] -08)--------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +08)--------------OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1 09)----------------ProjectionExec: expr=[zip_code@0 as zip_code, country@1 as country, sn@2 as sn, ts@3 as ts, currency@4 as currency, amount@5 as amount, sum(l.amount) ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@6 as sum_amount] 10)------------------BoundedWindowAggExec: wdw=[sum(l.amount) ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING: Ok(Field { name: "sum(l.amount) ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING", data_type: Float64, 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] 11)--------------------MemoryExec: partitions=1, partition_sizes=[1] @@ -3762,7 +3762,7 @@ physical_plan 01)AggregateExec: mode=Final, gby=[], aggr=[last_value(foo.x)] 02)--CoalescePartitionsExec 03)----AggregateExec: mode=Partial, gby=[], aggr=[last_value(foo.x)] -04)------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +04)------OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1 05)--------MemoryExec: partitions=1, partition_sizes=[1] query I @@ -3784,7 +3784,7 @@ physical_plan 01)AggregateExec: mode=Final, gby=[], aggr=[first_value(foo.x)] 02)--CoalescePartitionsExec 03)----AggregateExec: mode=Partial, gby=[], aggr=[first_value(foo.x)] -04)------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +04)------OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1 05)--------MemoryExec: partitions=1, partition_sizes=[1] # Since both ordering requirements are satisfied, there shouldn't be @@ -3805,7 +3805,7 @@ physical_plan 03)----CoalesceBatchesExec: target_batch_size=2 04)------RepartitionExec: partitioning=Hash([d@0], 8), input_partitions=8 05)--------AggregateExec: mode=Partial, gby=[d@2 as d], aggr=[first_value(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.a ASC NULLS LAST], first_value(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.c ASC NULLS LAST]] -06)----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +06)----------OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1 07)------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c, d], output_orderings=[[a@0 ASC NULLS LAST], [c@1 ASC NULLS LAST]], has_header=true query II rowsort @@ -3917,7 +3917,7 @@ physical_plan 03)----CoalesceBatchesExec: target_batch_size=2 04)------RepartitionExec: partitioning=Hash([c@0, b@1], 8), input_partitions=8 05)--------AggregateExec: mode=Partial, gby=[c@1 as c, b@0 as b], aggr=[sum(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallySorted([0]) -06)----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +06)----------OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1 07)------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], has_header=true # drop table multiple_ordered_table_with_pk @@ -3958,7 +3958,7 @@ physical_plan 03)----CoalesceBatchesExec: target_batch_size=2 04)------RepartitionExec: partitioning=Hash([c@0, b@1], 8), input_partitions=8 05)--------AggregateExec: mode=Partial, gby=[c@1 as c, b@0 as b], aggr=[sum(multiple_ordered_table_with_pk.d)], ordering_mode=PartiallySorted([0]) -06)----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +06)----------OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1 07)------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[b, c, d], output_ordering=[c@1 ASC NULLS LAST], has_header=true statement ok @@ -4179,7 +4179,7 @@ physical_plan 02)--AggregateExec: mode=FinalPartitioned, gby=[y@0 as y], aggr=[sum(DISTINCT t1.x), max(DISTINCT t1.x)] 03)----CoalesceBatchesExec: target_batch_size=2 04)------RepartitionExec: partitioning=Hash([y@0], 8), input_partitions=8 -05)--------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +05)--------OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1 06)----------AggregateExec: mode=Partial, gby=[y@1 as y], aggr=[sum(DISTINCT t1.x), max(DISTINCT t1.x)] 07)------------MemoryExec: partitions=1, partition_sizes=[1] @@ -4201,7 +4201,7 @@ physical_plan 06)----------AggregateExec: mode=FinalPartitioned, gby=[y@0 as y, alias1@1 as alias1], aggr=[] 07)------------CoalesceBatchesExec: target_batch_size=2 08)--------------RepartitionExec: partitioning=Hash([y@0, alias1@1], 8), input_partitions=8 -09)----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +09)----------------OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1 10)------------------AggregateExec: mode=Partial, gby=[y@1 as y, __common_expr_1@0 as alias1], aggr=[] 11)--------------------ProjectionExec: expr=[CAST(x@0 AS Float64) as __common_expr_1, y@1 as y] 12)----------------------MemoryExec: partitions=1, partition_sizes=[1] @@ -4236,7 +4236,7 @@ physical_plan 04)------CoalesceBatchesExec: target_batch_size=2 05)--------RepartitionExec: partitioning=Hash([date_bin(Utf8("15 minutes"),unbounded_csv_with_timestamps.ts)@0], 8), input_partitions=8, preserve_order=true, sort_exprs=date_bin(Utf8("15 minutes"),unbounded_csv_with_timestamps.ts)@0 DESC 06)----------AggregateExec: mode=Partial, gby=[date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 900000000000 }, ts@0) as date_bin(Utf8("15 minutes"),unbounded_csv_with_timestamps.ts)], aggr=[], ordering_mode=Sorted -07)------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +07)------------OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1 08)--------------StreamingTableExec: partition_sizes=1, projection=[ts], infinite_source=true, output_ordering=[ts@0 DESC] query P @@ -4291,7 +4291,7 @@ physical_plan 05)--------CoalesceBatchesExec: target_batch_size=2 06)----------RepartitionExec: partitioning=Hash([date_part(Utf8("MONTH"),csv_with_timestamps.ts)@0], 8), input_partitions=8 07)------------AggregateExec: mode=Partial, gby=[date_part(MONTH, ts@0) as date_part(Utf8("MONTH"),csv_with_timestamps.ts)], aggr=[] -08)--------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +08)--------------OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1 09)----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/timestamps.csv]]}, projection=[ts], output_ordering=[ts@0 DESC], has_header=false query I @@ -4331,7 +4331,7 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [name@0 DESC, time_chunks@1 DESC], fetch=5 02)--ProjectionExec: expr=[name@0 as name, date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 900000000000 }, ts@1) as time_chunks] -03)----RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +03)----OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1 04)------StreamingTableExec: partition_sizes=1, projection=[name, ts], infinite_source=true, output_ordering=[name@0 DESC, ts@1 DESC] statement ok @@ -4407,7 +4407,7 @@ physical_plan 09)----------------CoalesceBatchesExec: target_batch_size=2 10)------------------RepartitionExec: partitioning=Hash([c1@0, alias1@1], 8), input_partitions=8 11)--------------------AggregateExec: mode=Partial, gby=[c1@0 as c1, c2@1 as alias1], aggr=[alias2, alias3] -12)----------------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +12)----------------------OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1 13)------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3, c4], has_header=true # Use PostgreSQL dialect @@ -4580,7 +4580,7 @@ physical_plan 04)------CoalesceBatchesExec: target_batch_size=2 05)--------RepartitionExec: partitioning=Hash([c2@0], 8), input_partitions=8 06)----------AggregateExec: mode=Partial, gby=[c2@1 as c2], aggr=[max(timestamp_table.t1)], lim=[4] -07)------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=4 +07)------------OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=4 08)--------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/group_by/timestamp_table/0.csv], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/group_by/timestamp_table/1.csv], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/group_by/timestamp_table/2.csv], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/group_by/timestamp_table/3.csv]]}, projection=[t1, c2], has_header=true # Clean up diff --git a/datafusion/sqllogictest/test_files/information_schema.slt b/datafusion/sqllogictest/test_files/information_schema.slt index 4d51a61c8a52..b12b0ad9e2ef 100644 --- a/datafusion/sqllogictest/test_files/information_schema.slt +++ b/datafusion/sqllogictest/test_files/information_schema.slt @@ -238,6 +238,7 @@ datafusion.explain.show_statistics false datafusion.optimizer.allow_symmetric_joins_without_pruning true datafusion.optimizer.default_filter_selectivity 20 datafusion.optimizer.enable_distinct_aggregation_soft_limit true +datafusion.optimizer.enable_on_demand_repartition true datafusion.optimizer.enable_round_robin_repartition true datafusion.optimizer.enable_topk_aggregation true datafusion.optimizer.expand_views_at_output false @@ -331,6 +332,7 @@ datafusion.explain.show_statistics false When set to true, the explain statement datafusion.optimizer.allow_symmetric_joins_without_pruning true Should DataFusion allow symmetric hash joins for unbounded data sources even when its inputs do not have any ordering or filtering If the flag is not enabled, the SymmetricHashJoin operator will be unable to prune its internal buffers, resulting in certain join types - such as Full, Left, LeftAnti, LeftSemi, Right, RightAnti, and RightSemi - being produced only at the end of the execution. This is not typical in stream processing. Additionally, without proper design for long runner execution, all types of joins may encounter out-of-memory errors. datafusion.optimizer.default_filter_selectivity 20 The default filter selectivity used by Filter Statistics when an exact selectivity cannot be determined. Valid values are between 0 (no selectivity) and 100 (all rows are selected). 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_on_demand_repartition true (empty) 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.enable_topk_aggregation true When set to true, the optimizer will attempt to perform limit operations during aggregations, if possible datafusion.optimizer.expand_views_at_output false When set to true, if the returned type is a view type then the output will be coerced to a non-view. Coerces `Utf8View` to `LargeUtf8`, and `BinaryView` to `LargeBinary`. diff --git a/datafusion/sqllogictest/test_files/insert.slt b/datafusion/sqllogictest/test_files/insert.slt index 804612287246..edde3cb40b79 100644 --- a/datafusion/sqllogictest/test_files/insert.slt +++ b/datafusion/sqllogictest/test_files/insert.slt @@ -72,7 +72,7 @@ physical_plan 06)----------SortExec: expr=[c1@0 ASC NULLS LAST, c9@2 ASC NULLS LAST], preserve_partitioning=[true] 07)------------CoalesceBatchesExec: target_batch_size=8192 08)--------------RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8 -09)----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +09)----------------OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1 10)------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c4, c9], has_header=true query I @@ -132,7 +132,7 @@ physical_plan 05)--------SortExec: expr=[c1@0 ASC NULLS LAST, c9@2 ASC NULLS LAST], preserve_partitioning=[true] 06)----------CoalesceBatchesExec: target_batch_size=8192 07)------------RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8 -08)--------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +08)--------------OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1 09)----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c4, c9], has_header=true @@ -183,7 +183,7 @@ physical_plan 06)----------SortExec: expr=[c1@0 ASC NULLS LAST, c9@2 ASC NULLS LAST], preserve_partitioning=[true] 07)------------CoalesceBatchesExec: target_batch_size=8192 08)--------------RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8 -09)----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +09)----------------OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1 10)------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c4, c9], has_header=true diff --git a/datafusion/sqllogictest/test_files/insert_to_external.slt b/datafusion/sqllogictest/test_files/insert_to_external.slt index edfc2ee75bd7..7b9f4f7d0529 100644 --- a/datafusion/sqllogictest/test_files/insert_to_external.slt +++ b/datafusion/sqllogictest/test_files/insert_to_external.slt @@ -361,7 +361,7 @@ physical_plan 06)----------SortExec: expr=[c1@0 ASC NULLS LAST, c9@2 ASC NULLS LAST], preserve_partitioning=[true] 07)------------CoalesceBatchesExec: target_batch_size=8192 08)--------------RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8 -09)----------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +09)----------------OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1 10)------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c4, c9], has_header=true query I @@ -422,7 +422,7 @@ physical_plan 05)--------SortExec: expr=[c1@0 ASC NULLS LAST, c9@2 ASC NULLS LAST], preserve_partitioning=[true] 06)----------CoalesceBatchesExec: target_batch_size=8192 07)------------RepartitionExec: partitioning=Hash([c1@0], 8), input_partitions=8 -08)--------------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +08)--------------OnDemandRepartitionExec: partitioning=OnDemand(8), input_partitions=1 09)----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c4, c9], has_header=true diff --git a/datafusion/sqllogictest/test_files/join.slt b/datafusion/sqllogictest/test_files/join.slt index 1feacc5ebe53..8511b1c5dc35 100644 --- a/datafusion/sqllogictest/test_files/join.slt +++ b/datafusion/sqllogictest/test_files/join.slt @@ -671,13 +671,13 @@ 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 01)CrossJoinExec 02)--MemoryExec: partitions=1, partition_sizes=[1] -03)--RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +03)--OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 04)----MemoryExec: partitions=1, partition_sizes=[1] statement ok @@ -963,7 +963,7 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: dept_name@2 != Engineering AND name@1 = Alice OR name@1 != Alice AND name@1 = Carol -03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +03)----OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 04)------CoalesceBatchesExec: target_batch_size=8192 05)--------HashJoinExec: mode=CollectLeft, join_type=Left, on=[(emp_id@0, emp_id@0)], projection=[emp_id@0, name@1, dept_name@3] 06)----------CoalesceBatchesExec: target_batch_size=8192 @@ -1161,7 +1161,7 @@ physical_plan 02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(CAST(t1.v0 AS Float64)@6, v1@1)], filter=v1@1 + CAST(v0@0 AS Float64) > 0, projection=[v0@0, v1@1, v2@3, v3@4, v4@5, v0@7, v1@8] 03)----CoalescePartitionsExec 04)------ProjectionExec: expr=[v0@0 as v0, v1@1 as v1, v0@2 as v0, v2@3 as v2, v3@4 as v3, v4@5 as v4, CAST(v0@0 AS Float64) as CAST(t1.v0 AS Float64)] -05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +05)--------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 06)----------CoalesceBatchesExec: target_batch_size=8192 07)------------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(v0@0, v0@0), (v1@1, v1@1)], projection=[v0@0, v1@1, v0@2, v2@4, v3@5, v4@6] 08)--------------MemoryExec: partitions=1, partition_sizes=[0] diff --git a/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt b/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt index cf897d628da5..68b70050eee2 100644 --- a/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt +++ b/datafusion/sqllogictest/test_files/join_disable_repartition_joins.slt @@ -58,7 +58,7 @@ physical_plan 02)--CoalesceBatchesExec: target_batch_size=8192, fetch=5 03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c@0, c@1)], projection=[a@1] 04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], has_header=true -05)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +05)------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 06)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c], output_ordering=[a@0 ASC NULLS LAST], has_header=true # preserve_inner_join @@ -101,7 +101,7 @@ physical_plan 05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], has_header=true 06)--------CoalesceBatchesExec: target_batch_size=8192 07)----------FilterExec: d@3 = 3 -08)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)------------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 09)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true # preserve_right_semi_join diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index 49aaa877caa6..33116b6dac5e 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -1343,11 +1343,11 @@ physical_plan 03)----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t1_id@0, t2_id@0)], projection=[t1_id@0] 04)------CoalesceBatchesExec: target_batch_size=2 05)--------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 -06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +06)----------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 07)------------MemoryExec: partitions=1, partition_sizes=[1] 08)------CoalesceBatchesExec: target_batch_size=2 09)--------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 -10)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +10)----------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 11)------------MemoryExec: partitions=1, partition_sizes=[1] # Join on struct @@ -1365,11 +1365,11 @@ physical_plan 02)--HashJoinExec: mode=Partitioned, join_type=Inner, on=[(s3@0, s4@0)] 03)----CoalesceBatchesExec: target_batch_size=2 04)------RepartitionExec: partitioning=Hash([s3@0], 2), input_partitions=2 -05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +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([s4@0], 2), input_partitions=2 -09)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +09)--------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 10)----------MemoryExec: partitions=1, partition_sizes=[1] query ?? @@ -1408,11 +1408,11 @@ physical_plan 04)------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t1_id@0, t2_id@0)], projection=[t1_id@0] 05)--------CoalesceBatchesExec: target_batch_size=2 06)----------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 -07)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +07)------------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 08)--------------MemoryExec: partitions=1, partition_sizes=[1] 09)--------CoalesceBatchesExec: target_batch_size=2 10)----------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 -11)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +11)------------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 12)--------------MemoryExec: partitions=1, partition_sizes=[1] query TT @@ -1439,11 +1439,11 @@ physical_plan 07)------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t1_id@0, t2_id@0)], projection=[t1_id@0] 08)--------------CoalesceBatchesExec: target_batch_size=2 09)----------------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 -10)------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +10)------------------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 11)--------------------MemoryExec: partitions=1, partition_sizes=[1] 12)--------------CoalesceBatchesExec: target_batch_size=2 13)----------------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 -14)------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +14)------------------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 15)--------------------MemoryExec: partitions=1, partition_sizes=[1] statement ok @@ -1507,10 +1507,10 @@ physical_plan 03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t1.t1_id + Int64(11)@3, CAST(join_t2.t2_id AS Int64)@3)], projection=[t1_id@0, t1_name@1, t1_int@2, t2_id@4, t2_name@5, t2_int@6] 04)------CoalescePartitionsExec 05)--------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] -06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +06)----------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 07)------------MemoryExec: partitions=1, partition_sizes=[1] 08)------ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, CAST(t2_id@0 AS Int64) as CAST(join_t2.t2_id AS Int64)] -09)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +09)--------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 10)----------MemoryExec: partitions=1, partition_sizes=[1] statement ok @@ -1534,12 +1534,12 @@ physical_plan 04)------CoalesceBatchesExec: target_batch_size=2 05)--------RepartitionExec: partitioning=Hash([join_t1.t1_id + Int64(11)@3], 2), input_partitions=2 06)----------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int, CAST(t1_id@0 AS Int64) + 11 as join_t1.t1_id + Int64(11)] -07)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +07)------------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 08)--------------MemoryExec: partitions=1, partition_sizes=[1] 09)------CoalesceBatchesExec: target_batch_size=2 10)--------RepartitionExec: partitioning=Hash([CAST(join_t2.t2_id AS Int64)@3], 2), input_partitions=2 11)----------ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, CAST(t2_id@0 AS Int64) as CAST(join_t2.t2_id AS Int64)] -12)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +12)------------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 13)--------------MemoryExec: partitions=1, partition_sizes=[1] # Both side expr key inner join @@ -1564,10 +1564,10 @@ physical_plan 03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t2.t2_id + UInt32(1)@1, join_t1.t1_id + UInt32(12)@2)], projection=[t2_id@0, t1_id@2, t1_name@3] 04)------CoalescePartitionsExec 05)--------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 + 1 as join_t2.t2_id + UInt32(1)] -06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +06)----------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 07)------------MemoryExec: partitions=1, partition_sizes=[1] 08)------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 12 as join_t1.t1_id + UInt32(12)] -09)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +09)--------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 10)----------MemoryExec: partitions=1, partition_sizes=[1] statement ok @@ -1591,12 +1591,12 @@ physical_plan 04)------CoalesceBatchesExec: target_batch_size=2 05)--------RepartitionExec: partitioning=Hash([join_t2.t2_id + UInt32(1)@1], 2), input_partitions=2 06)----------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 + 1 as join_t2.t2_id + UInt32(1)] -07)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +07)------------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 08)--------------MemoryExec: partitions=1, partition_sizes=[1] 09)------CoalesceBatchesExec: target_batch_size=2 10)--------RepartitionExec: partitioning=Hash([join_t1.t1_id + UInt32(12)@2], 2), input_partitions=2 11)----------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 12 as join_t1.t1_id + UInt32(12)] -12)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +12)------------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 13)--------------MemoryExec: partitions=1, partition_sizes=[1] # Left side expr key inner join @@ -1622,7 +1622,7 @@ physical_plan 03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t2_id@0, join_t1.t1_id + UInt32(11)@2)], projection=[t2_id@0, t1_id@1, t1_name@2] 04)------MemoryExec: partitions=1, partition_sizes=[1] 05)------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 11 as join_t1.t1_id + UInt32(11)] -06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +06)--------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 07)----------MemoryExec: partitions=1, partition_sizes=[1] statement ok @@ -1646,12 +1646,12 @@ physical_plan 03)----HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t2_id@0, join_t1.t1_id + UInt32(11)@2)], projection=[t2_id@0, t1_id@1, t1_name@2] 04)------CoalesceBatchesExec: target_batch_size=2 05)--------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 -06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +06)----------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 07)------------MemoryExec: partitions=1, partition_sizes=[1] 08)------CoalesceBatchesExec: target_batch_size=2 09)--------RepartitionExec: partitioning=Hash([join_t1.t1_id + UInt32(11)@2], 2), input_partitions=2 10)----------ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_id@0 + 11 as join_t1.t1_id + UInt32(11)] -11)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +11)------------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 12)--------------MemoryExec: partitions=1, partition_sizes=[1] # Right side expr key inner join @@ -1677,9 +1677,9 @@ physical_plan 03)----HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(join_t2.t2_id - UInt32(11)@1, t1_id@0)], projection=[t2_id@0, t1_id@2, t1_name@3] 04)------CoalescePartitionsExec 05)--------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] -06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +06)----------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 07)------------MemoryExec: partitions=1, partition_sizes=[1] -08)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +08)------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 09)--------MemoryExec: partitions=1, partition_sizes=[1] statement ok @@ -1704,11 +1704,11 @@ physical_plan 04)------CoalesceBatchesExec: target_batch_size=2 05)--------RepartitionExec: partitioning=Hash([join_t2.t2_id - UInt32(11)@1], 2), input_partitions=2 06)----------ProjectionExec: expr=[t2_id@0 as t2_id, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] -07)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +07)------------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 08)--------------MemoryExec: partitions=1, partition_sizes=[1] 09)------CoalesceBatchesExec: target_batch_size=2 10)--------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 -11)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +11)----------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 12)------------MemoryExec: partitions=1, partition_sizes=[1] # Select wildcard with expr key inner join @@ -1732,7 +1732,7 @@ physical_plan 02)--HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(t1_id@0, join_t2.t2_id - UInt32(11)@3)], projection=[t1_id@0, t1_name@1, t1_int@2, t2_id@3, t2_name@4, t2_int@5] 03)----MemoryExec: partitions=1, partition_sizes=[1] 04)----ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] -05)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +05)------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 06)--------MemoryExec: partitions=1, partition_sizes=[1] statement ok @@ -1754,12 +1754,12 @@ physical_plan 02)--HashJoinExec: mode=Partitioned, join_type=Inner, on=[(t1_id@0, join_t2.t2_id - UInt32(11)@3)], projection=[t1_id@0, t1_name@1, t1_int@2, t2_id@3, t2_name@4, t2_int@5] 03)----CoalesceBatchesExec: target_batch_size=2 04)------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 -05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +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([join_t2.t2_id - UInt32(11)@3], 2), input_partitions=2 09)--------ProjectionExec: expr=[t2_id@0 as t2_id, t2_name@1 as t2_name, t2_int@2 as t2_int, t2_id@0 - 11 as join_t2.t2_id - UInt32(11)] -10)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +10)----------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 11)------------MemoryExec: partitions=1, partition_sizes=[1] ##### @@ -2084,11 +2084,11 @@ physical_plan 03)----CoalescePartitionsExec 04)------CoalesceBatchesExec: target_batch_size=2 05)--------FilterExec: t2_int@1 > 1, projection=[t2_id@0] -06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +06)----------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 07)------------MemoryExec: partitions=1, partition_sizes=[1] 08)----CoalesceBatchesExec: target_batch_size=2 09)------FilterExec: t1_id@0 > 10 -10)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +10)--------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 11)----------MemoryExec: partitions=1, partition_sizes=[1] query II @@ -2123,11 +2123,11 @@ physical_plan 02)--CoalescePartitionsExec 03)----CoalesceBatchesExec: target_batch_size=2 04)------FilterExec: t1_id@0 > 22 -05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +05)--------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 06)----------MemoryExec: partitions=1, partition_sizes=[1] 07)--CoalesceBatchesExec: target_batch_size=2 08)----FilterExec: t2_id@0 > 11 -09)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +09)------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 10)--------MemoryExec: partitions=1, partition_sizes=[1] query II @@ -2601,11 +2601,11 @@ physical_plan 02)--HashJoinExec: mode=Partitioned, join_type=Inner, on=[(millis@2, millis@2)] 03)----CoalesceBatchesExec: target_batch_size=2 04)------RepartitionExec: partitioning=Hash([millis@2], 2), input_partitions=2 -05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +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([millis@2], 2), input_partitions=2 -09)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +09)--------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 10)----------MemoryExec: partitions=1, partition_sizes=[1] # left_join_using_2 @@ -2774,12 +2774,12 @@ physical_plan 02)--SortExec: expr=[c1@0 ASC], preserve_partitioning=[true] 03)----CoalesceBatchesExec: target_batch_size=2 04)------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 -05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +05)--------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 06)----------MemoryExec: partitions=1, partition_sizes=[1] 07)--SortExec: expr=[c1@0 ASC], preserve_partitioning=[true] 08)----CoalesceBatchesExec: target_batch_size=2 09)------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 -10)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +10)--------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 11)----------MemoryExec: partitions=1, partition_sizes=[1] # sort_merge_join_on_date32 inner sort merge join on data type (Date32) @@ -2806,12 +2806,12 @@ physical_plan 04)------CoalesceBatchesExec: target_batch_size=2 05)--------RepartitionExec: partitioning=Hash([CAST(t1.c3 AS Decimal128(10, 2))@4], 2), input_partitions=2 06)----------ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, c3@2 as c3, c4@3 as c4, CAST(c3@2 AS Decimal128(10, 2)) as CAST(t1.c3 AS Decimal128(10, 2))] -07)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +07)------------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 08)--------------MemoryExec: partitions=1, partition_sizes=[1] 09)----SortExec: expr=[c3@2 ASC], preserve_partitioning=[true] 10)------CoalesceBatchesExec: target_batch_size=2 11)--------RepartitionExec: partitioning=Hash([c3@2], 2), input_partitions=2 -12)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +12)----------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 13)------------MemoryExec: partitions=1, partition_sizes=[1] # sort_merge_join_on_decimal right join on data type (Decimal) @@ -2868,12 +2868,12 @@ physical_plan 03)----HashJoinExec: mode=Partitioned, join_type=RightSemi, on=[(t2_id@0, t1_id@0)] 04)------CoalesceBatchesExec: target_batch_size=2 05)--------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 -06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +06)----------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 07)------------MemoryExec: partitions=1, partition_sizes=[1] 08)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] 09)--------CoalesceBatchesExec: target_batch_size=2 10)----------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 -11)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +11)------------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 12)--------------MemoryExec: partitions=1, partition_sizes=[1] query IT rowsort @@ -2909,12 +2909,12 @@ physical_plan 03)----HashJoinExec: mode=Partitioned, join_type=RightSemi, on=[(t2_id@0, t1_id@0)] 04)------CoalesceBatchesExec: target_batch_size=2 05)--------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 -06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +06)----------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 07)------------MemoryExec: partitions=1, partition_sizes=[1] 08)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] 09)--------CoalesceBatchesExec: target_batch_size=2 10)----------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 -11)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +11)------------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 12)--------------MemoryExec: partitions=1, partition_sizes=[1] query IT @@ -2971,7 +2971,7 @@ physical_plan 03)----HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)] 04)------MemoryExec: partitions=1, partition_sizes=[1] 05)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] -06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +06)--------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 07)----------MemoryExec: partitions=1, partition_sizes=[1] query IT rowsort @@ -3007,7 +3007,7 @@ physical_plan 03)----HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)] 04)------MemoryExec: partitions=1, partition_sizes=[1] 05)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] -06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +06)--------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 07)----------MemoryExec: partitions=1, partition_sizes=[1] query IT @@ -3065,12 +3065,12 @@ physical_plan 03)----HashJoinExec: mode=Partitioned, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], filter=t2_name@1 != t1_name@0 04)------CoalesceBatchesExec: target_batch_size=2 05)--------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 -06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +06)----------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 07)------------MemoryExec: partitions=1, partition_sizes=[1] 08)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] 09)--------CoalesceBatchesExec: target_batch_size=2 10)----------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 -11)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +11)------------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 12)--------------MemoryExec: partitions=1, partition_sizes=[1] query ITI rowsort @@ -3087,12 +3087,12 @@ physical_plan 03)----HashJoinExec: mode=Partitioned, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], filter=t2_name@0 != t1_name@1 04)------CoalesceBatchesExec: target_batch_size=2 05)--------RepartitionExec: partitioning=Hash([t2_id@0], 2), input_partitions=2 -06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +06)----------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 07)------------MemoryExec: partitions=1, partition_sizes=[1] 08)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] 09)--------CoalesceBatchesExec: target_batch_size=2 10)----------RepartitionExec: partitioning=Hash([t1_id@0], 2), input_partitions=2 -11)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +11)------------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 12)--------------MemoryExec: partitions=1, partition_sizes=[1] query ITI rowsort @@ -3147,7 +3147,7 @@ physical_plan 03)----HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], filter=t2_name@1 != t1_name@0 04)------MemoryExec: partitions=1, partition_sizes=[1] 05)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] -06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +06)--------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 07)----------MemoryExec: partitions=1, partition_sizes=[1] query ITI rowsort @@ -3164,7 +3164,7 @@ physical_plan 03)----HashJoinExec: mode=CollectLeft, join_type=RightSemi, on=[(t2_id@0, t1_id@0)], filter=t2_name@0 != t1_name@1 04)------MemoryExec: partitions=1, partition_sizes=[1] 05)------SortExec: expr=[t1_id@0 ASC NULLS LAST], preserve_partitioning=[true] -06)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +06)--------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 07)----------MemoryExec: partitions=1, partition_sizes=[1] query ITI rowsort @@ -3244,13 +3244,13 @@ physical_plan 02)--SortMergeJoin: join_type=Inner, on=[(a@1, a@1)] 03)----CoalesceBatchesExec: target_batch_size=2 04)------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST, rn1@5 ASC NULLS LAST -05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +05)--------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 06)----------ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@5 as rn1] 07)------------BoundedWindowAggExec: wdw=[row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] 08)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true 09)----CoalesceBatchesExec: target_batch_size=2 10)------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST -11)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +11)--------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 12)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true # sort merge join should propagate ordering equivalence of the right side @@ -3278,11 +3278,11 @@ physical_plan 02)--SortMergeJoin: join_type=Right, on=[(a@1, a@1)] 03)----CoalesceBatchesExec: target_batch_size=2 04)------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST -05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +05)--------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true 07)----CoalesceBatchesExec: target_batch_size=2 08)------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST, rn1@5 ASC NULLS LAST -09)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +09)--------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 10)----------ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@5 as rn1] 11)------------BoundedWindowAggExec: wdw=[row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] 12)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true @@ -3321,14 +3321,14 @@ physical_plan 04)------SortExec: expr=[a@1 ASC], preserve_partitioning=[true] 05)--------CoalesceBatchesExec: target_batch_size=2 06)----------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2 -07)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +07)------------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 08)--------------ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@5 as rn1] 09)----------------BoundedWindowAggExec: wdw=[row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] 10)------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true 11)------SortExec: expr=[a@1 ASC], preserve_partitioning=[true] 12)--------CoalesceBatchesExec: target_batch_size=2 13)----------RepartitionExec: partitioning=Hash([a@1], 2), input_partitions=2 -14)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +14)------------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 15)--------------ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@5 as rn1] 16)----------------BoundedWindowAggExec: wdw=[row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "row_number() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] 17)------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true @@ -3507,11 +3507,11 @@ physical_plan 09)----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a@0)] 10)------------------CoalesceBatchesExec: target_batch_size=2 11)--------------------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 -12)----------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +12)----------------------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 13)------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], output_ordering=[a@0 ASC, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true 14)------------------CoalesceBatchesExec: target_batch_size=2 15)--------------------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 -16)----------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +16)----------------------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 17)------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC, b@1 ASC NULLS LAST], has_header=true query TT @@ -3528,7 +3528,7 @@ logical_plan physical_plan 01)NestedLoopJoinExec: join_type=Inner, filter=a@1 < a@0 02)--CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true -03)--RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +03)--OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 04)----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true # Currently datafusion can pushdown filter conditions with scalar UDF into @@ -3547,7 +3547,7 @@ logical_plan physical_plan 01)NestedLoopJoinExec: join_type=Inner, filter=example(CAST(a@0 AS Float64), CAST(a@1 AS Float64)) > 3 02)--CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true -03)--RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +03)--OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 04)----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true #### @@ -4344,12 +4344,12 @@ physical_plan 05)--------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 06)----------CoalesceBatchesExec: target_batch_size=3 07)------------FilterExec: b@1 > 3, projection=[a@0] -08)--------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +08)--------------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 09)----------------MemoryExec: partitions=1, partition_sizes=[1] 10)------SortExec: expr=[c@2 DESC], preserve_partitioning=[true] 11)--------CoalesceBatchesExec: target_batch_size=3 12)----------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 -13)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +13)------------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 14)--------------MemoryExec: partitions=1, partition_sizes=[1] query TT @@ -4371,12 +4371,12 @@ physical_plan 05)--------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 06)----------CoalesceBatchesExec: target_batch_size=3 07)------------FilterExec: b@1 > 3, projection=[a@0] -08)--------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +08)--------------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 09)----------------MemoryExec: partitions=1, partition_sizes=[1] 10)------SortExec: expr=[c@2 DESC NULLS LAST], preserve_partitioning=[true] 11)--------CoalesceBatchesExec: target_batch_size=3 12)----------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 -13)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +13)------------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 14)--------------MemoryExec: partitions=1, partition_sizes=[1] query III diff --git a/datafusion/sqllogictest/test_files/json.slt b/datafusion/sqllogictest/test_files/json.slt index 0903c2427649..195bb1337e62 100644 --- a/datafusion/sqllogictest/test_files/json.slt +++ b/datafusion/sqllogictest/test_files/json.slt @@ -60,7 +60,7 @@ physical_plan 01)AggregateExec: mode=Final, gby=[], aggr=[count(*)] 02)--CoalescePartitionsExec 03)----AggregateExec: mode=Partial, gby=[], aggr=[count(*)] -04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +04)------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 05)--------JsonExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/2.json]]} query ? diff --git a/datafusion/sqllogictest/test_files/limit.slt b/datafusion/sqllogictest/test_files/limit.slt index 5b98392f1aa0..f62ce361ce4d 100644 --- a/datafusion/sqllogictest/test_files/limit.slt +++ b/datafusion/sqllogictest/test_files/limit.slt @@ -369,7 +369,7 @@ physical_plan 01)AggregateExec: mode=Final, gby=[], aggr=[count(*)] 02)--CoalescePartitionsExec 03)----AggregateExec: mode=Partial, gby=[], aggr=[count(*)] -04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +04)------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 05)--------ProjectionExec: expr=[] 06)----------GlobalLimitExec: skip=6, fetch=3 07)------------CoalesceBatchesExec: target_batch_size=8192, fetch=9 @@ -402,7 +402,7 @@ physical_plan 01)AggregateExec: mode=FinalPartitioned, gby=[i@0 as i], aggr=[] 02)--CoalesceBatchesExec: target_batch_size=8192 03)----RepartitionExec: partitioning=Hash([i@0], 4), input_partitions=4 -04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +04)------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 05)--------AggregateExec: mode=Partial, gby=[i@0 as i], aggr=[] 06)----------MemoryExec: partitions=1 @@ -635,7 +635,7 @@ physical_plan 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------RepartitionExec: partitioning=Hash([b@0], 4), input_partitions=4 07)------------AggregateExec: mode=Partial, gby=[b@1 as b], aggr=[sum(ordered_table.a)] -08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)--------------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 09)----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], has_header=true # Applying offset & limit when multiple streams from union @@ -661,11 +661,11 @@ physical_plan 03)----UnionExec 04)------SortExec: TopK(fetch=14), expr=[c@0 DESC], preserve_partitioning=[true] 05)--------ProjectionExec: expr=[CAST(c@0 AS Int64) as c] -06)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +06)----------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 07)------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], output_ordering=[c@0 ASC NULLS LAST], has_header=true 08)------SortExec: TopK(fetch=14), expr=[c@0 DESC], preserve_partitioning=[true] 09)--------ProjectionExec: expr=[CAST(d@0 AS Int64) as c] -10)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +10)----------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 11)------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[d], has_header=true # Applying LIMIT & OFFSET to subquery. diff --git a/datafusion/sqllogictest/test_files/monotonic_projection_test.slt b/datafusion/sqllogictest/test_files/monotonic_projection_test.slt index abf48fac5364..ce0a54857918 100644 --- a/datafusion/sqllogictest/test_files/monotonic_projection_test.slt +++ b/datafusion/sqllogictest/test_files/monotonic_projection_test.slt @@ -46,7 +46,7 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [a_big@0 ASC NULLS LAST, b@1 ASC NULLS LAST] 02)--ProjectionExec: expr=[CAST(a@0 AS Int64) as a_big, b@1 as b] -03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +03)----OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], has_header=true query TT @@ -62,7 +62,7 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [a@0 ASC NULLS LAST, b@2 ASC NULLS LAST] 02)--ProjectionExec: expr=[a@0 as a, CAST(a@0 AS Int64) as a_big, b@1 as b] -03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +03)----OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], has_header=true # Cast to larger types as well as preserving ordering @@ -83,7 +83,7 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [a_big@1 ASC NULLS LAST, b@2 ASC NULLS LAST] 02)--ProjectionExec: expr=[a@0 as a, CAST(a@0 AS Int64) as a_big, b@1 as b] -03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +03)----OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], has_header=true # test for common rename @@ -135,7 +135,7 @@ physical_plan 01)SortPreservingMergeExec: [a_str@0 ASC NULLS LAST, b@1 ASC NULLS LAST] 02)--SortExec: expr=[a_str@0 ASC NULLS LAST, b@1 ASC NULLS LAST], preserve_partitioning=[true] 03)----ProjectionExec: expr=[CAST(a@0 AS Utf8) as a_str, b@1 as b] -04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +04)------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], has_header=true # We cannot determine a+b is ordered from the @@ -170,5 +170,5 @@ physical_plan 01)SortPreservingMergeExec: [sum_expr@0 ASC NULLS LAST] 02)--SortExec: expr=[sum_expr@0 ASC NULLS LAST], preserve_partitioning=[true] 03)----ProjectionExec: expr=[CAST(a@0 + b@1 AS Int64) as sum_expr, a@0 as a, b@1 as b] -04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +04)------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], has_header=true diff --git a/datafusion/sqllogictest/test_files/order.slt b/datafusion/sqllogictest/test_files/order.slt index a46040aa532e..134940eb6330 100644 --- a/datafusion/sqllogictest/test_files/order.slt +++ b/datafusion/sqllogictest/test_files/order.slt @@ -456,7 +456,7 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [result@0 ASC NULLS LAST] 02)--ProjectionExec: expr=[b@1 + a@0 + c@2 as result] -03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +03)----OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], output_orderings=[[a@0 ASC NULLS LAST], [b@1 ASC NULLS LAST], [c@2 ASC NULLS LAST]], has_header=true statement ok @@ -487,7 +487,7 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [db15@0 ASC NULLS LAST] 02)--ProjectionExec: expr=[date_bin(IntervalMonthDayNano { months: 0, days: 0, nanoseconds: 900000000000 }, ts@0, 1659537600000000000) as db15] -03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +03)----OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/timestamps.csv]]}, projection=[ts], output_ordering=[ts@0 ASC NULLS LAST], has_header=false query TT @@ -502,7 +502,7 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [dt_day@0 ASC NULLS LAST] 02)--ProjectionExec: expr=[date_trunc(DAY, ts@0) as dt_day] -03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +03)----OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/timestamps.csv]]}, projection=[ts], output_ordering=[ts@0 ASC NULLS LAST], has_header=false statement ok @@ -545,7 +545,7 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [atan_c11@0 ASC NULLS LAST] 02)--ProjectionExec: expr=[atan(c11@0) as atan_c11] -03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +03)----OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c11], output_ordering=[c11@0 ASC NULLS LAST], has_header=true query TT @@ -560,7 +560,7 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [ceil_c11@0 ASC NULLS LAST] 02)--ProjectionExec: expr=[ceil(c11@0) as ceil_c11] -03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +03)----OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c11], output_ordering=[c11@0 ASC NULLS LAST], has_header=true query TT @@ -575,7 +575,7 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [log_c11_base_c12@0 ASC NULLS LAST] 02)--ProjectionExec: expr=[log(c12@1, CAST(c11@0 AS Float64)) as log_c11_base_c12] -03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +03)----OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c11, c12], output_orderings=[[c11@0 ASC NULLS LAST], [c12@1 DESC NULLS LAST]], has_header=true query TT @@ -590,7 +590,7 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [log_c12_base_c11@0 DESC NULLS LAST] 02)--ProjectionExec: expr=[log(CAST(c11@0 AS Float64), c12@1) as log_c12_base_c11] -03)----RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +03)----OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c11, c12], output_orderings=[[c11@0 ASC NULLS LAST], [c12@1 DESC NULLS LAST]], has_header=true statement ok @@ -783,7 +783,7 @@ physical_plan 05)--------AggregateExec: mode=FinalPartitioned, gby=[t@0 as t], aggr=[] 06)----------CoalesceBatchesExec: target_batch_size=8192 07)------------RepartitionExec: partitioning=Hash([t@0], 2), input_partitions=2 -08)--------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +08)--------------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 09)----------------AggregateExec: mode=Partial, gby=[t@0 as t], aggr=[] 10)------------------ProjectionExec: expr=[column1@0 as t] 11)--------------------ValuesExec @@ -791,7 +791,7 @@ physical_plan 13)--------AggregateExec: mode=FinalPartitioned, gby=[t@0 as t], aggr=[] 14)----------CoalesceBatchesExec: target_batch_size=8192 15)------------RepartitionExec: partitioning=Hash([t@0], 2), input_partitions=2 -16)--------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +16)--------------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 17)----------------AggregateExec: mode=Partial, gby=[t@0 as t], aggr=[] 18)------------------ProjectionExec: expr=[column1@0 as t] 19)--------------------ValuesExec @@ -1024,7 +1024,7 @@ physical_plan 01)SortPreservingMergeExec: [c_str@0 ASC NULLS LAST], fetch=5 02)--SortExec: TopK(fetch=5), expr=[c_str@0 ASC NULLS LAST], preserve_partitioning=[true] 03)----ProjectionExec: expr=[CAST(c@0 AS Utf8) as c_str] -04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +04)------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], output_ordering=[c@0 ASC NULLS LAST], has_header=true @@ -1054,7 +1054,7 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [c_bigint@0 ASC NULLS LAST], fetch=5 02)--ProjectionExec: expr=[CAST(c@0 AS Int64) as c_bigint] -03)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +03)----OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], output_ordering=[c@0 ASC NULLS LAST], has_header=true statement ok @@ -1090,7 +1090,7 @@ physical_plan 01)SortPreservingMergeExec: [abs_c@0 ASC NULLS LAST], fetch=5 02)--SortExec: TopK(fetch=5), expr=[abs_c@0 ASC NULLS LAST], preserve_partitioning=[true] 03)----ProjectionExec: expr=[abs(c@0) as abs_c] -04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +04)------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], output_ordering=[c@0 ASC NULLS LAST], has_header=true statement ok @@ -1124,7 +1124,7 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [abs_c@0 ASC NULLS LAST], fetch=5 02)--ProjectionExec: expr=[abs(c@0) as abs_c] -03)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +03)----OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], output_ordering=[c@0 ASC NULLS LAST], has_header=true # Boolean to integer casts preserve the order. @@ -1150,7 +1150,7 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [c@0 ASC NULLS LAST] 02)--ProjectionExec: expr=[CAST(inc_col@0 > desc_col@1 AS Int32) as c] -03)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +03)----OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_1.csv]]}, projection=[inc_col, desc_col], output_orderings=[[inc_col@0 ASC NULLS LAST], [desc_col@1 DESC]], has_header=true # Union a query with the actual data and one with a constant @@ -1173,7 +1173,7 @@ logical_plan 03)----TableScan: ordered_table projection=[a, b] physical_plan 01)ProjectionExec: expr=[a@0 + b@1 as sum1] -02)--RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +02)--OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 03)----SortExec: TopK(fetch=1), expr=[a@0 ASC NULLS LAST], preserve_partitioning=[false] 04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], has_header=true @@ -1212,7 +1212,7 @@ logical_plan 03)----TableScan: ordered_table projection=[a, b] physical_plan 01)ProjectionExec: expr=[a@0 + b@1 as sum1] -02)--RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +02)--OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 03)----SortExec: TopK(fetch=1), expr=[a@0 ASC NULLS LAST], preserve_partitioning=[false] 04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], has_header=true diff --git a/datafusion/sqllogictest/test_files/parquet.slt b/datafusion/sqllogictest/test_files/parquet.slt index 253ebb9ea0ac..625714c5ebbd 100644 --- a/datafusion/sqllogictest/test_files/parquet.slt +++ b/datafusion/sqllogictest/test_files/parquet.slt @@ -410,7 +410,7 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: CAST(binary_col@0 AS Utf8View) LIKE %a% AND CAST(largebinary_col@1 AS Utf8View) LIKE %a% AND CAST(binaryview_col@2 AS Utf8View) LIKE %a% -03)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +03)----OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 04)------ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/binary_as_string.parquet]]}, projection=[binary_col, largebinary_col, binaryview_col], predicate=CAST(binary_col@0 AS Utf8View) LIKE %a% AND CAST(largebinary_col@1 AS Utf8View) LIKE %a% AND CAST(binaryview_col@2 AS Utf8View) LIKE %a% @@ -458,7 +458,7 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: binary_col@0 LIKE %a% AND largebinary_col@1 LIKE %a% AND binaryview_col@2 LIKE %a% -03)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +03)----OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 04)------ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/binary_as_string.parquet]]}, projection=[binary_col, largebinary_col, binaryview_col], predicate=binary_col@0 LIKE %a% AND largebinary_col@1 LIKE %a% AND binaryview_col@2 LIKE %a% @@ -509,7 +509,7 @@ logical_plan physical_plan 01)CoalesceBatchesExec: target_batch_size=8192 02)--FilterExec: binary_col@0 LIKE %a% AND largebinary_col@1 LIKE %a% AND binaryview_col@2 LIKE %a% -03)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +03)----OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 04)------ParquetExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet/binary_as_string.parquet]]}, projection=[binary_col, largebinary_col, binaryview_col], predicate=binary_col@0 LIKE %a% AND largebinary_col@1 LIKE %a% AND binaryview_col@2 LIKE %a% diff --git a/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt b/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt index 24ffb963bbe2..dd8e29eb20b8 100644 --- a/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt +++ b/datafusion/sqllogictest/test_files/parquet_filter_pushdown.slt @@ -112,7 +112,7 @@ physical_plan 02)--SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true] 03)----CoalesceBatchesExec: target_batch_size=8192 04)------FilterExec: b@1 > 2, projection=[a@0] -05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2 +05)--------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=2 06)----------ParquetExec: file_groups={2 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/1.parquet], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/scratch/parquet_filter_pushdown/parquet_table/2.parquet]]}, projection=[a, b], predicate=b@1 > 2, pruning_predicate=CASE WHEN b_null_count@1 = b_row_count@2 THEN false ELSE b_max@0 > 2 END, required_guarantees=[] # also test querying on columns that are not in all the files diff --git a/datafusion/sqllogictest/test_files/predicates.slt b/datafusion/sqllogictest/test_files/predicates.slt index 878d7c8a4dfb..c86fab6765af 100644 --- a/datafusion/sqllogictest/test_files/predicates.slt +++ b/datafusion/sqllogictest/test_files/predicates.slt @@ -674,13 +674,13 @@ physical_plan 04)------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 05)--------CoalesceBatchesExec: target_batch_size=8192 06)----------FilterExec: l_quantity@1 >= Some(100),15,2 AND l_quantity@1 <= Some(1100),15,2 OR l_quantity@1 >= Some(1000),15,2 AND l_quantity@1 <= Some(2000),15,2 OR l_quantity@1 >= Some(2000),15,2 AND l_quantity@1 <= Some(3000),15,2 -07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +07)------------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 08)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/lineitem.csv]]}, projection=[l_partkey, l_quantity], has_header=true 09)----CoalesceBatchesExec: target_batch_size=8192 10)------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 11)--------CoalesceBatchesExec: target_batch_size=8192 12)----------FilterExec: (p_brand@1 = Brand#12 AND p_size@2 <= 5 OR p_brand@1 = Brand#23 AND p_size@2 <= 10 OR p_brand@1 = Brand#34 AND p_size@2 <= 15) AND p_size@2 >= 1 -13)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +13)------------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 14)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/part.csv]]}, projection=[p_partkey, p_brand, p_size], has_header=true ######## @@ -766,13 +766,13 @@ physical_plan 05)--------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)], projection=[l_extendedprice@1, l_discount@2, p_partkey@3] 06)----------CoalesceBatchesExec: target_batch_size=8192 07)------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 -08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)--------------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 09)----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/lineitem.csv]]}, projection=[l_partkey, l_extendedprice, l_discount], has_header=true 10)----------CoalesceBatchesExec: target_batch_size=8192 11)------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 12)--------------CoalesceBatchesExec: target_batch_size=8192 13)----------------FilterExec: p_brand@1 = Brand#12 OR p_brand@1 = Brand#23, projection=[p_partkey@0] -14)------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +14)------------------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 15)--------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/tpch-csv/part.csv]]}, projection=[p_partkey, p_brand], has_header=true 16)------CoalesceBatchesExec: target_batch_size=8192 17)--------RepartitionExec: partitioning=Hash([ps_partkey@0], 4), input_partitions=1 diff --git a/datafusion/sqllogictest/test_files/repartition.slt b/datafusion/sqllogictest/test_files/repartition.slt index 630674bb09ed..8b5112cfc7f8 100644 --- a/datafusion/sqllogictest/test_files/repartition.slt +++ b/datafusion/sqllogictest/test_files/repartition.slt @@ -47,7 +47,7 @@ physical_plan 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 +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 round robin repartitioning @@ -125,7 +125,7 @@ physical_plan 02)--CoalescePartitionsExec 03)----CoalesceBatchesExec: target_batch_size=8192, fetch=5 04)------FilterExec: c3@2 > 0 -05)--------RepartitionExec: partitioning=RoundRobinBatch(3), input_partitions=1 +05)--------OnDemandRepartitionExec: partitioning=OnDemand(3), input_partitions=1 06)----------StreamingTableExec: partition_sizes=1, projection=[c1, c2, c3], infinite_source=true # Start repratition on empty column test. diff --git a/datafusion/sqllogictest/test_files/select.slt b/datafusion/sqllogictest/test_files/select.slt index c687429ae6ec..26952d45e3a4 100644 --- a/datafusion/sqllogictest/test_files/select.slt +++ b/datafusion/sqllogictest/test_files/select.slt @@ -1404,7 +1404,7 @@ logical_plan physical_plan 01)SortPreservingMergeExec: [a@0 ASC NULLS LAST] 02)--ProjectionExec: expr=[a@0 as a, a@0 + b@1 as annotated_data_finite2.a + annotated_data_finite2.b] -03)----RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +03)----OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 04)------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], has_header=true # since query below doesn't computation @@ -1442,7 +1442,7 @@ physical_plan 01)SortPreservingMergeExec: [b@2 ASC NULLS LAST, c@3 ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=8192 03)----FilterExec: a@1 = 0 -04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +04)------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true # source is ordered by a,b,c @@ -1463,7 +1463,7 @@ physical_plan 01)SortPreservingMergeExec: [c@3 ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=8192 03)----FilterExec: a@1 = 0 AND b@2 = 0 -04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +04)------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true # source is ordered by a,b,c @@ -1484,7 +1484,7 @@ physical_plan 01)SortPreservingMergeExec: [b@2 ASC NULLS LAST, c@3 ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=8192 03)----FilterExec: a@1 = 0 AND b@2 = 0 -04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +04)------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true # source is ordered by a,b,c @@ -1505,7 +1505,7 @@ physical_plan 01)SortPreservingMergeExec: [a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=8192 03)----FilterExec: a@1 = 0 AND b@2 = 0 -04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +04)------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true # source is ordered by a,b,c @@ -1527,7 +1527,7 @@ physical_plan 02)--SortExec: expr=[c@3 ASC NULLS LAST], preserve_partitioning=[true] 03)----CoalesceBatchesExec: target_batch_size=8192 04)------FilterExec: a@1 = 0 OR b@2 = 0 -05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +05)--------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true # When ordering lost during projection, we shouldn't keep the SortExec. @@ -1551,7 +1551,7 @@ physical_plan 02)--CoalesceBatchesExec: target_batch_size=8192 03)----RepartitionExec: partitioning=Hash([c2@0], 2), input_partitions=2 04)------AggregateExec: mode=Partial, gby=[c2@0 as c2], aggr=[count(*)] -05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +05)--------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 06)----------ProjectionExec: expr=[c2@0 as c2] 07)------------SortExec: TopK(fetch=4), expr=[c1@1 ASC NULLS LAST, c2@0 ASC NULLS LAST], preserve_partitioning=[false] 08)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c1], has_header=true @@ -1573,7 +1573,7 @@ physical_plan 01)SortPreservingMergeExec: [CAST(round(CAST(b@2 AS Float64)) AS Int32) ASC NULLS LAST] 02)--CoalesceBatchesExec: target_batch_size=8192 03)----FilterExec: CAST(round(CAST(b@2 AS Float64)) AS Int32) = a@1 -04)------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +04)------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 05)--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true diff --git a/datafusion/sqllogictest/test_files/subquery.slt b/datafusion/sqllogictest/test_files/subquery.slt index 027b5ca8dcfb..728199341e82 100644 --- a/datafusion/sqllogictest/test_files/subquery.slt +++ b/datafusion/sqllogictest/test_files/subquery.slt @@ -208,11 +208,11 @@ physical_plan 06)----------CoalesceBatchesExec: target_batch_size=2 07)------------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 08)--------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int)] -09)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +09)----------------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 10)------------------MemoryExec: partitions=1, partition_sizes=[1] 11)------CoalesceBatchesExec: target_batch_size=2 12)--------RepartitionExec: partitioning=Hash([t1_id@0], 4), input_partitions=4 -13)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +13)----------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 14)------------MemoryExec: partitions=1, partition_sizes=[1] query II rowsort @@ -244,11 +244,11 @@ physical_plan 06)----------CoalesceBatchesExec: target_batch_size=2 07)------------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 08)--------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int * Float64(1))] -09)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +09)----------------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 10)------------------MemoryExec: partitions=1, partition_sizes=[1] 11)------CoalesceBatchesExec: target_batch_size=2 12)--------RepartitionExec: partitioning=Hash([t1_id@0], 4), input_partitions=4 -13)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +13)----------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 14)------------MemoryExec: partitions=1, partition_sizes=[1] query IR rowsort @@ -280,11 +280,11 @@ physical_plan 06)----------CoalesceBatchesExec: target_batch_size=2 07)------------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 08)--------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int)] -09)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +09)----------------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 10)------------------MemoryExec: partitions=1, partition_sizes=[1] 11)------CoalesceBatchesExec: target_batch_size=2 12)--------RepartitionExec: partitioning=Hash([t1_id@0], 4), input_partitions=4 -13)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +13)----------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 14)------------MemoryExec: partitions=1, partition_sizes=[1] query II rowsort @@ -319,11 +319,11 @@ physical_plan 08)--------------CoalesceBatchesExec: target_batch_size=2 09)----------------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 10)------------------AggregateExec: mode=Partial, gby=[t2_id@0 as t2_id], aggr=[sum(t2.t2_int)] -11)--------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +11)--------------------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 12)----------------------MemoryExec: partitions=1, partition_sizes=[1] 13)------CoalesceBatchesExec: target_batch_size=2 14)--------RepartitionExec: partitioning=Hash([t1_id@0], 4), input_partitions=4 -15)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +15)----------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 16)------------MemoryExec: partitions=1, partition_sizes=[1] query II rowsort @@ -1152,11 +1152,11 @@ physical_plan 04)------HashJoinExec: mode=Partitioned, join_type=LeftMark, on=[(t1_id@0, t2_id@0)] 05)--------CoalesceBatchesExec: target_batch_size=2 06)----------RepartitionExec: partitioning=Hash([t1_id@0], 4), input_partitions=4 -07)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +07)------------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 08)--------------MemoryExec: partitions=1, partition_sizes=[1] 09)--------CoalesceBatchesExec: target_batch_size=2 10)----------RepartitionExec: partitioning=Hash([t2_id@0], 4), input_partitions=4 -11)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +11)------------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 12)--------------MemoryExec: partitions=1, partition_sizes=[1] statement ok diff --git a/datafusion/sqllogictest/test_files/subquery_sort.slt b/datafusion/sqllogictest/test_files/subquery_sort.slt index a3717dd838d6..4421dcb342a8 100644 --- a/datafusion/sqllogictest/test_files/subquery_sort.slt +++ b/datafusion/sqllogictest/test_files/subquery_sort.slt @@ -125,7 +125,7 @@ physical_plan 06)----------CoalesceBatchesExec: target_batch_size=8192 07)------------RepartitionExec: partitioning=Hash([c1@0], 4), input_partitions=4 08)--------------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[first_value(sink_table.c1) ORDER BY [sink_table.c1 ASC NULLS LAST, sink_table.c3 DESC NULLS FIRST, sink_table.c9 ASC NULLS LAST], first_value(sink_table.c2) ORDER BY [sink_table.c1 ASC NULLS LAST, sink_table.c3 DESC NULLS FIRST, sink_table.c9 ASC NULLS LAST], first_value(sink_table.c3) ORDER BY [sink_table.c1 ASC NULLS LAST, sink_table.c3 DESC NULLS FIRST, sink_table.c9 ASC NULLS LAST], first_value(sink_table.c9) ORDER BY [sink_table.c1 ASC NULLS LAST, sink_table.c3 DESC NULLS FIRST, sink_table.c9 ASC NULLS LAST]] -09)----------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +09)----------------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 10)------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c3, c9], has_header=true diff --git a/datafusion/sqllogictest/test_files/union.slt b/datafusion/sqllogictest/test_files/union.slt index d94780744db9..85263d9e6563 100644 --- a/datafusion/sqllogictest/test_files/union.slt +++ b/datafusion/sqllogictest/test_files/union.slt @@ -236,7 +236,7 @@ physical_plan 01)AggregateExec: mode=FinalPartitioned, gby=[name@0 as name], aggr=[] 02)--CoalesceBatchesExec: target_batch_size=8192 03)----RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=4 -04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=3 +04)------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=3 05)--------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[] 06)----------UnionExec 07)------------MemoryExec: partitions=1, partition_sizes=[1] @@ -313,12 +313,12 @@ physical_plan 05)--------CoalesceBatchesExec: target_batch_size=2 06)----------RepartitionExec: partitioning=Hash([id@0, name@1], 4), input_partitions=4 07)------------AggregateExec: mode=Partial, gby=[id@0 as id, name@1 as name], aggr=[] -08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)--------------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 09)----------------MemoryExec: partitions=1, partition_sizes=[1] 10)------CoalesceBatchesExec: target_batch_size=2 11)--------RepartitionExec: partitioning=Hash([CAST(t2.id AS Int32)@2, name@1], 4), input_partitions=4 12)----------ProjectionExec: expr=[id@0 as id, name@1 as name, CAST(id@0 AS Int32) as CAST(t2.id AS Int32)] -13)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +13)------------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 14)--------------MemoryExec: partitions=1, partition_sizes=[1] 15)--ProjectionExec: expr=[CAST(id@0 AS Int32) as id, name@1 as name] 16)----CoalesceBatchesExec: target_batch_size=2 @@ -330,11 +330,11 @@ physical_plan 22)----------------CoalesceBatchesExec: target_batch_size=2 23)------------------RepartitionExec: partitioning=Hash([id@0, name@1], 4), input_partitions=4 24)--------------------AggregateExec: mode=Partial, gby=[id@0 as id, name@1 as name], aggr=[] -25)----------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +25)----------------------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 26)------------------------MemoryExec: partitions=1, partition_sizes=[1] 27)--------CoalesceBatchesExec: target_batch_size=2 28)----------RepartitionExec: partitioning=Hash([id@0, name@1], 4), input_partitions=4 -29)------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +29)------------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 30)--------------MemoryExec: partitions=1, partition_sizes=[1] @@ -387,11 +387,11 @@ physical_plan 05)--------CoalesceBatchesExec: target_batch_size=2 06)----------RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=4 07)------------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[] -08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)--------------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 09)----------------MemoryExec: partitions=1, partition_sizes=[1] 10)------CoalesceBatchesExec: target_batch_size=2 11)--------RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=4 -12)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +12)----------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 13)------------MemoryExec: partitions=1, partition_sizes=[1] 14)--CoalesceBatchesExec: target_batch_size=2 15)----HashJoinExec: mode=Partitioned, join_type=LeftAnti, on=[(name@0, name@0)] @@ -399,11 +399,11 @@ physical_plan 17)--------CoalesceBatchesExec: target_batch_size=2 18)----------RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=4 19)------------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[] -20)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +20)--------------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 21)----------------MemoryExec: partitions=1, partition_sizes=[1] 22)------CoalesceBatchesExec: target_batch_size=2 23)--------RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=4 -24)----------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +24)----------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 25)------------MemoryExec: partitions=1, partition_sizes=[1] # union_upcast_types @@ -422,11 +422,11 @@ physical_plan 02)--UnionExec 03)----SortExec: TopK(fetch=5), expr=[c9@1 DESC], preserve_partitioning=[true] 04)------ProjectionExec: expr=[c1@0 as c1, CAST(c9@1 AS Int64) as c9] -05)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +05)--------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 06)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c9], has_header=true 07)----SortExec: TopK(fetch=5), expr=[c9@1 DESC], preserve_partitioning=[true] 08)------ProjectionExec: expr=[c1@0 as c1, CAST(c3@1 AS Int64) as c9] -09)--------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +09)--------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 10)----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c3], has_header=true query TI @@ -463,13 +463,13 @@ physical_plan 05)--------CoalesceBatchesExec: target_batch_size=2 06)----------RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=4 07)------------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[] -08)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +08)--------------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 09)----------------MemoryExec: partitions=1, partition_sizes=[1] 10)------AggregateExec: mode=FinalPartitioned, gby=[name@0 as name], aggr=[] 11)--------CoalesceBatchesExec: target_batch_size=2 12)----------RepartitionExec: partitioning=Hash([name@0], 4), input_partitions=4 13)------------AggregateExec: mode=Partial, gby=[name@0 as name], aggr=[] -14)--------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +14)--------------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 15)----------------MemoryExec: partitions=1, partition_sizes=[1] # Union with limit push down 3 children test case @@ -524,7 +524,7 @@ physical_plan 12)----------------------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[] 13)------------------------CoalesceBatchesExec: target_batch_size=2 14)--------------------------FilterExec: c13@1 != C2GT5KVyOPZpgKVl110TyZO0NcJ434, projection=[c1@0] -15)----------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +15)----------------------------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 16)------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c13], has_header=true 17)------ProjectionExec: expr=[1 as cnt] 18)--------PlaceholderRowExec diff --git a/datafusion/sqllogictest/test_files/unnest.slt b/datafusion/sqllogictest/test_files/unnest.slt index 1c54006bd2a0..a568dcc091db 100644 --- a/datafusion/sqllogictest/test_files/unnest.slt +++ b/datafusion/sqllogictest/test_files/unnest.slt @@ -601,7 +601,7 @@ logical_plan 05)--------TableScan: recursive_unnest_table projection=[column3] physical_plan 01)UnnestExec -02)--RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +02)--OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 03)----ProjectionExec: expr=[__unnest_placeholder(recursive_unnest_table.column3,depth=1)@0 as __unnest_placeholder(UNNEST(recursive_unnest_table.column3)), column3@1 as column3] 04)------UnnestExec 05)--------ProjectionExec: expr=[column3@0 as __unnest_placeholder(recursive_unnest_table.column3), column3@0 as column3] @@ -660,7 +660,7 @@ physical_plan 01)ProjectionExec: expr=[__unnest_placeholder(UNNEST(recursive_unnest_table.column3)[c1],depth=2)@0 as UNNEST(UNNEST(UNNEST(recursive_unnest_table.column3)[c1])), column3@1 as column3] 02)--UnnestExec 03)----ProjectionExec: expr=[get_field(__unnest_placeholder(recursive_unnest_table.column3,depth=1)@0, c1) as __unnest_placeholder(UNNEST(recursive_unnest_table.column3)[c1]), column3@1 as column3] -04)------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +04)------OnDemandRepartitionExec: partitioning=OnDemand(4), input_partitions=1 05)--------UnnestExec 06)----------ProjectionExec: expr=[column3@0 as __unnest_placeholder(recursive_unnest_table.column3), column3@0 as column3] 07)------------MemoryExec: partitions=1, partition_sizes=[1] diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 188e2ae0915f..41a5434ea031 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -1320,7 +1320,7 @@ physical_plan 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)--------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +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 @@ -1733,7 +1733,7 @@ physical_plan 06)----------SortPreservingMergeExec: [__common_expr_1@0 DESC, c9@3 DESC, c2@1 ASC NULLS LAST] 07)------------SortExec: expr=[__common_expr_1@0 DESC, c9@3 DESC, c2@1 ASC NULLS LAST], preserve_partitioning=[true] 08)--------------ProjectionExec: expr=[c3@1 + c4@2 as __common_expr_1, c2@0 as c2, c3@1 as c3, c9@3 as c9] -09)----------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +09)----------------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 10)------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3, c4, c9], has_header=true @@ -1784,7 +1784,7 @@ physical_plan 09)----------------AggregateExec: mode=Partial, gby=[c1@0 as c1], aggr=[] 10)------------------CoalesceBatchesExec: target_batch_size=4096 11)--------------------FilterExec: c13@1 != C2GT5KVyOPZpgKVl110TyZO0NcJ434, projection=[c1@0] -12)----------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +12)----------------------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 13)------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c13], has_header=true @@ -1826,7 +1826,7 @@ physical_plan 04)------SortExec: expr=[c3@0 ASC NULLS LAST, c9@1 DESC], preserve_partitioning=[true] 05)--------CoalesceBatchesExec: target_batch_size=4096 06)----------RepartitionExec: partitioning=Hash([c3@0], 2), input_partitions=2 -07)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +07)------------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 08)--------------ProjectionExec: expr=[c3@1 as c3, c9@2 as c9, sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] 09)----------------BoundedWindowAggExec: wdw=[sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "sum(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int16(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] 10)------------------SortExec: expr=[c3@1 DESC, c9@2 DESC, c2@0 ASC NULLS LAST], preserve_partitioning=[false] @@ -1868,7 +1868,7 @@ physical_plan 04)------SortExec: expr=[c1@0 ASC NULLS LAST], preserve_partitioning=[true] 05)--------CoalesceBatchesExec: target_batch_size=4096 06)----------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 -07)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +07)------------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 08)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], has_header=true query TI @@ -1997,7 +1997,7 @@ physical_plan 04)------SortExec: expr=[c1@0 ASC NULLS LAST], preserve_partitioning=[true] 05)--------CoalesceBatchesExec: target_batch_size=4096 06)----------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 -07)------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +07)------------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 08)--------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], has_header=true statement ok @@ -2026,7 +2026,7 @@ physical_plan 07)------------SortExec: expr=[c1@0 ASC NULLS LAST, c9@1 ASC NULLS LAST], preserve_partitioning=[true] 08)--------------CoalesceBatchesExec: target_batch_size=4096 09)----------------RepartitionExec: partitioning=Hash([c1@0], 2), input_partitions=2 -10)------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +10)------------------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 11)--------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c9], has_header=true # test_window_agg_with_global_limit @@ -2046,7 +2046,7 @@ physical_plan 02)--AggregateExec: mode=Final, gby=[], aggr=[array_agg(aggregate_test_100.c13)] 03)----CoalescePartitionsExec 04)------AggregateExec: mode=Partial, gby=[], aggr=[array_agg(aggregate_test_100.c13)] -05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +05)--------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 06)----------SortExec: TopK(fetch=1), expr=[c13@0 ASC NULLS LAST], preserve_partitioning=[false] 07)------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c13], has_header=true @@ -3276,7 +3276,7 @@ physical_plan 13)------------------------CoalesceBatchesExec: target_batch_size=4096 14)--------------------------RepartitionExec: partitioning=Hash([a@1, b@2], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST, __common_expr_1@0 ASC NULLS LAST 15)----------------------------ProjectionExec: expr=[CAST(a@0 AS Int64) as __common_expr_1, a@0 as a, b@1 as b, c@2 as c, d@3 as d] -16)------------------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +16)------------------------------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 17)--------------------------------StreamingTableExec: partition_sizes=1, projection=[a, b, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST] # reset the partition number 1 again @@ -3625,7 +3625,7 @@ physical_plan 03)----BoundedWindowAggExec: wdw=[avg(multiple_ordered_table_inf.d) PARTITION BY [multiple_ordered_table_inf.d] ORDER BY [multiple_ordered_table_inf.a ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND CURRENT ROW: Ok(Field { name: "avg(multiple_ordered_table_inf.d) PARTITION BY [multiple_ordered_table_inf.d] ORDER BY [multiple_ordered_table_inf.a ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND CURRENT ROW", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: CurrentRow, is_causal: false }], mode=[Linear] 04)------CoalesceBatchesExec: target_batch_size=4096 05)--------RepartitionExec: partitioning=Hash([d@4], 2), input_partitions=2, preserve_order=true, sort_exprs=a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST -06)----------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +06)----------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 07)------------StreamingTableExec: partition_sizes=1, projection=[a0, a, b, c, d], infinite_source=true, output_orderings=[[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST], [c@3 ASC NULLS LAST]] # CTAS with NTILE function @@ -4120,7 +4120,7 @@ physical_plan 02)--BoundedWindowAggExec: wdw=[count(*) PARTITION BY [a.a] ORDER BY [a.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "count(*) PARTITION BY [a.a] ORDER BY [a.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow, is_causal: false }], mode=[Sorted] 03)----CoalesceBatchesExec: target_batch_size=4096 04)------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 -05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +05)--------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 06)----------CoalesceBatchesExec: target_batch_size=4096 07)------------FilterExec: a@0 = 1 08)--------------MemoryExec: partitions=1, partition_sizes=[1] @@ -4143,7 +4143,7 @@ physical_plan 02)--BoundedWindowAggExec: wdw=[row_number() PARTITION BY [a.a] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "row_number() PARTITION BY [a.a] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)), is_causal: false }], mode=[Sorted] 03)----CoalesceBatchesExec: target_batch_size=4096 04)------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 -05)--------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +05)--------OnDemandRepartitionExec: partitioning=OnDemand(2), input_partitions=1 06)----------CoalesceBatchesExec: target_batch_size=4096 07)------------FilterExec: a@0 = 1 08)--------------MemoryExec: partitions=1, partition_sizes=[1] @@ -5054,7 +5054,7 @@ select b, row_number() over (order by a) from (select TRUE as a, 1 as b); 1 1 # test window functions on boolean columns -query T +statement count 0 create table t1 (id int, bool_col boolean) as values (1, true), (2, false), From 65824b105ec13e5c9cef27ec82432e28cace1df3 Mon Sep 17 00:00:00 2001 From: Weijun-H Date: Sat, 18 Jan 2025 23:11:37 +0800 Subject: [PATCH 15/20] fix all tests and enable fetching on RecordBatch per partition --- benchmarks/bench.sh | 6 + .../physical-plan/src/repartition/mod.rs | 46 ++++---- .../src/repartition/on_demand_repartition.rs | 105 +++++++----------- 3 files changed, 73 insertions(+), 84 deletions(-) diff --git a/benchmarks/bench.sh b/benchmarks/bench.sh index b02bfee2454e..bf8b639df972 100755 --- a/benchmarks/bench.sh +++ b/benchmarks/bench.sh @@ -156,6 +156,9 @@ main() { tpch10) data_tpch "10" ;; + tpch50) + data_tpch "50" + ;; tpch_mem10) # same data as for tpch10 data_tpch "10" @@ -233,6 +236,9 @@ main() { tpch10) run_tpch "10" ;; + tpch50) + run_tpch "50" + ;; tpch_mem10) run_tpch_mem "10" ;; diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index f9090e4bbeea..20399a6852cb 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -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(), ) } } @@ -194,7 +194,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(); @@ -221,12 +221,22 @@ impl RepartitionExecState { 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() + }; 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), )) @@ -244,21 +254,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); } diff --git a/datafusion/physical-plan/src/repartition/on_demand_repartition.rs b/datafusion/physical-plan/src/repartition/on_demand_repartition.rs index 7f59c2f15b63..0d6bd88c0957 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::{ready, FutureExt, StreamExt, TryStreamExt}; +use futures::{channel, ready, FutureExt, StreamExt, TryStreamExt}; use log::{debug, trace}; use parking_lot::Mutex; @@ -59,8 +59,8 @@ use parking_lot::Mutex; pub struct OnDemandRepartitionExec { base: RepartitionExecBase, /// Channel to send partition number to the downstream task - partition_channel: - Arc, Receiver)>>>, + partition_channels: + Arc>, Vec>)>>>, } impl OnDemandRepartitionExec { @@ -182,7 +182,7 @@ impl ExecutionPlan for OnDemandRepartitionExec { ); let lazy_state = Arc::clone(&self.base.state); - let partition_channel = Arc::clone(&self.partition_channel); + 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(); @@ -196,16 +196,36 @@ impl ExecutionPlan for OnDemandRepartitionExec { let stream = futures::stream::once(async move { let num_input_partitions = input.output_partitioning().partition_count(); + let num_output_partitions = 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_channel = partition_channel - .get_or_init(|| async move { Mutex::new(async_channel::unbounded()) }) + let partition_channels = partition_channels + .get_or_init(|| async move { + if preserve_order { + let (txs, rxs) = (0..num_input_partitions) + .map(|_| async_channel::unbounded()) + .unzip::<_, _, Vec<_>, Vec<_>>(); + // TODO: this approach is not ideal, as it pre-fetches too many partitions + for i in 0..num_output_partitions { + txs.iter().for_each(|tx| { + tx.send_blocking(i).expect("send partition number"); + }); + } + Mutex::new((txs, rxs)) + } else { + let (tx, rx) = async_channel::unbounded(); + for i in num_input_partitions..num_output_partitions { + tx.send(i).await.expect("send partition number"); + } + Mutex::new((vec![tx], vec![rx])) + } + }) .await; - let (partition_tx, partition_rx) = { - let channel = partition_channel.lock(); + let (partition_txs, partition_rxs) = { + let channel = partition_channels.lock(); (channel.0.clone(), channel.1.clone()) }; @@ -214,7 +234,7 @@ impl ExecutionPlan for OnDemandRepartitionExec { Mutex::new( RepartitionExecStateBuilder::new() .enable_pull_based(true) - .partition_receiver(partition_rx.clone()) + .partition_receivers(partition_rxs.clone()) .build( input_captured, partitioning.clone(), @@ -249,16 +269,22 @@ impl ExecutionPlan for OnDemandRepartitionExec { ); if preserve_order { + // TODO: remove this later + debug!("patition number: {}", partition); + debug!("rx number: {}", rx.len()); + // 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 @@ -289,7 +315,7 @@ 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) @@ -335,7 +361,7 @@ impl OnDemandRepartitionExec { preserve_order, cache, }, - partition_channel: Default::default(), + partition_channels: Default::default(), }) } @@ -366,15 +392,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() { + // 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: {}", e ) })?; - - // fetch the next batch - let timer = metrics.fetch_time.timer(); // TODO: To be removed debug!( "On demand pull from input Part: {} \n Are Output channels empty? {}", @@ -382,6 +406,8 @@ impl OnDemandRepartitionExec { output_channels.is_empty() ); + // fetch the next batch + let timer = metrics.fetch_time.timer(); let result = stream.next().await; timer.done(); @@ -435,51 +461,6 @@ 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. - 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(); - } - } - } - } } /// This struct converts a receiver to a stream. @@ -530,7 +511,7 @@ impl Stream for OnDemandPerPartitionStream { "On Demand Repartition per partition poll {}, send partition number", self.partition ); - // self.is_requested = true; + self.is_requested = true; } match ready!(self.receiver.recv().poll_unpin(cx)) { From f367d78fbf4157575b3c18c913d41c6a34b473f3 Mon Sep 17 00:00:00 2001 From: Weijun-H Date: Sun, 19 Jan 2025 13:47:23 +0800 Subject: [PATCH 16/20] chore --- .../src/repartition/on_demand_repartition.rs | 47 +++++++++---------- 1 file changed, 22 insertions(+), 25 deletions(-) diff --git a/datafusion/physical-plan/src/repartition/on_demand_repartition.rs b/datafusion/physical-plan/src/repartition/on_demand_repartition.rs index 0d6bd88c0957..1cc13d6ea4b7 100644 --- a/datafusion/physical-plan/src/repartition/on_demand_repartition.rs +++ b/datafusion/physical-plan/src/repartition/on_demand_repartition.rs @@ -204,24 +204,21 @@ impl ExecutionPlan for OnDemandRepartitionExec { let context_captured = Arc::clone(&context); let partition_channels = partition_channels .get_or_init(|| async move { - if preserve_order { - let (txs, rxs) = (0..num_input_partitions) + let (txs, rxs) = if preserve_order { + (0..num_input_partitions) .map(|_| async_channel::unbounded()) - .unzip::<_, _, Vec<_>, Vec<_>>(); - // TODO: this approach is not ideal, as it pre-fetches too many partitions - for i in 0..num_output_partitions { - txs.iter().for_each(|tx| { - tx.send_blocking(i).expect("send partition number"); - }); - } - Mutex::new((txs, rxs)) + .unzip::<_, _, Vec<_>, Vec<_>>() } else { let (tx, rx) = async_channel::unbounded(); - for i in num_input_partitions..num_output_partitions { - tx.send(i).await.expect("send partition number"); - } - Mutex::new((vec![tx], vec![rx])) + (vec![tx], vec![rx]) + }; + // Send partition number to each input partition in order to prefetch 1 RecordBatch per partition + for i in 0..num_output_partitions { + txs.iter().for_each(|tx| { + tx.send_blocking(i).expect("send partition number"); + }); } + Mutex::new((txs, rxs)) }) .await; let (partition_txs, partition_rxs) = { @@ -392,6 +389,17 @@ 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() { + // 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, + }; + // Get the partition number from the output partition let partition = output_partition_rx.recv().await.map_err(|e| { internal_datafusion_err!( @@ -406,17 +414,6 @@ impl OnDemandRepartitionExec { 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 (_, batch) = res?; let size = batch.get_array_memory_size(); From cb4c368b46367fdabc19dffdd4fc5a623a2f5fa4 Mon Sep 17 00:00:00 2001 From: Weijun-H Date: Thu, 23 Jan 2025 15:25:15 +0800 Subject: [PATCH 17/20] refactor: spawn a task to process child operator --- .../src/repartition/on_demand_repartition.rs | 75 +++++++++---------- 1 file changed, 36 insertions(+), 39 deletions(-) diff --git a/datafusion/physical-plan/src/repartition/on_demand_repartition.rs b/datafusion/physical-plan/src/repartition/on_demand_repartition.rs index 1cc13d6ea4b7..c6a4670072b5 100644 --- a/datafusion/physical-plan/src/repartition/on_demand_repartition.rs +++ b/datafusion/physical-plan/src/repartition/on_demand_repartition.rs @@ -26,7 +26,7 @@ use std::{any::Any, vec}; use super::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use super::{ - BatchPartitioner, DisplayAs, ExecutionPlanProperties, MaybeBatch, RecordBatchStream, + DisplayAs, ExecutionPlanProperties, MaybeBatch, RecordBatchStream, RepartitionExecBase, RepartitionMetrics, SendableRecordBatchStream, }; use crate::common::SharedMemoryReservation; @@ -44,14 +44,14 @@ 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; use datafusion_common::HashMap; use futures::stream::Stream; -use futures::{channel, ready, FutureExt, StreamExt, TryStreamExt}; +use futures::{ready, FutureExt, StreamExt, TryStreamExt}; use log::{debug, trace}; use parking_lot::Mutex; @@ -196,7 +196,6 @@ impl ExecutionPlan for OnDemandRepartitionExec { let stream = futures::stream::once(async move { let num_input_partitions = input.output_partitioning().partition_count(); - let num_output_partitions = partitioning.partition_count(); let input_captured = Arc::clone(&input); let metrics_captured = metrics.clone(); @@ -212,12 +211,6 @@ impl ExecutionPlan for OnDemandRepartitionExec { let (tx, rx) = async_channel::unbounded(); (vec![tx], vec![rx]) }; - // Send partition number to each input partition in order to prefetch 1 RecordBatch per partition - for i in 0..num_output_partitions { - txs.iter().for_each(|tx| { - tx.send_blocking(i).expect("send partition number"); - }); - } Mutex::new((txs, rxs)) }) .await; @@ -378,25 +371,29 @@ impl OnDemandRepartitionExec { 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(); + // let timer = metrics.fetch_time.timer(); + // let mut stream = input.execute(partition, context)?; + // timer.done(); + + let (buffer_tx, mut buffer_rx) = tokio::sync::mpsc::channel::(2); + let processing_task = tokio::task::spawn(async move { + let mut stream = input.execute(partition, context).unwrap(); + while let Some(batch) = stream.next().await { + buffer_tx.send(batch.unwrap()).await.unwrap(); + } + debug!( + "On demand input partition {} processing finished", + partition + ); + }); // 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() { - // 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?, + let batch = match buffer_rx.recv().await { + Some(result) => result, None => break, }; @@ -414,23 +411,20 @@ impl OnDemandRepartitionExec { output_channels.is_empty() ); - 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 @@ -450,12 +444,15 @@ impl OnDemandRepartitionExec { // 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.await.map_err(|e| { + internal_datafusion_err!("Error waiting for processing task to finish: {}", e) + })?; Ok(()) } } @@ -682,8 +679,8 @@ mod tests { 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_common::{cast::as_string_array, DataFusionError}; use datafusion_execution::runtime_env::RuntimeEnvBuilder; use tokio::task::JoinSet; From 49401597a5906df633ceeee373fa83bb0e00c1bf Mon Sep 17 00:00:00 2001 From: Weijun-H Date: Mon, 27 Jan 2025 22:59:35 +0800 Subject: [PATCH 18/20] chore --- .../physical-plan/src/repartition/mod.rs | 12 ---- .../src/repartition/on_demand_repartition.rs | 59 ++++++++++++++----- 2 files changed, 43 insertions(+), 28 deletions(-) diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 20399a6852cb..c27dba734f1e 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -768,18 +768,6 @@ impl ExecutionPlan for RepartitionExec { partition: usize, context: Arc, ) -> Result { - // TODO: make sure that this is only called for hash partitioning - // match self.partitioning() { - // Partitioning::Hash(_, _) => {} - // _ => { - // panic!( - // "RepartitionExec::execute should never be called directly. \ - // Partition type: {:?}", - // self.partitioning() - // ); - // } - // } - trace!( "Start {}::execute for partition: {}", self.name(), diff --git a/datafusion/physical-plan/src/repartition/on_demand_repartition.rs b/datafusion/physical-plan/src/repartition/on_demand_repartition.rs index c6a4670072b5..290894f35781 100644 --- a/datafusion/physical-plan/src/repartition/on_demand_repartition.rs +++ b/datafusion/physical-plan/src/repartition/on_demand_repartition.rs @@ -26,7 +26,7 @@ use std::{any::Any, vec}; use super::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use super::{ - DisplayAs, ExecutionPlanProperties, MaybeBatch, RecordBatchStream, + BatchPartitioner, DisplayAs, ExecutionPlanProperties, MaybeBatch, RecordBatchStream, RepartitionExecBase, RepartitionMetrics, SendableRecordBatchStream, }; use crate::common::SharedMemoryReservation; @@ -355,6 +355,36 @@ impl OnDemandRepartitionExec { }) } + async fn process_input( + input: Arc, + partition: usize, + buffer_tx: tokio::sync::mpsc::Sender, + context: Arc, + ) -> Result<()> { + let mut stream = input.execute(partition, context).map_err(|e| { + internal_datafusion_err!( + "Error executing input partition {} for on demand repartitioning: {}", + partition, + e + ) + })?; + while let Some(batch) = stream.next().await { + buffer_tx.send(batch?).await.map_err(|e| { + internal_datafusion_err!( + "Error sending batch to buffer channel for partition {}: {}", + partition, + e + ) + })?; + } + debug!( + "On demand input partition {} processing finished", + partition + ); + + Ok(()) + } + /// Pulls data from the specified input plan, feeding it to the /// output partitions based on the desired partitioning /// @@ -371,22 +401,19 @@ impl OnDemandRepartitionExec { metrics: RepartitionMetrics, context: Arc, ) -> Result<()> { - // execute the child operator - // let timer = metrics.fetch_time.timer(); - // let mut stream = input.execute(partition, context)?; - // timer.done(); + let _ = BatchPartitioner::try_new( + partitioning.clone(), + metrics.repartition_time.clone(), + )?; + // execute the child operator in a separate task let (buffer_tx, mut buffer_rx) = tokio::sync::mpsc::channel::(2); - let processing_task = tokio::task::spawn(async move { - let mut stream = input.execute(partition, context).unwrap(); - while let Some(batch) = stream.next().await { - buffer_tx.send(batch.unwrap()).await.unwrap(); - } - debug!( - "On demand input partition {} processing finished", - partition - ); - }); + let processing_task = tokio::task::spawn(Self::process_input( + Arc::clone(&input), + partition, + buffer_tx, + Arc::clone(&context), + )); // While there are still outputs to send to, keep pulling inputs let mut batches_until_yield = partitioning.partition_count(); @@ -452,7 +479,7 @@ impl OnDemandRepartitionExec { processing_task.await.map_err(|e| { internal_datafusion_err!("Error waiting for processing task to finish: {}", e) - })?; + })??; Ok(()) } } From b2c89f1da95f1ed849844f5252dc0283f804d6bf Mon Sep 17 00:00:00 2001 From: Weijun-H Date: Fri, 31 Jan 2025 10:47:37 +0800 Subject: [PATCH 19/20] chore: Only compare OnDemandRepartition effect --- .../src/sorts/sort_preserving_merge.rs | 39 +++++-------------- 1 file changed, 10 insertions(+), 29 deletions(-) diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index 58b4958145bb..1ddcca8e73c7 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -82,7 +82,6 @@ pub struct SortPreservingMergeExec { cache: PlanProperties, /// Configuration parameter to enable round-robin selection of tied winners of loser tree. enable_round_robin_repartition: bool, - enable_pull_based_execution: bool, } impl SortPreservingMergeExec { @@ -96,7 +95,6 @@ impl SortPreservingMergeExec { fetch: None, cache, enable_round_robin_repartition: true, - enable_pull_based_execution: true, } } @@ -192,7 +190,6 @@ impl ExecutionPlan for SortPreservingMergeExec { fetch: limit, cache: self.cache.clone(), enable_round_robin_repartition: true, - enable_pull_based_execution: true, })) } @@ -274,34 +271,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)) From 4bbb785bd9922a7bf5ebd137b36f2db502e04259 Mon Sep 17 00:00:00 2001 From: Weijun-H Date: Fri, 31 Jan 2025 11:51:21 +0800 Subject: [PATCH 20/20] chore: use bounded channel --- .../src/repartition/on_demand_repartition.rs | 40 +++++++++++-------- 1 file changed, 24 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 290894f35781..17fd8f4e6907 100644 --- a/datafusion/physical-plan/src/repartition/on_demand_repartition.rs +++ b/datafusion/physical-plan/src/repartition/on_demand_repartition.rs @@ -205,10 +205,10 @@ impl ExecutionPlan for OnDemandRepartitionExec { .get_or_init(|| async move { let (txs, rxs) = if preserve_order { (0..num_input_partitions) - .map(|_| async_channel::unbounded()) + .map(|_| async_channel::bounded(2)) .unzip::<_, _, Vec<_>, Vec<_>>() } else { - let (tx, rx) = async_channel::unbounded(); + let (tx, rx) = async_channel::bounded(2); (vec![tx], vec![rx]) }; Mutex::new((txs, rxs)) @@ -521,13 +521,17 @@ impl Stream for OnDemandPerPartitionStream { "On Demand Repartition per partition poll {}, start, is_requested {}", self.partition, self.is_requested, ); - if !self.sender.is_closed() && !self.is_requested { - self.sender.send_blocking(self.partition).map_err(|e| { - internal_datafusion_err!( - "Error sending partition number to input partitions: {}", - e - ) - })?; + if !self.is_requested { + match self.sender.try_send(self.partition) { + Ok(_) => {} + Err(e) => { + debug!( + "On Demand Repartition poll {}, error sending partition number: {}", + self.partition, e + ); + return Poll::Ready(None); + } + } debug!( "On Demand Repartition per partition poll {}, send partition number", self.partition @@ -621,13 +625,17 @@ impl Stream for OnDemandRepartitionStream { ); loop { // Send partition number to input partitions - if !self.sender.is_closed() && !self.is_requested { - self.sender.send_blocking(self.partition).map_err(|e| { - internal_datafusion_err!( - "Error sending partition number to input partitions: {}", - e - ) - })?; + if !self.is_requested { + match self.sender.try_send(self.partition) { + Ok(_) => {} + Err(e) => { + debug!( + "On Demand Repartition poll {}, error sending partition number: {}", + self.partition, e + ); + return Poll::Ready(None); + } + } debug!( "On Demand Repartition poll {}, send partition number", self.partition