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),