diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index 86567f487c1b..ae9c82fa1249 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -1592,6 +1592,7 @@ dependencies = [ "datafusion-execution", "datafusion-expr", "datafusion-expr-common", + "datafusion-functions-aggregate", "datafusion-physical-expr", "datafusion-physical-expr-common", "datafusion-physical-plan", diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs deleted file mode 100644 index 4681efaf16b2..000000000000 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ /dev/null @@ -1,4662 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -//! EnforceDistribution optimizer rule inspects the physical plan with respect -//! to distribution requirements and adds [`RepartitionExec`]s to satisfy them -//! when necessary. If increasing parallelism is beneficial (and also desirable -//! according to the configuration), this rule increases partition counts in -//! the physical plan. - -use std::fmt::Debug; -use std::sync::Arc; - -use crate::config::ConfigOptions; -use crate::error::Result; -use crate::physical_plan::aggregates::{AggregateExec, AggregateMode, PhysicalGroupBy}; -use crate::physical_plan::coalesce_partitions::CoalescePartitionsExec; -use crate::physical_plan::joins::{ - CrossJoinExec, HashJoinExec, PartitionMode, SortMergeJoinExec, -}; -use crate::physical_plan::projection::ProjectionExec; -use crate::physical_plan::repartition::RepartitionExec; -use crate::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; -use crate::physical_plan::tree_node::PlanContext; -use crate::physical_plan::union::{can_interleave, InterleaveExec, UnionExec}; -use crate::physical_plan::windows::WindowAggExec; -use crate::physical_plan::{Distribution, ExecutionPlan, Partitioning}; - -use arrow::compute::SortOptions; -use datafusion_common::stats::Precision; -use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; -use datafusion_expr::logical_plan::JoinType; -use datafusion_physical_expr::expressions::{Column, NoOp}; -use datafusion_physical_expr::utils::map_columns_before_projection; -use datafusion_physical_expr::{ - physical_exprs_equal, EquivalenceProperties, PhysicalExpr, PhysicalExprRef, -}; -use datafusion_physical_expr_common::sort_expr::LexOrdering; -use datafusion_physical_optimizer::output_requirements::OutputRequirementExec; -use datafusion_physical_optimizer::utils::{ - add_sort_above_with_check, is_coalesce_partitions, is_repartition, - is_sort_preserving_merge, -}; -use datafusion_physical_optimizer::PhysicalOptimizerRule; -use datafusion_physical_plan::execution_plan::EmissionType; -use datafusion_physical_plan::windows::{get_best_fitting_window, BoundedWindowAggExec}; -use datafusion_physical_plan::ExecutionPlanProperties; - -use itertools::izip; - -/// 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. -/// -/// For example, given an input such as: -/// -/// -/// ```text -/// ┌─────────────────────────────────┐ -/// │ │ -/// │ ExecutionPlan │ -/// │ │ -/// └─────────────────────────────────┘ -/// ▲ ▲ -/// │ │ -/// ┌─────┘ └─────┐ -/// │ │ -/// │ │ -/// │ │ -/// ┌───────────┐ ┌───────────┐ -/// │ │ │ │ -/// │ batch A1 │ │ batch B1 │ -/// │ │ │ │ -/// ├───────────┤ ├───────────┤ -/// │ │ │ │ -/// │ batch A2 │ │ batch B2 │ -/// │ │ │ │ -/// ├───────────┤ ├───────────┤ -/// │ │ │ │ -/// │ batch A3 │ │ batch B3 │ -/// │ │ │ │ -/// └───────────┘ └───────────┘ -/// -/// Input Input -/// A B -/// ``` -/// -/// This rule will attempt to add a `RepartitionExec` to increase parallelism -/// (to 3, in this case) and create the following arrangement: -/// -/// ```text -/// ┌─────────────────────────────────┐ -/// │ │ -/// │ ExecutionPlan │ -/// │ │ -/// └─────────────────────────────────┘ -/// ▲ ▲ ▲ Input now has 3 -/// │ │ │ partitions -/// ┌───────┘ │ └───────┐ -/// │ │ │ -/// │ │ │ -/// ┌───────────┐ ┌───────────┐ ┌───────────┐ -/// │ │ │ │ │ │ -/// │ batch A1 │ │ batch A3 │ │ batch B3 │ -/// │ │ │ │ │ │ -/// ├───────────┤ ├───────────┤ ├───────────┤ -/// │ │ │ │ │ │ -/// │ batch B2 │ │ batch B1 │ │ batch A2 │ -/// │ │ │ │ │ │ -/// └───────────┘ └───────────┘ └───────────┘ -/// ▲ ▲ ▲ -/// │ │ │ -/// └─────────┐ │ ┌──────────┘ -/// │ │ │ -/// │ │ │ -/// ┌─────────────────────────────────┐ batches are -/// │ RepartitionExec(3) │ repartitioned -/// │ RoundRobin │ -/// │ │ -/// └─────────────────────────────────┘ -/// ▲ ▲ -/// │ │ -/// ┌─────┘ └─────┐ -/// │ │ -/// │ │ -/// │ │ -/// ┌───────────┐ ┌───────────┐ -/// │ │ │ │ -/// │ batch A1 │ │ batch B1 │ -/// │ │ │ │ -/// ├───────────┤ ├───────────┤ -/// │ │ │ │ -/// │ batch A2 │ │ batch B2 │ -/// │ │ │ │ -/// ├───────────┤ ├───────────┤ -/// │ │ │ │ -/// │ batch A3 │ │ batch B3 │ -/// │ │ │ │ -/// └───────────┘ └───────────┘ -/// -/// -/// Input Input -/// A B -/// ``` -/// -/// The `EnforceDistribution` rule -/// - is idempotent; i.e. it can be applied multiple times, each time producing -/// the same result. -/// - always produces a valid plan in terms of distribution requirements. Its -/// input plan can be valid or invalid with respect to distribution requirements, -/// but the output plan will always be valid. -/// - produces a valid plan in terms of ordering requirements, *if* its input is -/// a valid plan in terms of ordering requirements. If the input plan is invalid, -/// this rule does not attempt to fix it as doing so is the responsibility of the -/// `EnforceSorting` rule. -/// -/// Note that distribution requirements are met in the strictest way. This may -/// result in more than strictly necessary [`RepartitionExec`]s in the plan, but -/// meeting the requirements in the strictest way may help avoid possible data -/// skew in joins. -/// -/// For example for a hash join with keys (a, b, c), the required Distribution(a, b, c) -/// can be satisfied by several alternative partitioning ways: (a, b, c), (a, b), -/// (a, c), (b, c), (a), (b), (c) and ( ). -/// -/// This rule only chooses the exact match and satisfies the Distribution(a, b, c) -/// by a HashPartition(a, b, c). -#[derive(Default, Debug)] -pub struct EnforceDistribution {} - -impl EnforceDistribution { - #[allow(missing_docs)] - pub fn new() -> Self { - Self {} - } -} - -impl PhysicalOptimizerRule for EnforceDistribution { - fn optimize( - &self, - plan: Arc, - config: &ConfigOptions, - ) -> Result> { - let top_down_join_key_reordering = config.optimizer.top_down_join_key_reordering; - - let adjusted = if top_down_join_key_reordering { - // Run a top-down process to adjust input key ordering recursively - let plan_requirements = PlanWithKeyRequirements::new_default(plan); - let adjusted = plan_requirements - .transform_down(adjust_input_keys_ordering) - .data()?; - adjusted.plan - } else { - // Run a bottom-up process - plan.transform_up(|plan| { - Ok(Transformed::yes(reorder_join_keys_to_inputs(plan)?)) - }) - .data()? - }; - - let distribution_context = DistributionContext::new_default(adjusted); - // Distribution enforcement needs to be applied bottom-up. - let distribution_context = distribution_context - .transform_up(|distribution_context| { - ensure_distribution(distribution_context, config) - }) - .data()?; - Ok(distribution_context.plan) - } - - fn name(&self) -> &str { - "EnforceDistribution" - } - - fn schema_check(&self) -> bool { - true - } -} - -/// When the physical planner creates the Joins, the ordering of join keys is from the original query. -/// That might not match with the output partitioning of the join node's children -/// A Top-Down process will use this method to adjust children's output partitioning based on the parent key reordering requirements: -/// -/// Example: -/// TopJoin on (a, b, c) -/// bottom left join on(b, a, c) -/// bottom right join on(c, b, a) -/// -/// Will be adjusted to: -/// TopJoin on (a, b, c) -/// bottom left join on(a, b, c) -/// bottom right join on(a, b, c) -/// -/// Example: -/// TopJoin on (a, b, c) -/// Agg1 group by (b, a, c) -/// Agg2 group by (c, b, a) -/// -/// Will be adjusted to: -/// TopJoin on (a, b, c) -/// Projection(b, a, c) -/// Agg1 group by (a, b, c) -/// Projection(c, b, a) -/// Agg2 group by (a, b, c) -/// -/// Following is the explanation of the reordering process: -/// -/// 1) If the current plan is Partitioned HashJoin, SortMergeJoin, check whether the requirements can be satisfied by adjusting join keys ordering: -/// Requirements can not be satisfied, clear the current requirements, generate new requirements(to pushdown) based on the current join keys, return the unchanged plan. -/// Requirements is already satisfied, clear the current requirements, generate new requirements(to pushdown) based on the current join keys, return the unchanged plan. -/// Requirements can be satisfied by adjusting keys ordering, clear the current requirements, generate new requirements(to pushdown) based on the adjusted join keys, return the changed plan. -/// -/// 2) If the current plan is Aggregation, check whether the requirements can be satisfied by adjusting group by keys ordering: -/// Requirements can not be satisfied, clear all the requirements, return the unchanged plan. -/// Requirements is already satisfied, clear all the requirements, return the unchanged plan. -/// Requirements can be satisfied by adjusting keys ordering, clear all the requirements, return the changed plan. -/// -/// 3) If the current plan is RepartitionExec, CoalescePartitionsExec or WindowAggExec, clear all the requirements, return the unchanged plan -/// 4) If the current plan is Projection, transform the requirements to the columns before the Projection and push down requirements -/// 5) For other types of operators, by default, pushdown the parent requirements to children. -/// -fn adjust_input_keys_ordering( - mut requirements: PlanWithKeyRequirements, -) -> Result> { - let plan = Arc::clone(&requirements.plan); - - if let Some(HashJoinExec { - left, - right, - on, - filter, - join_type, - projection, - mode, - null_equals_null, - .. - }) = plan.as_any().downcast_ref::() - { - match mode { - PartitionMode::Partitioned => { - let join_constructor = |new_conditions: ( - Vec<(PhysicalExprRef, PhysicalExprRef)>, - Vec, - )| { - HashJoinExec::try_new( - Arc::clone(left), - Arc::clone(right), - new_conditions.0, - filter.clone(), - join_type, - // TODO: although projection is not used in the join here, because projection pushdown is after enforce_distribution. Maybe we need to handle it later. Same as filter. - projection.clone(), - PartitionMode::Partitioned, - *null_equals_null, - ) - .map(|e| Arc::new(e) as _) - }; - return reorder_partitioned_join_keys( - requirements, - on, - &[], - &join_constructor, - ) - .map(Transformed::yes); - } - PartitionMode::CollectLeft => { - // Push down requirements to the right side - requirements.children[1].data = match join_type { - JoinType::Inner | JoinType::Right => shift_right_required( - &requirements.data, - left.schema().fields().len(), - ) - .unwrap_or_default(), - JoinType::RightSemi | JoinType::RightAnti => { - requirements.data.clone() - } - JoinType::Left - | JoinType::LeftSemi - | JoinType::LeftAnti - | JoinType::Full - | JoinType::LeftMark => vec![], - }; - } - PartitionMode::Auto => { - // Can not satisfy, clear the current requirements and generate new empty requirements - requirements.data.clear(); - } - } - } else if let Some(CrossJoinExec { left, .. }) = - plan.as_any().downcast_ref::() - { - let left_columns_len = left.schema().fields().len(); - // Push down requirements to the right side - requirements.children[1].data = - shift_right_required(&requirements.data, left_columns_len) - .unwrap_or_default(); - } else if let Some(SortMergeJoinExec { - left, - right, - on, - filter, - join_type, - sort_options, - null_equals_null, - .. - }) = plan.as_any().downcast_ref::() - { - let join_constructor = |new_conditions: ( - Vec<(PhysicalExprRef, PhysicalExprRef)>, - Vec, - )| { - SortMergeJoinExec::try_new( - Arc::clone(left), - Arc::clone(right), - new_conditions.0, - filter.clone(), - *join_type, - new_conditions.1, - *null_equals_null, - ) - .map(|e| Arc::new(e) as _) - }; - return reorder_partitioned_join_keys( - requirements, - on, - sort_options, - &join_constructor, - ) - .map(Transformed::yes); - } else if let Some(aggregate_exec) = plan.as_any().downcast_ref::() { - if !requirements.data.is_empty() { - if aggregate_exec.mode() == &AggregateMode::FinalPartitioned { - return reorder_aggregate_keys(requirements, aggregate_exec) - .map(Transformed::yes); - } else { - requirements.data.clear(); - } - } else { - // Keep everything unchanged - return Ok(Transformed::no(requirements)); - } - } else if let Some(proj) = plan.as_any().downcast_ref::() { - let expr = proj.expr(); - // For Projection, we need to transform the requirements to the columns before the Projection - // And then to push down the requirements - // Construct a mapping from new name to the original Column - let new_required = map_columns_before_projection(&requirements.data, expr); - if new_required.len() == requirements.data.len() { - requirements.children[0].data = new_required; - } else { - // Can not satisfy, clear the current requirements and generate new empty requirements - requirements.data.clear(); - } - } else if plan.as_any().downcast_ref::().is_some() - || plan - .as_any() - .downcast_ref::() - .is_some() - || plan.as_any().downcast_ref::().is_some() - { - requirements.data.clear(); - } else { - // By default, push down the parent requirements to children - for child in requirements.children.iter_mut() { - child.data.clone_from(&requirements.data); - } - } - Ok(Transformed::yes(requirements)) -} - -fn reorder_partitioned_join_keys( - mut join_plan: PlanWithKeyRequirements, - on: &[(PhysicalExprRef, PhysicalExprRef)], - sort_options: &[SortOptions], - join_constructor: &F, -) -> Result -where - F: Fn( - (Vec<(PhysicalExprRef, PhysicalExprRef)>, Vec), - ) -> Result>, -{ - let parent_required = &join_plan.data; - let join_key_pairs = extract_join_keys(on); - let eq_properties = join_plan.plan.equivalence_properties(); - - let ( - JoinKeyPairs { - left_keys, - right_keys, - }, - positions, - ) = try_reorder(join_key_pairs, parent_required, eq_properties); - - if let Some(positions) = positions { - if !positions.is_empty() { - let new_join_on = new_join_conditions(&left_keys, &right_keys); - let new_sort_options = (0..sort_options.len()) - .map(|idx| sort_options[positions[idx]]) - .collect(); - join_plan.plan = join_constructor((new_join_on, new_sort_options))?; - } - } - - join_plan.children[0].data = left_keys; - join_plan.children[1].data = right_keys; - Ok(join_plan) -} - -fn reorder_aggregate_keys( - mut agg_node: PlanWithKeyRequirements, - agg_exec: &AggregateExec, -) -> Result { - let parent_required = &agg_node.data; - let output_columns = agg_exec - .group_expr() - .expr() - .iter() - .enumerate() - .map(|(index, (_, name))| Column::new(name, index)) - .collect::>(); - - let output_exprs = output_columns - .iter() - .map(|c| Arc::new(c.clone()) as _) - .collect::>(); - - if parent_required.len() == output_exprs.len() - && agg_exec.group_expr().null_expr().is_empty() - && !physical_exprs_equal(&output_exprs, parent_required) - { - if let Some(positions) = expected_expr_positions(&output_exprs, parent_required) { - if let Some(agg_exec) = - agg_exec.input().as_any().downcast_ref::() - { - if matches!(agg_exec.mode(), &AggregateMode::Partial) { - let group_exprs = agg_exec.group_expr().expr(); - let new_group_exprs = positions - .into_iter() - .map(|idx| group_exprs[idx].clone()) - .collect(); - let partial_agg = Arc::new(AggregateExec::try_new( - AggregateMode::Partial, - PhysicalGroupBy::new_single(new_group_exprs), - agg_exec.aggr_expr().to_vec(), - agg_exec.filter_expr().to_vec(), - Arc::clone(agg_exec.input()), - Arc::clone(&agg_exec.input_schema), - )?); - // Build new group expressions that correspond to the output - // of the "reordered" aggregator: - let group_exprs = partial_agg.group_expr().expr(); - let new_group_by = PhysicalGroupBy::new_single( - partial_agg - .output_group_expr() - .into_iter() - .enumerate() - .map(|(idx, expr)| (expr, group_exprs[idx].1.clone())) - .collect(), - ); - let new_final_agg = Arc::new(AggregateExec::try_new( - AggregateMode::FinalPartitioned, - new_group_by, - agg_exec.aggr_expr().to_vec(), - agg_exec.filter_expr().to_vec(), - Arc::clone(&partial_agg) as _, - agg_exec.input_schema(), - )?); - - agg_node.plan = Arc::clone(&new_final_agg) as _; - agg_node.data.clear(); - agg_node.children = vec![PlanWithKeyRequirements::new( - partial_agg as _, - vec![], - agg_node.children.swap_remove(0).children, - )]; - - // Need to create a new projection to change the expr ordering back - let agg_schema = new_final_agg.schema(); - let mut proj_exprs = output_columns - .iter() - .map(|col| { - let name = col.name(); - let index = agg_schema.index_of(name)?; - Ok((Arc::new(Column::new(name, index)) as _, name.to_owned())) - }) - .collect::>>()?; - let agg_fields = agg_schema.fields(); - for (idx, field) in - agg_fields.iter().enumerate().skip(output_columns.len()) - { - let name = field.name(); - let plan = Arc::new(Column::new(name, idx)) as _; - proj_exprs.push((plan, name.clone())) - } - return ProjectionExec::try_new(proj_exprs, new_final_agg).map(|p| { - PlanWithKeyRequirements::new(Arc::new(p), vec![], vec![agg_node]) - }); - } - } - } - } - Ok(agg_node) -} - -fn shift_right_required( - parent_required: &[Arc], - left_columns_len: usize, -) -> Option>> { - let new_right_required = parent_required - .iter() - .filter_map(|r| { - r.as_any().downcast_ref::().and_then(|col| { - col.index() - .checked_sub(left_columns_len) - .map(|index| Arc::new(Column::new(col.name(), index)) as _) - }) - }) - .collect::>(); - - // if the parent required are all coming from the right side, the requirements can be pushdown - (new_right_required.len() == parent_required.len()).then_some(new_right_required) -} - -/// When the physical planner creates the Joins, the ordering of join keys is from the original query. -/// That might not match with the output partitioning of the join node's children -/// This method will try to change the ordering of the join keys to match with the -/// partitioning of the join nodes' children. If it can not match with both sides, it will try to -/// match with one, either the left side or the right side. -/// -/// Example: -/// TopJoin on (a, b, c) -/// bottom left join on(b, a, c) -/// bottom right join on(c, b, a) -/// -/// Will be adjusted to: -/// TopJoin on (b, a, c) -/// bottom left join on(b, a, c) -/// bottom right join on(c, b, a) -/// -/// Compared to the Top-Down reordering process, this Bottom-Up approach is much simpler, but might not reach a best result. -/// The Bottom-Up approach will be useful in future if we plan to support storage partition-wised Joins. -/// In that case, the datasources/tables might be pre-partitioned and we can't adjust the key ordering of the datasources -/// and then can't apply the Top-Down reordering process. -pub(crate) fn reorder_join_keys_to_inputs( - plan: Arc, -) -> Result> { - let plan_any = plan.as_any(); - if let Some(HashJoinExec { - left, - right, - on, - filter, - join_type, - projection, - mode, - null_equals_null, - .. - }) = plan_any.downcast_ref::() - { - if matches!(mode, PartitionMode::Partitioned) { - let (join_keys, positions) = reorder_current_join_keys( - extract_join_keys(on), - Some(left.output_partitioning()), - Some(right.output_partitioning()), - left.equivalence_properties(), - right.equivalence_properties(), - ); - if positions.is_some_and(|idxs| !idxs.is_empty()) { - let JoinKeyPairs { - left_keys, - right_keys, - } = join_keys; - let new_join_on = new_join_conditions(&left_keys, &right_keys); - return Ok(Arc::new(HashJoinExec::try_new( - Arc::clone(left), - Arc::clone(right), - new_join_on, - filter.clone(), - join_type, - projection.clone(), - PartitionMode::Partitioned, - *null_equals_null, - )?)); - } - } - } else if let Some(SortMergeJoinExec { - left, - right, - on, - filter, - join_type, - sort_options, - null_equals_null, - .. - }) = plan_any.downcast_ref::() - { - let (join_keys, positions) = reorder_current_join_keys( - extract_join_keys(on), - Some(left.output_partitioning()), - Some(right.output_partitioning()), - left.equivalence_properties(), - right.equivalence_properties(), - ); - if let Some(positions) = positions { - if !positions.is_empty() { - let JoinKeyPairs { - left_keys, - right_keys, - } = join_keys; - let new_join_on = new_join_conditions(&left_keys, &right_keys); - let new_sort_options = (0..sort_options.len()) - .map(|idx| sort_options[positions[idx]]) - .collect(); - return SortMergeJoinExec::try_new( - Arc::clone(left), - Arc::clone(right), - new_join_on, - filter.clone(), - *join_type, - new_sort_options, - *null_equals_null, - ) - .map(|smj| Arc::new(smj) as _); - } - } - } - Ok(plan) -} - -/// Reorder the current join keys ordering based on either left partition or right partition -fn reorder_current_join_keys( - join_keys: JoinKeyPairs, - left_partition: Option<&Partitioning>, - right_partition: Option<&Partitioning>, - left_equivalence_properties: &EquivalenceProperties, - right_equivalence_properties: &EquivalenceProperties, -) -> (JoinKeyPairs, Option>) { - match (left_partition, right_partition) { - (Some(Partitioning::Hash(left_exprs, _)), _) => { - match try_reorder(join_keys, left_exprs, left_equivalence_properties) { - (join_keys, None) => reorder_current_join_keys( - join_keys, - None, - right_partition, - left_equivalence_properties, - right_equivalence_properties, - ), - result => result, - } - } - (_, Some(Partitioning::Hash(right_exprs, _))) => { - try_reorder(join_keys, right_exprs, right_equivalence_properties) - } - _ => (join_keys, None), - } -} - -fn try_reorder( - join_keys: JoinKeyPairs, - expected: &[Arc], - equivalence_properties: &EquivalenceProperties, -) -> (JoinKeyPairs, Option>) { - let eq_groups = equivalence_properties.eq_group(); - let mut normalized_expected = vec![]; - let mut normalized_left_keys = vec![]; - let mut normalized_right_keys = vec![]; - if join_keys.left_keys.len() != expected.len() { - return (join_keys, None); - } - if physical_exprs_equal(expected, &join_keys.left_keys) - || physical_exprs_equal(expected, &join_keys.right_keys) - { - return (join_keys, Some(vec![])); - } else if !equivalence_properties.eq_group().is_empty() { - normalized_expected = expected - .iter() - .map(|e| eq_groups.normalize_expr(Arc::clone(e))) - .collect(); - - normalized_left_keys = join_keys - .left_keys - .iter() - .map(|e| eq_groups.normalize_expr(Arc::clone(e))) - .collect(); - - normalized_right_keys = join_keys - .right_keys - .iter() - .map(|e| eq_groups.normalize_expr(Arc::clone(e))) - .collect(); - - if physical_exprs_equal(&normalized_expected, &normalized_left_keys) - || physical_exprs_equal(&normalized_expected, &normalized_right_keys) - { - return (join_keys, Some(vec![])); - } - } - - let Some(positions) = expected_expr_positions(&join_keys.left_keys, expected) - .or_else(|| expected_expr_positions(&join_keys.right_keys, expected)) - .or_else(|| expected_expr_positions(&normalized_left_keys, &normalized_expected)) - .or_else(|| { - expected_expr_positions(&normalized_right_keys, &normalized_expected) - }) - else { - return (join_keys, None); - }; - - let mut new_left_keys = vec![]; - let mut new_right_keys = vec![]; - for pos in positions.iter() { - new_left_keys.push(Arc::clone(&join_keys.left_keys[*pos])); - new_right_keys.push(Arc::clone(&join_keys.right_keys[*pos])); - } - let pairs = JoinKeyPairs { - left_keys: new_left_keys, - right_keys: new_right_keys, - }; - - (pairs, Some(positions)) -} - -/// Return the expected expressions positions. -/// For example, the current expressions are ['c', 'a', 'a', b'], the expected expressions are ['b', 'c', 'a', 'a'], -/// -/// This method will return a Vec [3, 0, 1, 2] -fn expected_expr_positions( - current: &[Arc], - expected: &[Arc], -) -> Option> { - if current.is_empty() || expected.is_empty() { - return None; - } - let mut indexes: Vec = vec![]; - let mut current = current.to_vec(); - for expr in expected.iter() { - // Find the position of the expected expr in the current expressions - if let Some(expected_position) = current.iter().position(|e| e.eq(expr)) { - current[expected_position] = Arc::new(NoOp::new()); - indexes.push(expected_position); - } else { - return None; - } - } - Some(indexes) -} - -fn extract_join_keys(on: &[(PhysicalExprRef, PhysicalExprRef)]) -> JoinKeyPairs { - let (left_keys, right_keys) = on - .iter() - .map(|(l, r)| (Arc::clone(l) as _, Arc::clone(r) as _)) - .unzip(); - JoinKeyPairs { - left_keys, - right_keys, - } -} - -fn new_join_conditions( - new_left_keys: &[Arc], - new_right_keys: &[Arc], -) -> Vec<(PhysicalExprRef, PhysicalExprRef)> { - new_left_keys - .iter() - .zip(new_right_keys.iter()) - .map(|(l_key, r_key)| (Arc::clone(l_key), Arc::clone(r_key))) - .collect() -} - -/// Adds RoundRobin repartition operator to the plan increase parallelism. -/// -/// # Arguments -/// -/// * `input`: Current node. -/// * `n_target`: desired target partition number, if partition number of the -/// current executor is less than this value. Partition number will be increased. -/// -/// # Returns -/// -/// A [`Result`] object that contains new execution plan where the desired -/// partition number is achieved by adding a RoundRobin repartition. -fn add_roundrobin_on_top( - input: DistributionContext, - n_target: usize, -) -> Result { - // Adding repartition is helpful: - if input.plan.output_partitioning().partition_count() < n_target { - // When there is an existing ordering, we preserve ordering - // during repartition. This will be un-done in the future - // If any of the following conditions is true - // - Preserving ordering is not helpful in terms of satisfying ordering requirements - // - Usage of order preserving variants is not desirable - // (determined by flag `config.optimizer.prefer_existing_sort`) - let partitioning = Partitioning::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) - } -} - -/// Adds a hash repartition operator: -/// - to increase parallelism, and/or -/// - to satisfy requirements of the subsequent operators. -/// -/// Repartition(Hash) is added on top of operator `input`. -/// -/// # Arguments -/// -/// * `input`: Current node. -/// * `hash_exprs`: Stores Physical Exprs that are used during hashing. -/// * `n_target`: desired target partition number, if partition number of the -/// current executor is less than this value. Partition number will be increased. -/// -/// # Returns -/// -/// A [`Result`] object that contains new execution plan where the desired -/// distribution is satisfied by adding a Hash repartition. -fn add_hash_on_top( - input: DistributionContext, - hash_exprs: Vec>, - n_target: usize, -) -> Result { - // Early return if hash repartition is unnecessary - // `RepartitionExec: partitioning=Hash([...], 1), input_partitions=1` is unnecessary. - if n_target == 1 && input.plan.output_partitioning().partition_count() == 1 { - return Ok(input); - } - - let dist = Distribution::HashPartitioned(hash_exprs); - let satisfied = input - .plan - .output_partitioning() - .satisfy(&dist, input.plan.equivalence_properties()); - - // Add hash repartitioning when: - // - The hash distribution requirement is not satisfied, or - // - We can increase parallelism by adding hash partitioning. - if !satisfied || n_target > input.plan.output_partitioning().partition_count() { - // When there is an existing ordering, we preserve ordering during - // repartition. This will be rolled back 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 (per the flag - // `config.optimizer.prefer_existing_sort`). - let partitioning = dist.create_partitioning(n_target); - let repartition = - RepartitionExec::try_new(Arc::clone(&input.plan), partitioning)? - .with_preserve_order(); - let plan = Arc::new(repartition) as _; - - return Ok(DistributionContext::new(plan, true, vec![input])); - } - - Ok(input) -} - -/// Adds a [`SortPreservingMergeExec`] operator on top of input executor -/// to satisfy single distribution requirement. -/// -/// # Arguments -/// -/// * `input`: Current node. -/// -/// # Returns -/// -/// Updated node with an execution plan, where desired single -/// distribution is satisfied by adding [`SortPreservingMergeExec`]. -fn add_spm_on_top(input: DistributionContext) -> DistributionContext { - // Add SortPreservingMerge only when partition count is larger than 1. - if input.plan.output_partitioning().partition_count() > 1 { - // When there is an existing ordering, we preserve ordering - // when decreasing partitions. 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.bounded_order_preserving_variants`) - let should_preserve_ordering = input.plan.output_ordering().is_some(); - - let new_plan = if should_preserve_ordering { - Arc::new(SortPreservingMergeExec::new( - input - .plan - .output_ordering() - .unwrap_or(&LexOrdering::default()) - .clone(), - Arc::clone(&input.plan), - )) as _ - } else { - Arc::new(CoalescePartitionsExec::new(Arc::clone(&input.plan))) as _ - }; - - DistributionContext::new(new_plan, true, vec![input]) - } else { - input - } -} - -/// Updates the physical plan inside [`DistributionContext`] so that distribution -/// changing operators are removed from the top. If they are necessary, they will -/// be added in subsequent stages. -/// -/// Assume that following plan is given: -/// ```text -/// "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10", -/// " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", -/// " ParquetExec: file_groups={2 groups: \[\[x], \[y]]}, projection=\[a, b, c, d, e], output_ordering=\[a@0 ASC]", -/// ``` -/// -/// Since `RepartitionExec`s change the distribution, this function removes -/// them and returns following plan: -/// -/// ```text -/// "ParquetExec: file_groups={2 groups: \[\[x], \[y]]}, projection=\[a, b, c, d, e], output_ordering=\[a@0 ASC]", -/// ``` -fn remove_dist_changing_operators( - mut distribution_context: DistributionContext, -) -> Result { - while is_repartition(&distribution_context.plan) - || is_coalesce_partitions(&distribution_context.plan) - || is_sort_preserving_merge(&distribution_context.plan) - { - // All of above operators have a single child. First child is only child. - // Remove any distribution changing operators at the beginning: - distribution_context = distribution_context.children.swap_remove(0); - // Note that they will be re-inserted later on if necessary or helpful. - } - - Ok(distribution_context) -} - -/// Updates the [`DistributionContext`] if preserving ordering while changing partitioning is not helpful or desirable. -/// -/// Assume that following plan is given: -/// ```text -/// "SortPreservingMergeExec: \[a@0 ASC]" -/// " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10, preserve_order=true", -/// " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2, preserve_order=true", -/// " ParquetExec: file_groups={2 groups: \[\[x], \[y]]}, projection=\[a, b, c, d, e], output_ordering=\[a@0 ASC]", -/// ``` -/// -/// This function converts plan above to the following: -/// -/// ```text -/// "CoalescePartitionsExec" -/// " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10", -/// " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", -/// " ParquetExec: file_groups={2 groups: \[\[x], \[y]]}, projection=\[a, b, c, d, e], output_ordering=\[a@0 ASC]", -/// ``` -fn replace_order_preserving_variants( - mut context: DistributionContext, -) -> Result { - context.children = context - .children - .into_iter() - .map(|child| { - if child.data { - replace_order_preserving_variants(child) - } else { - Ok(child) - } - }) - .collect::>>()?; - - if is_sort_preserving_merge(&context.plan) { - let child_plan = Arc::clone(&context.children[0].plan); - context.plan = Arc::new(CoalescePartitionsExec::new(child_plan)); - return Ok(context); - } else if let Some(repartition) = - 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(), - )?); - return Ok(context); - } - } - - context.update_plan_from_children() -} - -/// A struct to keep track of repartition requirements for each child node. -struct RepartitionRequirementStatus { - /// The distribution requirement for the node. - requirement: Distribution, - /// Designates whether round robin partitioning is theoretically beneficial; - /// i.e. the operator can actually utilize parallelism. - roundrobin_beneficial: bool, - /// Designates whether round robin partitioning is beneficial according to - /// the statistical information we have on the number of rows. - roundrobin_beneficial_stats: bool, - /// Designates whether hash partitioning is necessary. - hash_necessary: bool, -} - -/// Calculates the `RepartitionRequirementStatus` for each children to generate -/// consistent and sensible (in terms of performance) distribution requirements. -/// As an example, a hash join's left (build) child might produce -/// -/// ```text -/// RepartitionRequirementStatus { -/// .., -/// hash_necessary: true -/// } -/// ``` -/// -/// while its right (probe) child might have very few rows and produce: -/// -/// ```text -/// RepartitionRequirementStatus { -/// .., -/// hash_necessary: false -/// } -/// ``` -/// -/// These statuses are not consistent as all children should agree on hash -/// partitioning. This function aligns the statuses to generate consistent -/// hash partitions for each children. After alignment, the right child's -/// status would turn into: -/// -/// ```text -/// RepartitionRequirementStatus { -/// .., -/// hash_necessary: true -/// } -/// ``` -fn get_repartition_requirement_status( - plan: &Arc, - batch_size: usize, - should_use_estimates: bool, -) -> Result> { - let mut needs_alignment = false; - let children = plan.children(); - let rr_beneficial = plan.benefits_from_input_partitioning(); - let requirements = plan.required_input_distribution(); - let mut repartition_status_flags = vec![]; - for (child, requirement, roundrobin_beneficial) in - izip!(children.into_iter(), requirements, rr_beneficial) - { - // Decide whether adding a round robin is beneficial depending on - // the statistical information we have on the number of rows: - let roundrobin_beneficial_stats = match child.statistics()?.num_rows { - Precision::Exact(n_rows) => n_rows > batch_size, - Precision::Inexact(n_rows) => !should_use_estimates || (n_rows > batch_size), - Precision::Absent => true, - }; - let is_hash = matches!(requirement, Distribution::HashPartitioned(_)); - // Hash re-partitioning is necessary when the input has more than one - // partitions: - let multi_partitions = child.output_partitioning().partition_count() > 1; - let roundrobin_sensible = roundrobin_beneficial && roundrobin_beneficial_stats; - needs_alignment |= is_hash && (multi_partitions || roundrobin_sensible); - repartition_status_flags.push(( - is_hash, - RepartitionRequirementStatus { - requirement, - roundrobin_beneficial, - roundrobin_beneficial_stats, - hash_necessary: is_hash && multi_partitions, - }, - )); - } - // Align hash necessary flags for hash partitions to generate consistent - // hash partitions at each children: - if needs_alignment { - // When there is at least one hash requirement that is necessary or - // beneficial according to statistics, make all children require hash - // repartitioning: - for (is_hash, status) in &mut repartition_status_flags { - if *is_hash { - status.hash_necessary = true; - } - } - } - Ok(repartition_status_flags - .into_iter() - .map(|(_, status)| status) - .collect()) -} - -/// This function checks whether we need to add additional data exchange -/// operators to satisfy distribution requirements. Since this function -/// takes care of such requirements, we should avoid manually adding data -/// exchange operators in other places. -fn ensure_distribution( - dist_context: DistributionContext, - config: &ConfigOptions, -) -> Result> { - let dist_context = update_children(dist_context)?; - - if dist_context.plan.children().is_empty() { - return Ok(Transformed::no(dist_context)); - } - - let target_partitions = config.execution.target_partitions; - // When `false`, round robin repartition will not be added to increase parallelism - let enable_round_robin = config.optimizer.enable_round_robin_repartition; - let repartition_file_scans = config.optimizer.repartition_file_scans; - let batch_size = config.execution.batch_size; - let should_use_estimates = config - .execution - .use_row_number_estimates_to_optimize_partitioning; - let unbounded_and_pipeline_friendly = dist_context.plan.boundedness().is_unbounded() - && matches!( - dist_context.plan.pipeline_behavior(), - EmissionType::Incremental | EmissionType::Both - ); - // Use order preserving variants either of the conditions true - // - it is desired according to config - // - when plan is unbounded - // - when it is pipeline friendly (can incrementally produce results) - let order_preserving_variants_desirable = - unbounded_and_pipeline_friendly || config.optimizer.prefer_existing_sort; - - // Remove unnecessary repartition from the physical plan if any - let DistributionContext { - mut plan, - data, - children, - } = remove_dist_changing_operators(dist_context)?; - - if let Some(exec) = plan.as_any().downcast_ref::() { - if let Some(updated_window) = get_best_fitting_window( - exec.window_expr(), - exec.input(), - &exec.partition_keys, - )? { - plan = updated_window; - } - } else if let Some(exec) = plan.as_any().downcast_ref::() { - if let Some(updated_window) = get_best_fitting_window( - exec.window_expr(), - exec.input(), - &exec.partition_keys, - )? { - plan = updated_window; - } - }; - - let repartition_status_flags = - get_repartition_requirement_status(&plan, batch_size, should_use_estimates)?; - // This loop iterates over all the children to: - // - Increase parallelism for every child if it is beneficial. - // - Satisfy the distribution requirements of every child, if it is not - // already satisfied. - // We store the updated children in `new_children`. - let children = izip!( - children.into_iter(), - plan.required_input_ordering(), - plan.maintains_input_order(), - repartition_status_flags.into_iter() - ) - .map( - |( - mut child, - required_input_ordering, - maintains, - RepartitionRequirementStatus { - requirement, - roundrobin_beneficial, - roundrobin_beneficial_stats, - hash_necessary, - }, - )| { - let add_roundrobin = enable_round_robin - // Operator benefits from partitioning (e.g. filter): - && roundrobin_beneficial - && roundrobin_beneficial_stats - // Unless partitioning increases the partition count, it is not beneficial: - && child.plan.output_partitioning().partition_count() < target_partitions; - - // When `repartition_file_scans` is set, attempt to increase - // parallelism at the source. - if repartition_file_scans && roundrobin_beneficial_stats { - if let Some(new_child) = - child.plan.repartitioned(target_partitions, config)? - { - child.plan = new_child; - } - } - - // Satisfy the distribution requirement if it is unmet. - match &requirement { - Distribution::SinglePartition => { - child = add_spm_on_top(child); - } - Distribution::HashPartitioned(exprs) => { - if add_roundrobin { - // Add round-robin repartitioning on top of the operator - // to increase parallelism. - child = add_roundrobin_on_top(child, target_partitions)?; - } - // When inserting hash is necessary to satisfy hash requirement, insert hash repartition. - if hash_necessary { - child = - add_hash_on_top(child, exprs.to_vec(), target_partitions)?; - } - } - Distribution::UnspecifiedDistribution => { - if add_roundrobin { - // Add round-robin repartitioning on top of the operator - // to increase parallelism. - child = add_roundrobin_on_top(child, target_partitions)?; - } - } - }; - - // There is an ordering requirement of the operator: - if let Some(required_input_ordering) = required_input_ordering { - // Either: - // - Ordering requirement cannot be satisfied by preserving ordering through repartitions, or - // - using order preserving variant is not desirable. - let ordering_satisfied = child - .plan - .equivalence_properties() - .ordering_satisfy_requirement(&required_input_ordering); - if (!ordering_satisfied || !order_preserving_variants_desirable) - && child.data - { - child = replace_order_preserving_variants(child)?; - // If ordering requirements were satisfied before repartitioning, - // make sure ordering requirements are still satisfied after. - if ordering_satisfied { - // Make sure to satisfy ordering requirement: - child = add_sort_above_with_check( - child, - required_input_ordering.clone(), - None, - ); - } - } - // Stop tracking distribution changing operators - child.data = false; - } else { - // no ordering requirement - match requirement { - // Operator requires specific distribution. - Distribution::SinglePartition | Distribution::HashPartitioned(_) => { - // Since there is no ordering requirement, preserving ordering is pointless - child = replace_order_preserving_variants(child)?; - } - Distribution::UnspecifiedDistribution => { - // Since ordering is lost, trying to preserve ordering is pointless - if !maintains || plan.as_any().is::() { - child = replace_order_preserving_variants(child)?; - } - } - } - } - Ok(child) - }, - ) - .collect::>>()?; - - let children_plans = children - .iter() - .map(|c| Arc::clone(&c.plan)) - .collect::>(); - - plan = if plan.as_any().is::() - && !config.optimizer.prefer_existing_union - && can_interleave(children_plans.iter()) - { - // Add a special case for [`UnionExec`] since we want to "bubble up" - // hash-partitioned data. So instead of - // - // Agg: - // Repartition (hash): - // Union: - // - Agg: - // Repartition (hash): - // Data - // - Agg: - // Repartition (hash): - // Data - // - // we can use: - // - // Agg: - // Interleave: - // - Agg: - // Repartition (hash): - // Data - // - Agg: - // Repartition (hash): - // Data - Arc::new(InterleaveExec::try_new(children_plans)?) - } else { - plan.with_new_children(children_plans)? - }; - - Ok(Transformed::yes(DistributionContext::new( - plan, data, children, - ))) -} - -/// Keeps track of distribution changing operators (like `RepartitionExec`, -/// `SortPreservingMergeExec`, `CoalescePartitionsExec`) and their ancestors. -/// Using this information, we can optimize distribution of the plan if/when -/// necessary. -type DistributionContext = PlanContext; - -fn update_children(mut dist_context: DistributionContext) -> Result { - for child_context in dist_context.children.iter_mut() { - let child_plan_any = child_context.plan.as_any(); - child_context.data = - if let Some(repartition) = child_plan_any.downcast_ref::() { - !matches!( - repartition.partitioning(), - Partitioning::UnknownPartitioning(_) - ) - } else { - child_plan_any.is::() - || child_plan_any.is::() - || child_context.plan.children().is_empty() - || child_context.children[0].data - || child_context - .plan - .required_input_distribution() - .iter() - .zip(child_context.children.iter()) - .any(|(required_dist, child_context)| { - child_context.data - && matches!( - required_dist, - Distribution::UnspecifiedDistribution - ) - }) - } - } - - dist_context.data = false; - Ok(dist_context) -} - -#[derive(Debug, Clone)] -struct JoinKeyPairs { - left_keys: Vec>, - right_keys: Vec>, -} - -/// Keeps track of parent required key orderings. -type PlanWithKeyRequirements = PlanContext>>; - -/// Since almost all of these tests explicitly use `ParquetExec` they only run with the parquet feature flag on -#[cfg(feature = "parquet")] -#[cfg(test)] -pub(crate) mod tests { - use std::ops::Deref; - - use super::*; - use crate::datasource::file_format::file_compression_type::FileCompressionType; - use crate::datasource::listing::PartitionedFile; - use crate::datasource::object_store::ObjectStoreUrl; - use crate::datasource::physical_plan::{CsvExec, FileScanConfig, ParquetExec}; - use crate::physical_optimizer::enforce_sorting::EnforceSorting; - use crate::physical_plan::coalesce_batches::CoalesceBatchesExec; - use crate::physical_plan::expressions::col; - use crate::physical_plan::filter::FilterExec; - use crate::physical_plan::joins::utils::JoinOn; - use crate::physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; - use crate::physical_plan::sorts::sort::SortExec; - use crate::physical_plan::{displayable, DisplayAs, DisplayFormatType, Statistics}; - use datafusion_physical_optimizer::output_requirements::OutputRequirements; - use datafusion_physical_optimizer::test_utils::{ - check_integrity, coalesce_partitions_exec, repartition_exec, - }; - - use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; - use datafusion_common::ScalarValue; - use datafusion_expr::Operator; - use datafusion_physical_expr::expressions::{BinaryExpr, Literal}; - use datafusion_physical_expr::{ - expressions::binary, expressions::lit, LexOrdering, PhysicalSortExpr, - }; - use datafusion_physical_expr_common::sort_expr::LexRequirement; - use datafusion_physical_plan::PlanProperties; - - /// Models operators like BoundedWindowExec that require an input - /// ordering but is easy to construct - #[derive(Debug)] - struct SortRequiredExec { - input: Arc, - expr: LexOrdering, - cache: PlanProperties, - } - - impl SortRequiredExec { - fn new_with_requirement( - input: Arc, - requirement: LexOrdering, - ) -> Self { - let cache = Self::compute_properties(&input); - Self { - input, - expr: requirement, - cache, - } - } - - /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. - fn compute_properties(input: &Arc) -> PlanProperties { - PlanProperties::new( - input.equivalence_properties().clone(), // Equivalence Properties - input.output_partitioning().clone(), // Output Partitioning - input.pipeline_behavior(), // Pipeline Behavior - input.boundedness(), // Boundedness - ) - } - } - - impl DisplayAs for SortRequiredExec { - fn fmt_as( - &self, - _t: DisplayFormatType, - f: &mut std::fmt::Formatter, - ) -> std::fmt::Result { - write!(f, "SortRequiredExec: [{}]", self.expr) - } - } - - impl ExecutionPlan for SortRequiredExec { - fn name(&self) -> &'static str { - "SortRequiredExec" - } - - fn as_any(&self) -> &dyn std::any::Any { - self - } - - fn properties(&self) -> &PlanProperties { - &self.cache - } - - fn benefits_from_input_partitioning(&self) -> Vec { - vec![false] - } - - fn children(&self) -> Vec<&Arc> { - vec![&self.input] - } - - // model that it requires the output ordering of its input - fn required_input_ordering(&self) -> Vec> { - if self.expr.is_empty() { - vec![None] - } else { - vec![Some(LexRequirement::from(self.expr.clone()))] - } - } - - fn with_new_children( - self: Arc, - mut children: Vec>, - ) -> Result> { - assert_eq!(children.len(), 1); - let child = children.pop().unwrap(); - Ok(Arc::new(Self::new_with_requirement( - child, - self.expr.clone(), - ))) - } - - fn execute( - &self, - _partition: usize, - _context: Arc, - ) -> Result { - unreachable!(); - } - - fn statistics(&self) -> Result { - self.input.statistics() - } - } - - pub(crate) fn schema() -> SchemaRef { - Arc::new(Schema::new(vec![ - Field::new("a", DataType::Int64, true), - Field::new("b", DataType::Int64, true), - Field::new("c", DataType::Int64, true), - Field::new("d", DataType::Int32, true), - Field::new("e", DataType::Boolean, true), - ])) - } - - fn parquet_exec() -> Arc { - parquet_exec_with_sort(vec![]) - } - - /// create a single parquet file that is sorted - pub(crate) fn parquet_exec_with_sort( - output_ordering: Vec, - ) -> Arc { - ParquetExec::builder( - FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema()) - .with_file(PartitionedFile::new("x".to_string(), 100)) - .with_output_ordering(output_ordering), - ) - .build_arc() - } - - fn parquet_exec_multiple() -> Arc { - parquet_exec_multiple_sorted(vec![]) - } - - /// Created a sorted parquet exec with multiple files - fn parquet_exec_multiple_sorted( - output_ordering: Vec, - ) -> Arc { - ParquetExec::builder( - FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema()) - .with_file_groups(vec![ - vec![PartitionedFile::new("x".to_string(), 100)], - vec![PartitionedFile::new("y".to_string(), 100)], - ]) - .with_output_ordering(output_ordering), - ) - .build_arc() - } - - fn csv_exec() -> Arc { - csv_exec_with_sort(vec![]) - } - - fn csv_exec_with_sort(output_ordering: Vec) -> Arc { - Arc::new( - CsvExec::builder( - FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema()) - .with_file(PartitionedFile::new("x".to_string(), 100)) - .with_output_ordering(output_ordering), - ) - .with_has_header(false) - .with_delimeter(b',') - .with_quote(b'"') - .with_escape(None) - .with_comment(None) - .with_newlines_in_values(false) - .with_file_compression_type(FileCompressionType::UNCOMPRESSED) - .build(), - ) - } - - fn csv_exec_multiple() -> Arc { - csv_exec_multiple_sorted(vec![]) - } - - // Created a sorted parquet exec with multiple files - fn csv_exec_multiple_sorted(output_ordering: Vec) -> Arc { - Arc::new( - CsvExec::builder( - FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema()) - .with_file_groups(vec![ - vec![PartitionedFile::new("x".to_string(), 100)], - vec![PartitionedFile::new("y".to_string(), 100)], - ]) - .with_output_ordering(output_ordering), - ) - .with_has_header(false) - .with_delimeter(b',') - .with_quote(b'"') - .with_escape(None) - .with_comment(None) - .with_newlines_in_values(false) - .with_file_compression_type(FileCompressionType::UNCOMPRESSED) - .build(), - ) - } - - fn projection_exec_with_alias( - input: Arc, - alias_pairs: Vec<(String, String)>, - ) -> Arc { - let mut exprs = vec![]; - for (column, alias) in alias_pairs.iter() { - exprs.push((col(column, &input.schema()).unwrap(), alias.to_string())); - } - Arc::new(ProjectionExec::try_new(exprs, input).unwrap()) - } - - fn aggregate_exec_with_alias( - input: Arc, - alias_pairs: Vec<(String, String)>, - ) -> Arc { - let schema = schema(); - let mut group_by_expr: Vec<(Arc, String)> = vec![]; - for (column, alias) in alias_pairs.iter() { - group_by_expr - .push((col(column, &input.schema()).unwrap(), alias.to_string())); - } - let group_by = PhysicalGroupBy::new_single(group_by_expr.clone()); - - let final_group_by_expr = group_by_expr - .iter() - .enumerate() - .map(|(index, (_col, name))| { - ( - Arc::new(Column::new(name, index)) as Arc, - name.clone(), - ) - }) - .collect::>(); - let final_grouping = PhysicalGroupBy::new_single(final_group_by_expr); - - Arc::new( - AggregateExec::try_new( - AggregateMode::FinalPartitioned, - final_grouping, - vec![], - vec![], - Arc::new( - AggregateExec::try_new( - AggregateMode::Partial, - group_by, - vec![], - vec![], - input, - schema.clone(), - ) - .unwrap(), - ), - schema, - ) - .unwrap(), - ) - } - - fn hash_join_exec( - left: Arc, - right: Arc, - join_on: &JoinOn, - join_type: &JoinType, - ) -> Arc { - Arc::new( - HashJoinExec::try_new( - left, - right, - join_on.clone(), - None, - join_type, - None, - PartitionMode::Partitioned, - false, - ) - .unwrap(), - ) - } - - fn sort_merge_join_exec( - left: Arc, - right: Arc, - join_on: &JoinOn, - join_type: &JoinType, - ) -> Arc { - Arc::new( - SortMergeJoinExec::try_new( - left, - right, - join_on.clone(), - None, - *join_type, - vec![SortOptions::default(); join_on.len()], - false, - ) - .unwrap(), - ) - } - - fn filter_exec(input: Arc) -> Arc { - let predicate = Arc::new(BinaryExpr::new( - col("c", &schema()).unwrap(), - Operator::Eq, - Arc::new(Literal::new(ScalarValue::Int64(Some(0)))), - )); - Arc::new(FilterExec::try_new(predicate, input).unwrap()) - } - - fn sort_exec( - sort_exprs: LexOrdering, - input: Arc, - preserve_partitioning: bool, - ) -> Arc { - let new_sort = SortExec::new(sort_exprs, input) - .with_preserve_partitioning(preserve_partitioning); - Arc::new(new_sort) - } - - fn sort_preserving_merge_exec( - sort_exprs: LexOrdering, - input: Arc, - ) -> Arc { - Arc::new(SortPreservingMergeExec::new(sort_exprs, input)) - } - - fn limit_exec(input: Arc) -> Arc { - Arc::new(GlobalLimitExec::new( - Arc::new(LocalLimitExec::new(input, 100)), - 0, - Some(100), - )) - } - - fn union_exec(input: Vec>) -> Arc { - Arc::new(UnionExec::new(input)) - } - - fn sort_required_exec_with_req( - input: Arc, - sort_exprs: LexOrdering, - ) -> Arc { - Arc::new(SortRequiredExec::new_with_requirement(input, sort_exprs)) - } - - pub(crate) fn trim_plan_display(plan: &str) -> Vec<&str> { - plan.split('\n') - .map(|s| s.trim()) - .filter(|s| !s.is_empty()) - .collect() - } - - fn ensure_distribution_helper( - plan: Arc, - target_partitions: usize, - prefer_existing_sort: bool, - ) -> Result> { - let distribution_context = DistributionContext::new_default(plan); - let mut config = ConfigOptions::new(); - config.execution.target_partitions = target_partitions; - config.optimizer.enable_round_robin_repartition = true; - config.optimizer.repartition_file_scans = false; - config.optimizer.repartition_file_min_size = 1024; - config.optimizer.prefer_existing_sort = prefer_existing_sort; - ensure_distribution(distribution_context, &config).map(|item| item.data.plan) - } - - /// Test whether plan matches with expected plan - macro_rules! plans_matches_expected { - ($EXPECTED_LINES: expr, $PLAN: expr) => { - let physical_plan = $PLAN; - let formatted = displayable(physical_plan.as_ref()).indent(true).to_string(); - let actual: Vec<&str> = formatted.trim().lines().collect(); - - let expected_plan_lines: Vec<&str> = $EXPECTED_LINES - .iter().map(|s| *s).collect(); - - assert_eq!( - expected_plan_lines, actual, - "\n**Original Plan Mismatch\n\nexpected:\n\n{expected_plan_lines:#?}\nactual:\n\n{actual:#?}\n\n" - ); - } - } - - /// Runs the repartition optimizer and asserts the plan against the expected - /// Arguments - /// * `EXPECTED_LINES` - Expected output plan - /// * `PLAN` - Input plan - /// * `FIRST_ENFORCE_DIST` - - /// true: (EnforceDistribution, EnforceDistribution, EnforceSorting) - /// false: else runs (EnforceSorting, EnforceDistribution, EnforceDistribution) - /// * `PREFER_EXISTING_SORT` (optional) - if true, will not repartition / resort data if it is already sorted - /// * `TARGET_PARTITIONS` (optional) - number of partitions to repartition to - /// * `REPARTITION_FILE_SCANS` (optional) - if true, will repartition file scans - /// * `REPARTITION_FILE_MIN_SIZE` (optional) - minimum file size to repartition - /// * `PREFER_EXISTING_UNION` (optional) - if true, will not attempt to convert Union to Interleave - macro_rules! assert_optimized { - ($EXPECTED_LINES: expr, $PLAN: expr, $FIRST_ENFORCE_DIST: expr) => { - assert_optimized!($EXPECTED_LINES, $PLAN, $FIRST_ENFORCE_DIST, false, 10, false, 1024, false); - }; - - ($EXPECTED_LINES: expr, $PLAN: expr, $FIRST_ENFORCE_DIST: expr, $PREFER_EXISTING_SORT: expr) => { - assert_optimized!($EXPECTED_LINES, $PLAN, $FIRST_ENFORCE_DIST, $PREFER_EXISTING_SORT, 10, false, 1024, false); - }; - - ($EXPECTED_LINES: expr, $PLAN: expr, $FIRST_ENFORCE_DIST: expr, $PREFER_EXISTING_SORT: expr, $PREFER_EXISTING_UNION: expr) => { - assert_optimized!($EXPECTED_LINES, $PLAN, $FIRST_ENFORCE_DIST, $PREFER_EXISTING_SORT, 10, false, 1024, $PREFER_EXISTING_UNION); - }; - - ($EXPECTED_LINES: expr, $PLAN: expr, $FIRST_ENFORCE_DIST: expr, $PREFER_EXISTING_SORT: expr, $TARGET_PARTITIONS: expr, $REPARTITION_FILE_SCANS: expr, $REPARTITION_FILE_MIN_SIZE: expr) => { - assert_optimized!($EXPECTED_LINES, $PLAN, $FIRST_ENFORCE_DIST, $PREFER_EXISTING_SORT, $TARGET_PARTITIONS, $REPARTITION_FILE_SCANS, $REPARTITION_FILE_MIN_SIZE, false); - }; - - ($EXPECTED_LINES: expr, $PLAN: expr, $FIRST_ENFORCE_DIST: expr, $PREFER_EXISTING_SORT: expr, $TARGET_PARTITIONS: expr, $REPARTITION_FILE_SCANS: expr, $REPARTITION_FILE_MIN_SIZE: expr, $PREFER_EXISTING_UNION: expr) => { - let expected_lines: Vec<&str> = $EXPECTED_LINES.iter().map(|s| *s).collect(); - - let mut config = ConfigOptions::new(); - config.execution.target_partitions = $TARGET_PARTITIONS; - config.optimizer.repartition_file_scans = $REPARTITION_FILE_SCANS; - config.optimizer.repartition_file_min_size = $REPARTITION_FILE_MIN_SIZE; - config.optimizer.prefer_existing_sort = $PREFER_EXISTING_SORT; - config.optimizer.prefer_existing_union = $PREFER_EXISTING_UNION; - // Use a small batch size, to trigger RoundRobin in tests - config.execution.batch_size = 1; - - // NOTE: These tests verify the joint `EnforceDistribution` + `EnforceSorting` cascade - // because they were written prior to the separation of `BasicEnforcement` into - // `EnforceSorting` and `EnforceDistribution`. - // TODO: Orthogonalize the tests here just to verify `EnforceDistribution` and create - // new tests for the cascade. - - // Add the ancillary output requirements operator at the start: - let optimizer = OutputRequirements::new_add_mode(); - let optimized = optimizer.optimize($PLAN.clone(), &config)?; - - // This file has 2 rules that use tree node, apply these rules to original plan consecutively - // After these operations tree nodes should be in a consistent state. - // This code block makes sure that these rules doesn't violate tree node integrity. - { - let adjusted = if config.optimizer.top_down_join_key_reordering { - // Run adjust_input_keys_ordering rule - let plan_requirements = - PlanWithKeyRequirements::new_default($PLAN.clone()); - let adjusted = plan_requirements - .transform_down(adjust_input_keys_ordering) - .data() - .and_then(check_integrity)?; - // TODO: End state payloads will be checked here. - adjusted.plan - } else { - // Run reorder_join_keys_to_inputs rule - $PLAN.clone().transform_up(|plan| { - Ok(Transformed::yes(reorder_join_keys_to_inputs(plan)?)) - }) - .data()? - }; - - // Then run ensure_distribution rule - DistributionContext::new_default(adjusted) - .transform_up(|distribution_context| { - ensure_distribution(distribution_context, &config) - }) - .data() - .and_then(check_integrity)?; - // TODO: End state payloads will be checked here. - } - - let optimized = if $FIRST_ENFORCE_DIST { - // Run enforce distribution rule first: - let optimizer = EnforceDistribution::new(); - let optimized = optimizer.optimize(optimized, &config)?; - // The rule should be idempotent. - // Re-running this rule shouldn't introduce unnecessary operators. - let optimizer = EnforceDistribution::new(); - let optimized = optimizer.optimize(optimized, &config)?; - // Run the enforce sorting rule: - let optimizer = EnforceSorting::new(); - let optimized = optimizer.optimize(optimized, &config)?; - optimized - } else { - // Run the enforce sorting rule first: - let optimizer = EnforceSorting::new(); - let optimized = optimizer.optimize(optimized, &config)?; - // Run enforce distribution rule: - let optimizer = EnforceDistribution::new(); - let optimized = optimizer.optimize(optimized, &config)?; - // The rule should be idempotent. - // Re-running this rule shouldn't introduce unnecessary operators. - let optimizer = EnforceDistribution::new(); - let optimized = optimizer.optimize(optimized, &config)?; - optimized - }; - - // Remove the ancillary output requirements operator when done: - let optimizer = OutputRequirements::new_remove_mode(); - let optimized = optimizer.optimize(optimized, &config)?; - - // Now format correctly - let plan = displayable(optimized.as_ref()).indent(true).to_string(); - let actual_lines = trim_plan_display(&plan); - - assert_eq!( - &expected_lines, &actual_lines, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected_lines, actual_lines - ); - }; - } - - macro_rules! assert_plan_txt { - ($EXPECTED_LINES: expr, $PLAN: expr) => { - let expected_lines: Vec<&str> = $EXPECTED_LINES.iter().map(|s| *s).collect(); - // Now format correctly - let plan = displayable($PLAN.as_ref()).indent(true).to_string(); - let actual_lines = trim_plan_display(&plan); - - assert_eq!( - &expected_lines, &actual_lines, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected_lines, actual_lines - ); - }; - } - - #[test] - fn multi_hash_joins() -> Result<()> { - let left = parquet_exec(); - let alias_pairs: Vec<(String, String)> = vec![ - ("a".to_string(), "a1".to_string()), - ("b".to_string(), "b1".to_string()), - ("c".to_string(), "c1".to_string()), - ("d".to_string(), "d1".to_string()), - ("e".to_string(), "e1".to_string()), - ]; - let right = projection_exec_with_alias(parquet_exec(), alias_pairs); - let join_types = vec![ - JoinType::Inner, - JoinType::Left, - JoinType::Right, - JoinType::Full, - JoinType::LeftSemi, - JoinType::LeftAnti, - JoinType::LeftMark, - JoinType::RightSemi, - JoinType::RightAnti, - ]; - - // Join on (a == b1) - let join_on = vec![( - Arc::new(Column::new_with_schema("a", &schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("b1", &right.schema()).unwrap()) as _, - )]; - - for join_type in join_types { - let join = hash_join_exec(left.clone(), right.clone(), &join_on, &join_type); - let join_plan = format!( - "HashJoinExec: mode=Partitioned, join_type={join_type}, on=[(a@0, b1@1)]" - ); - - match join_type { - JoinType::Inner - | JoinType::Left - | JoinType::Right - | JoinType::Full - | JoinType::LeftSemi - | JoinType::LeftAnti - | JoinType::LeftMark => { - // Join on (a == c) - let top_join_on = vec![( - Arc::new(Column::new_with_schema("a", &join.schema()).unwrap()) - as _, - Arc::new(Column::new_with_schema("c", &schema()).unwrap()) as _, - )]; - let top_join = hash_join_exec( - join.clone(), - parquet_exec(), - &top_join_on, - &join_type, - ); - let top_join_plan = - format!("HashJoinExec: mode=Partitioned, join_type={join_type}, on=[(a@0, c@2)]"); - - let expected = match join_type { - // Should include 3 RepartitionExecs - JoinType::Inner | JoinType::Left | JoinType::LeftSemi | JoinType::LeftAnti | JoinType::LeftMark => vec![ - 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", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - "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", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ], - // Should include 4 RepartitionExecs - _ => vec![ - top_join_plan.as_str(), - "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", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - "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", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ], - }; - assert_optimized!(expected, top_join.clone(), true); - assert_optimized!(expected, top_join, false); - } - JoinType::RightSemi | JoinType::RightAnti => {} - } - - match join_type { - JoinType::Inner - | JoinType::Left - | JoinType::Right - | JoinType::Full - | JoinType::RightSemi - | JoinType::RightAnti => { - // This time we use (b1 == c) for top join - // Join on (b1 == c) - let top_join_on = vec![( - Arc::new(Column::new_with_schema("b1", &join.schema()).unwrap()) - as _, - Arc::new(Column::new_with_schema("c", &schema()).unwrap()) as _, - )]; - - let top_join = - hash_join_exec(join, parquet_exec(), &top_join_on, &join_type); - let top_join_plan = match join_type { - JoinType::RightSemi | JoinType::RightAnti => - format!("HashJoinExec: mode=Partitioned, join_type={join_type}, on=[(b1@1, c@2)]"), - _ => - format!("HashJoinExec: mode=Partitioned, join_type={join_type}, on=[(b1@6, c@2)]"), - }; - - let expected = match join_type { - // Should include 3 RepartitionExecs - JoinType::Inner | JoinType::Right | JoinType::RightSemi | JoinType::RightAnti => - vec![ - 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", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - "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", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ], - // Should include 4 RepartitionExecs - _ => - vec![ - top_join_plan.as_str(), - "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", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - "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", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ], - }; - assert_optimized!(expected, top_join.clone(), true); - assert_optimized!(expected, top_join, false); - } - JoinType::LeftSemi | JoinType::LeftAnti | JoinType::LeftMark => {} - } - } - - Ok(()) - } - - #[test] - fn multi_joins_after_alias() -> Result<()> { - let left = parquet_exec(); - let right = parquet_exec(); - - // Join on (a == b) - let join_on = vec![( - Arc::new(Column::new_with_schema("a", &schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("b", &schema()).unwrap()) as _, - )]; - let join = hash_join_exec(left, right.clone(), &join_on, &JoinType::Inner); - - // Projection(a as a1, a as a2) - let alias_pairs: Vec<(String, String)> = vec![ - ("a".to_string(), "a1".to_string()), - ("a".to_string(), "a2".to_string()), - ]; - let projection = projection_exec_with_alias(join, alias_pairs); - - // Join on (a1 == c) - let top_join_on = vec![( - Arc::new(Column::new_with_schema("a1", &projection.schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("c", &schema()).unwrap()) as _, - )]; - - let top_join = hash_join_exec( - projection.clone(), - right.clone(), - &top_join_on, - &JoinType::Inner, - ); - - // Output partition need to respect the Alias and should not introduce additional RepartitionExec - let expected = &[ - "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a1@0, c@2)]", - "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", - "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", - "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", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - assert_optimized!(expected, top_join.clone(), true); - assert_optimized!(expected, top_join, false); - - // Join on (a2 == c) - let top_join_on = vec![( - Arc::new(Column::new_with_schema("a2", &projection.schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("c", &schema()).unwrap()) as _, - )]; - - let top_join = hash_join_exec(projection, right, &top_join_on, &JoinType::Inner); - - // Output partition need to respect the Alias and should not introduce additional RepartitionExec - let expected = &[ - "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a2@1, c@2)]", - "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", - "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", - "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", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - assert_optimized!(expected, top_join.clone(), true); - assert_optimized!(expected, top_join, false); - - Ok(()) - } - - #[test] - fn multi_joins_after_multi_alias() -> Result<()> { - let left = parquet_exec(); - let right = parquet_exec(); - - // Join on (a == b) - let join_on = vec![( - Arc::new(Column::new_with_schema("a", &schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("b", &schema()).unwrap()) as _, - )]; - - let join = hash_join_exec(left, right.clone(), &join_on, &JoinType::Inner); - - // Projection(c as c1) - let alias_pairs: Vec<(String, String)> = - vec![("c".to_string(), "c1".to_string())]; - let projection = projection_exec_with_alias(join, alias_pairs); - - // Projection(c1 as a) - let alias_pairs: Vec<(String, String)> = - vec![("c1".to_string(), "a".to_string())]; - let projection2 = projection_exec_with_alias(projection, alias_pairs); - - // Join on (a == c) - let top_join_on = vec![( - Arc::new(Column::new_with_schema("a", &projection2.schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("c", &schema()).unwrap()) as _, - )]; - - let top_join = hash_join_exec(projection2, right, &top_join_on, &JoinType::Inner); - - // The Column 'a' has different meaning now after the two Projections - // The original Output partition can not satisfy the Join requirements and need to add an additional RepartitionExec - let expected = &[ - "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, c@2)]", - "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - "ProjectionExec: expr=[c1@0 as a]", - "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", - "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", - "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", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - - assert_optimized!(expected, top_join.clone(), true); - assert_optimized!(expected, top_join, false); - - Ok(()) - } - - #[test] - fn join_after_agg_alias() -> Result<()> { - // group by (a as a1) - let left = aggregate_exec_with_alias( - parquet_exec(), - vec![("a".to_string(), "a1".to_string())], - ); - // group by (a as a2) - let right = aggregate_exec_with_alias( - parquet_exec(), - vec![("a".to_string(), "a2".to_string())], - ); - - // Join on (a1 == a2) - let join_on = vec![( - Arc::new(Column::new_with_schema("a1", &left.schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("a2", &right.schema()).unwrap()) as _, - )]; - let join = hash_join_exec(left, right.clone(), &join_on, &JoinType::Inner); - - // Only two RepartitionExecs added - let expected = &[ - "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a1@0, a2@0)]", - "AggregateExec: mode=FinalPartitioned, gby=[a1@0 as a1], aggr=[]", - "RepartitionExec: partitioning=Hash([a1@0], 10), input_partitions=10", - "AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[]", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "AggregateExec: mode=FinalPartitioned, gby=[a2@0 as a2], aggr=[]", - "RepartitionExec: partitioning=Hash([a2@0], 10), input_partitions=10", - "AggregateExec: mode=Partial, gby=[a@0 as a2], aggr=[]", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - assert_optimized!(expected, join.clone(), true); - assert_optimized!(expected, join, false); - - Ok(()) - } - - #[test] - fn hash_join_key_ordering() -> Result<()> { - // group by (a as a1, b as b1) - let left = aggregate_exec_with_alias( - parquet_exec(), - vec![ - ("a".to_string(), "a1".to_string()), - ("b".to_string(), "b1".to_string()), - ], - ); - // group by (b, a) - let right = aggregate_exec_with_alias( - parquet_exec(), - vec![ - ("b".to_string(), "b".to_string()), - ("a".to_string(), "a".to_string()), - ], - ); - - // Join on (b1 == b && a1 == a) - let join_on = vec![ - ( - Arc::new(Column::new_with_schema("b1", &left.schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("b", &right.schema()).unwrap()) as _, - ), - ( - Arc::new(Column::new_with_schema("a1", &left.schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("a", &right.schema()).unwrap()) as _, - ), - ]; - let join = hash_join_exec(left, right.clone(), &join_on, &JoinType::Inner); - - // Only two RepartitionExecs added - let expected = &[ - "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(b1@1, b@0), (a1@0, a@1)]", - "ProjectionExec: expr=[a1@1 as a1, b1@0 as b1]", - "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", - "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", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - assert_optimized!(expected, join.clone(), true); - assert_optimized!(expected, join, false); - - Ok(()) - } - - #[test] - fn multi_hash_join_key_ordering() -> Result<()> { - let left = parquet_exec(); - let alias_pairs: Vec<(String, String)> = vec![ - ("a".to_string(), "a1".to_string()), - ("b".to_string(), "b1".to_string()), - ("c".to_string(), "c1".to_string()), - ]; - let right = projection_exec_with_alias(parquet_exec(), alias_pairs); - - // Join on (a == a1 and b == b1 and c == c1) - let join_on = vec![ - ( - Arc::new(Column::new_with_schema("a", &schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("a1", &right.schema()).unwrap()) as _, - ), - ( - Arc::new(Column::new_with_schema("b", &schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("b1", &right.schema()).unwrap()) as _, - ), - ( - Arc::new(Column::new_with_schema("c", &schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("c1", &right.schema()).unwrap()) as _, - ), - ]; - let bottom_left_join = - hash_join_exec(left.clone(), right.clone(), &join_on, &JoinType::Inner); - - // Projection(a as A, a as AA, b as B, c as C) - let alias_pairs: Vec<(String, String)> = vec![ - ("a".to_string(), "A".to_string()), - ("a".to_string(), "AA".to_string()), - ("b".to_string(), "B".to_string()), - ("c".to_string(), "C".to_string()), - ]; - let bottom_left_projection = - projection_exec_with_alias(bottom_left_join, alias_pairs); - - // Join on (c == c1 and b == b1 and a == a1) - let join_on = vec![ - ( - Arc::new(Column::new_with_schema("c", &schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("c1", &right.schema()).unwrap()) as _, - ), - ( - Arc::new(Column::new_with_schema("b", &schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("b1", &right.schema()).unwrap()) as _, - ), - ( - Arc::new(Column::new_with_schema("a", &schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("a1", &right.schema()).unwrap()) as _, - ), - ]; - let bottom_right_join = - hash_join_exec(left, right.clone(), &join_on, &JoinType::Inner); - - // Join on (B == b1 and C == c and AA = a1) - let top_join_on = vec![ - ( - Arc::new( - Column::new_with_schema("B", &bottom_left_projection.schema()) - .unwrap(), - ) as _, - Arc::new( - Column::new_with_schema("b1", &bottom_right_join.schema()).unwrap(), - ) as _, - ), - ( - Arc::new( - Column::new_with_schema("C", &bottom_left_projection.schema()) - .unwrap(), - ) as _, - Arc::new( - Column::new_with_schema("c", &bottom_right_join.schema()).unwrap(), - ) as _, - ), - ( - Arc::new( - Column::new_with_schema("AA", &bottom_left_projection.schema()) - .unwrap(), - ) as _, - Arc::new( - Column::new_with_schema("a1", &bottom_right_join.schema()).unwrap(), - ) as _, - ), - ]; - - let top_join = hash_join_exec( - bottom_left_projection.clone(), - bottom_right_join, - &top_join_on, - &JoinType::Inner, - ); - - let predicate: Arc = binary( - col("c", top_join.schema().deref())?, - Operator::Gt, - lit(1i64), - top_join.schema().deref(), - )?; - - let filter_top_join: Arc = - Arc::new(FilterExec::try_new(predicate, top_join)?); - - // The bottom joins' join key ordering is adjusted based on the top join. And the top join should not introduce additional RepartitionExec - let expected = &[ - "FilterExec: c@6 > 1", - "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(B@2, b1@6), (C@3, c@2), (AA@1, a1@5)]", - "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", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "RepartitionExec: partitioning=Hash([b1@1, c1@2, a1@0], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1]", - "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", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "RepartitionExec: partitioning=Hash([b1@1, c1@2, a1@0], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - assert_optimized!(expected, filter_top_join.clone(), true); - assert_optimized!(expected, filter_top_join, false); - - Ok(()) - } - - #[test] - fn reorder_join_keys_to_left_input() -> Result<()> { - let left = parquet_exec(); - let alias_pairs: Vec<(String, String)> = vec![ - ("a".to_string(), "a1".to_string()), - ("b".to_string(), "b1".to_string()), - ("c".to_string(), "c1".to_string()), - ]; - let right = projection_exec_with_alias(parquet_exec(), alias_pairs); - - // Join on (a == a1 and b == b1 and c == c1) - let join_on = vec![ - ( - Arc::new(Column::new_with_schema("a", &schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("a1", &right.schema()).unwrap()) as _, - ), - ( - Arc::new(Column::new_with_schema("b", &schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("b1", &right.schema()).unwrap()) as _, - ), - ( - Arc::new(Column::new_with_schema("c", &schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("c1", &right.schema()).unwrap()) as _, - ), - ]; - - let bottom_left_join = ensure_distribution_helper( - hash_join_exec(left.clone(), right.clone(), &join_on, &JoinType::Inner), - 10, - true, - )?; - - // Projection(a as A, a as AA, b as B, c as C) - let alias_pairs: Vec<(String, String)> = vec![ - ("a".to_string(), "A".to_string()), - ("a".to_string(), "AA".to_string()), - ("b".to_string(), "B".to_string()), - ("c".to_string(), "C".to_string()), - ]; - let bottom_left_projection = - projection_exec_with_alias(bottom_left_join, alias_pairs); - - // Join on (c == c1 and b == b1 and a == a1) - let join_on = vec![ - ( - Arc::new(Column::new_with_schema("c", &schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("c1", &right.schema()).unwrap()) as _, - ), - ( - Arc::new(Column::new_with_schema("b", &schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("b1", &right.schema()).unwrap()) as _, - ), - ( - Arc::new(Column::new_with_schema("a", &schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("a1", &right.schema()).unwrap()) as _, - ), - ]; - let bottom_right_join = ensure_distribution_helper( - hash_join_exec(left, right.clone(), &join_on, &JoinType::Inner), - 10, - true, - )?; - - // Join on (B == b1 and C == c and AA = a1) - let top_join_on = vec![ - ( - Arc::new( - Column::new_with_schema("B", &bottom_left_projection.schema()) - .unwrap(), - ) as _, - Arc::new( - Column::new_with_schema("b1", &bottom_right_join.schema()).unwrap(), - ) as _, - ), - ( - Arc::new( - Column::new_with_schema("C", &bottom_left_projection.schema()) - .unwrap(), - ) as _, - Arc::new( - Column::new_with_schema("c", &bottom_right_join.schema()).unwrap(), - ) as _, - ), - ( - Arc::new( - Column::new_with_schema("AA", &bottom_left_projection.schema()) - .unwrap(), - ) as _, - Arc::new( - Column::new_with_schema("a1", &bottom_right_join.schema()).unwrap(), - ) as _, - ), - ]; - - let join_types = vec![ - JoinType::Inner, - JoinType::Left, - JoinType::Right, - JoinType::Full, - JoinType::LeftSemi, - JoinType::LeftAnti, - JoinType::RightSemi, - JoinType::RightAnti, - ]; - - for join_type in join_types { - let top_join = hash_join_exec( - bottom_left_projection.clone(), - bottom_right_join.clone(), - &top_join_on, - &join_type, - ); - let top_join_plan = - format!("HashJoinExec: mode=Partitioned, join_type={:?}, on=[(AA@1, a1@5), (B@2, b1@6), (C@3, c@2)]", &join_type); - - let reordered = reorder_join_keys_to_inputs(top_join)?; - - // The top joins' join key ordering is adjusted based on the children inputs. - let expected = &[ - top_join_plan.as_str(), - "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", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "RepartitionExec: partitioning=Hash([a1@0, b1@1, c1@2], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1]", - "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", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "RepartitionExec: partitioning=Hash([c1@2, b1@1, a1@0], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - - assert_plan_txt!(expected, reordered); - } - - Ok(()) - } - - #[test] - fn reorder_join_keys_to_right_input() -> Result<()> { - let left = parquet_exec(); - let alias_pairs: Vec<(String, String)> = vec![ - ("a".to_string(), "a1".to_string()), - ("b".to_string(), "b1".to_string()), - ("c".to_string(), "c1".to_string()), - ]; - let right = projection_exec_with_alias(parquet_exec(), alias_pairs); - - // Join on (a == a1 and b == b1) - let join_on = vec![ - ( - Arc::new(Column::new_with_schema("a", &schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("a1", &right.schema()).unwrap()) as _, - ), - ( - Arc::new(Column::new_with_schema("b", &schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("b1", &right.schema()).unwrap()) as _, - ), - ]; - let bottom_left_join = ensure_distribution_helper( - hash_join_exec(left.clone(), right.clone(), &join_on, &JoinType::Inner), - 10, - true, - )?; - - // Projection(a as A, a as AA, b as B, c as C) - let alias_pairs: Vec<(String, String)> = vec![ - ("a".to_string(), "A".to_string()), - ("a".to_string(), "AA".to_string()), - ("b".to_string(), "B".to_string()), - ("c".to_string(), "C".to_string()), - ]; - let bottom_left_projection = - projection_exec_with_alias(bottom_left_join, alias_pairs); - - // Join on (c == c1 and b == b1 and a == a1) - let join_on = vec![ - ( - Arc::new(Column::new_with_schema("c", &schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("c1", &right.schema()).unwrap()) as _, - ), - ( - Arc::new(Column::new_with_schema("b", &schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("b1", &right.schema()).unwrap()) as _, - ), - ( - Arc::new(Column::new_with_schema("a", &schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("a1", &right.schema()).unwrap()) as _, - ), - ]; - let bottom_right_join = ensure_distribution_helper( - hash_join_exec(left, right.clone(), &join_on, &JoinType::Inner), - 10, - true, - )?; - - // Join on (B == b1 and C == c and AA = a1) - let top_join_on = vec![ - ( - Arc::new( - Column::new_with_schema("B", &bottom_left_projection.schema()) - .unwrap(), - ) as _, - Arc::new( - Column::new_with_schema("b1", &bottom_right_join.schema()).unwrap(), - ) as _, - ), - ( - Arc::new( - Column::new_with_schema("C", &bottom_left_projection.schema()) - .unwrap(), - ) as _, - Arc::new( - Column::new_with_schema("c", &bottom_right_join.schema()).unwrap(), - ) as _, - ), - ( - Arc::new( - Column::new_with_schema("AA", &bottom_left_projection.schema()) - .unwrap(), - ) as _, - Arc::new( - Column::new_with_schema("a1", &bottom_right_join.schema()).unwrap(), - ) as _, - ), - ]; - - let join_types = vec![ - JoinType::Inner, - JoinType::Left, - JoinType::Right, - JoinType::Full, - JoinType::LeftSemi, - JoinType::LeftAnti, - JoinType::RightSemi, - JoinType::RightAnti, - ]; - - for join_type in join_types { - let top_join = hash_join_exec( - bottom_left_projection.clone(), - bottom_right_join.clone(), - &top_join_on, - &join_type, - ); - let top_join_plan = - format!("HashJoinExec: mode=Partitioned, join_type={:?}, on=[(C@3, c@2), (B@2, b1@6), (AA@1, a1@5)]", &join_type); - - let reordered = reorder_join_keys_to_inputs(top_join)?; - - // The top joins' join key ordering is adjusted based on the children inputs. - let expected = &[ - top_join_plan.as_str(), - "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", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "RepartitionExec: partitioning=Hash([a1@0, b1@1], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1]", - "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", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "RepartitionExec: partitioning=Hash([c1@2, b1@1, a1@0], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - - assert_plan_txt!(expected, reordered); - } - - Ok(()) - } - - #[test] - fn multi_smj_joins() -> Result<()> { - let left = parquet_exec(); - let alias_pairs: Vec<(String, String)> = vec![ - ("a".to_string(), "a1".to_string()), - ("b".to_string(), "b1".to_string()), - ("c".to_string(), "c1".to_string()), - ("d".to_string(), "d1".to_string()), - ("e".to_string(), "e1".to_string()), - ]; - let right = projection_exec_with_alias(parquet_exec(), alias_pairs); - - // SortMergeJoin does not support RightSemi and RightAnti join now - let join_types = vec![ - JoinType::Inner, - JoinType::Left, - JoinType::Right, - JoinType::Full, - JoinType::LeftSemi, - JoinType::LeftAnti, - ]; - - // Join on (a == b1) - let join_on = vec![( - Arc::new(Column::new_with_schema("a", &schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("b1", &right.schema()).unwrap()) as _, - )]; - - for join_type in join_types { - let join = - sort_merge_join_exec(left.clone(), right.clone(), &join_on, &join_type); - let join_plan = - format!("SortMergeJoin: join_type={join_type}, on=[(a@0, b1@1)]"); - - // Top join on (a == c) - let top_join_on = vec![( - Arc::new(Column::new_with_schema("a", &join.schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("c", &schema()).unwrap()) as _, - )]; - let top_join = sort_merge_join_exec( - join.clone(), - parquet_exec(), - &top_join_on, - &join_type, - ); - let top_join_plan = - format!("SortMergeJoin: join_type={join_type}, on=[(a@0, c@2)]"); - - let expected = match join_type { - // Should include 6 RepartitionExecs (3 hash, 3 round-robin), 3 SortExecs - JoinType::Inner | JoinType::Left | JoinType::LeftSemi | JoinType::LeftAnti => - vec![ - top_join_plan.as_str(), - 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", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "SortExec: expr=[b1@1 ASC], preserve_partitioning=[true]", - "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - "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", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ], - // Should include 7 RepartitionExecs (4 hash, 3 round-robin), 4 SortExecs - // Since ordering of the left child is not preserved after SortMergeJoin - // when mode is Right, RightSemi, RightAnti, Full - // - We need to add one additional SortExec after SortMergeJoin in contrast the test cases - // when mode is Inner, Left, LeftSemi, LeftAnti - // Similarly, since partitioning of the left side is not preserved - // when mode is Right, RightSemi, RightAnti, Full - // - We need to add one additional Hash Repartition after SortMergeJoin in contrast the test - // cases when mode is Inner, Left, LeftSemi, LeftAnti - _ => vec![ - top_join_plan.as_str(), - // Below 2 operators are differences introduced, when join mode is changed - "SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", - "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - 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", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "SortExec: expr=[b1@1 ASC], preserve_partitioning=[true]", - "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - "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", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ], - }; - assert_optimized!(expected, top_join.clone(), true, true); - - let expected_first_sort_enforcement = match join_type { - // Should include 6 RepartitionExecs (3 hash, 3 round-robin), 3 SortExecs - JoinType::Inner | JoinType::Left | JoinType::LeftSemi | JoinType::LeftAnti => - vec![ - 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", - "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", - "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", - "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ], - // Should include 8 RepartitionExecs (4 hash, 8 round-robin), 4 SortExecs - // Since ordering of the left child is not preserved after SortMergeJoin - // when mode is Right, RightSemi, RightAnti, Full - // - We need to add one additional SortExec after SortMergeJoin in contrast the test cases - // when mode is Inner, Left, LeftSemi, LeftAnti - // Similarly, since partitioning of the left side is not preserved - // when mode is Right, RightSemi, RightAnti, Full - // - We need to add one additional Hash Repartition and Roundrobin repartition after - // SortMergeJoin in contrast the test cases when mode is Inner, Left, LeftSemi, LeftAnti - _ => vec![ - 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", - "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", - "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", - "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", - "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ], - }; - assert_optimized!(expected_first_sort_enforcement, top_join, false, true); - - match join_type { - JoinType::Inner | JoinType::Left | JoinType::Right | JoinType::Full => { - // This time we use (b1 == c) for top join - // Join on (b1 == c) - let top_join_on = vec![( - Arc::new(Column::new_with_schema("b1", &join.schema()).unwrap()) - as _, - Arc::new(Column::new_with_schema("c", &schema()).unwrap()) as _, - )]; - let top_join = sort_merge_join_exec( - join, - parquet_exec(), - &top_join_on, - &join_type, - ); - let top_join_plan = - format!("SortMergeJoin: join_type={join_type}, on=[(b1@6, c@2)]"); - - let expected = match join_type { - // Should include 6 RepartitionExecs(3 hash, 3 round-robin) and 3 SortExecs - JoinType::Inner | JoinType::Right => vec![ - top_join_plan.as_str(), - 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", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "SortExec: expr=[b1@1 ASC], preserve_partitioning=[true]", - "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - "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", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ], - // Should include 7 RepartitionExecs (4 hash, 3 round-robin) and 4 SortExecs - JoinType::Left | JoinType::Full => vec![ - top_join_plan.as_str(), - "SortExec: expr=[b1@6 ASC], preserve_partitioning=[true]", - "RepartitionExec: partitioning=Hash([b1@6], 10), input_partitions=10", - 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", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "SortExec: expr=[b1@1 ASC], preserve_partitioning=[true]", - "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - "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", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ], - // this match arm cannot be reached - _ => unreachable!() - }; - assert_optimized!(expected, top_join.clone(), true, true); - - let expected_first_sort_enforcement = match join_type { - // Should include 6 RepartitionExecs (3 of them preserves order) and 3 SortExecs - JoinType::Inner | JoinType::Right => vec![ - 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", - "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", - "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", - "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ], - // Should include 8 RepartitionExecs (4 of them preserves order) and 4 SortExecs - JoinType::Left | JoinType::Full => vec![ - 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", - "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", - "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", - "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", - "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ], - // this match arm cannot be reached - _ => unreachable!() - }; - assert_optimized!( - expected_first_sort_enforcement, - top_join, - false, - true - ); - } - _ => {} - } - } - - Ok(()) - } - - #[test] - fn smj_join_key_ordering() -> Result<()> { - // group by (a as a1, b as b1) - let left = aggregate_exec_with_alias( - parquet_exec(), - vec![ - ("a".to_string(), "a1".to_string()), - ("b".to_string(), "b1".to_string()), - ], - ); - //Projection(a1 as a3, b1 as b3) - let alias_pairs: Vec<(String, String)> = vec![ - ("a1".to_string(), "a3".to_string()), - ("b1".to_string(), "b3".to_string()), - ]; - let left = projection_exec_with_alias(left, alias_pairs); - - // group by (b, a) - let right = aggregate_exec_with_alias( - parquet_exec(), - vec![ - ("b".to_string(), "b".to_string()), - ("a".to_string(), "a".to_string()), - ], - ); - - //Projection(a as a2, b as b2) - let alias_pairs: Vec<(String, String)> = vec![ - ("a".to_string(), "a2".to_string()), - ("b".to_string(), "b2".to_string()), - ]; - let right = projection_exec_with_alias(right, alias_pairs); - - // Join on (b3 == b2 && a3 == a2) - let join_on = vec![ - ( - Arc::new(Column::new_with_schema("b3", &left.schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("b2", &right.schema()).unwrap()) as _, - ), - ( - Arc::new(Column::new_with_schema("a3", &left.schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("a2", &right.schema()).unwrap()) as _, - ), - ]; - let join = sort_merge_join_exec(left, right.clone(), &join_on, &JoinType::Inner); - - // Only two RepartitionExecs added - let expected = &[ - "SortMergeJoin: join_type=Inner, on=[(b3@1, b2@1), (a3@0, a2@0)]", - "SortExec: expr=[b3@1 ASC, a3@0 ASC], preserve_partitioning=[true]", - "ProjectionExec: expr=[a1@0 as a3, b1@1 as b3]", - "ProjectionExec: expr=[a1@1 as a1, b1@0 as b1]", - "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", - "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", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - assert_optimized!(expected, join.clone(), true, true); - - 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", - "SortExec: expr=[b3@1 ASC, a3@0 ASC], preserve_partitioning=[false]", - "CoalescePartitionsExec", - "ProjectionExec: expr=[a1@0 as a3, b1@1 as b3]", - "ProjectionExec: expr=[a1@1 as a1, b1@0 as b1]", - "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", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "RepartitionExec: partitioning=Hash([b2@1, a2@0], 10), input_partitions=10, preserve_order=true, sort_exprs=b2@1 ASC, a2@0 ASC", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "SortExec: expr=[b2@1 ASC, a2@0 ASC], preserve_partitioning=[false]", - "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", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - assert_optimized!(expected_first_sort_enforcement, join, false, true); - - Ok(()) - } - - #[test] - fn merge_does_not_need_sort() -> Result<()> { - // see https://github.com/apache/datafusion/issues/4331 - let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("a", &schema).unwrap(), - options: SortOptions::default(), - }]); - - // Scan some sorted parquet files - let exec = parquet_exec_multiple_sorted(vec![sort_key.clone()]); - - // CoalesceBatchesExec to mimic behavior after a filter - let exec = Arc::new(CoalesceBatchesExec::new(exec, 4096)); - - // Merge from multiple parquet files and keep the data sorted - let exec: Arc = - Arc::new(SortPreservingMergeExec::new(sort_key, exec)); - - // The optimizer should not add an additional SortExec as the - // data is already sorted - let expected = &[ - "SortPreservingMergeExec: [a@0 ASC]", - "CoalesceBatchesExec: target_batch_size=4096", - "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", - ]; - assert_optimized!(expected, exec, true); - - // In this case preserving ordering through order preserving operators is not desirable - // (according to flag: PREFER_EXISTING_SORT) - // hence in this case ordering lost during CoalescePartitionsExec and re-introduced with - // SortExec at the top. - let expected = &[ - "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - "CoalescePartitionsExec", - "CoalesceBatchesExec: target_batch_size=4096", - "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", - ]; - assert_optimized!(expected, exec, false); - - Ok(()) - } - - #[test] - fn union_to_interleave() -> Result<()> { - // group by (a as a1) - let left = aggregate_exec_with_alias( - parquet_exec(), - vec![("a".to_string(), "a1".to_string())], - ); - // group by (a as a2) - let right = aggregate_exec_with_alias( - parquet_exec(), - vec![("a".to_string(), "a1".to_string())], - ); - - // Union - let plan = Arc::new(UnionExec::new(vec![left, right])); - - // final agg - let plan = - aggregate_exec_with_alias(plan, vec![("a1".to_string(), "a2".to_string())]); - - // Only two RepartitionExecs added, no final RepartitionExec required - let expected = &[ - "AggregateExec: mode=FinalPartitioned, gby=[a2@0 as a2], aggr=[]", - "AggregateExec: mode=Partial, gby=[a1@0 as a2], aggr=[]", - "InterleaveExec", - "AggregateExec: mode=FinalPartitioned, gby=[a1@0 as a1], aggr=[]", - "RepartitionExec: partitioning=Hash([a1@0], 10), input_partitions=10", - "AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[]", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "AggregateExec: mode=FinalPartitioned, gby=[a1@0 as a1], aggr=[]", - "RepartitionExec: partitioning=Hash([a1@0], 10), input_partitions=10", - "AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[]", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - assert_optimized!(expected, plan.clone(), true); - assert_optimized!(expected, plan.clone(), false); - - Ok(()) - } - - #[test] - fn union_not_to_interleave() -> Result<()> { - // group by (a as a1) - let left = aggregate_exec_with_alias( - parquet_exec(), - vec![("a".to_string(), "a1".to_string())], - ); - // group by (a as a2) - let right = aggregate_exec_with_alias( - parquet_exec(), - vec![("a".to_string(), "a1".to_string())], - ); - - // Union - let plan = Arc::new(UnionExec::new(vec![left, right])); - - // final agg - let plan = - aggregate_exec_with_alias(plan, vec![("a1".to_string(), "a2".to_string())]); - - // Only two RepartitionExecs added, no final RepartitionExec required - let expected = &[ - "AggregateExec: mode=FinalPartitioned, gby=[a2@0 as a2], aggr=[]", - "RepartitionExec: partitioning=Hash([a2@0], 10), input_partitions=20", - "AggregateExec: mode=Partial, gby=[a1@0 as a2], aggr=[]", - "UnionExec", - "AggregateExec: mode=FinalPartitioned, gby=[a1@0 as a1], aggr=[]", - "RepartitionExec: partitioning=Hash([a1@0], 10), input_partitions=10", - "AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[]", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "AggregateExec: mode=FinalPartitioned, gby=[a1@0 as a1], aggr=[]", - "RepartitionExec: partitioning=Hash([a1@0], 10), input_partitions=10", - "AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[]", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - // no sort in the plan but since we need it as a parameter, make it default false - let prefer_existing_sort = false; - let first_enforce_distribution = true; - let prefer_existing_union = true; - - assert_optimized!( - expected, - plan.clone(), - first_enforce_distribution, - prefer_existing_sort, - prefer_existing_union - ); - assert_optimized!( - expected, - plan, - !first_enforce_distribution, - prefer_existing_sort, - prefer_existing_union - ); - - Ok(()) - } - - #[test] - fn added_repartition_to_single_partition() -> Result<()> { - let alias = vec![("a".to_string(), "a".to_string())]; - let plan = aggregate_exec_with_alias(parquet_exec(), alias); - - let expected = [ - "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", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - assert_optimized!(expected, plan.clone(), true); - assert_optimized!(expected, plan, false); - - Ok(()) - } - - #[test] - fn repartition_deepest_node() -> Result<()> { - let alias = vec![("a".to_string(), "a".to_string())]; - let plan = aggregate_exec_with_alias(filter_exec(parquet_exec()), alias); - - let expected = &[ - "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=[]", - "FilterExec: c@2 = 0", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - assert_optimized!(expected, plan.clone(), true); - assert_optimized!(expected, plan, false); - - Ok(()) - } - - #[test] - - fn repartition_unsorted_limit() -> Result<()> { - let plan = limit_exec(filter_exec(parquet_exec())); - - let expected = &[ - "GlobalLimitExec: skip=0, fetch=100", - "CoalescePartitionsExec", - "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", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - - assert_optimized!(expected, plan.clone(), true); - assert_optimized!(expected, plan, false); - - Ok(()) - } - - #[test] - fn repartition_sorted_limit() -> Result<()> { - let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("c", &schema).unwrap(), - options: SortOptions::default(), - }]); - let plan = limit_exec(sort_exec(sort_key, parquet_exec(), false)); - - let expected = &[ - "GlobalLimitExec: skip=0, fetch=100", - "LocalLimitExec: fetch=100", - // data is sorted so can't repartition here - "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - assert_optimized!(expected, plan.clone(), true); - assert_optimized!(expected, plan, false); - - Ok(()) - } - - #[test] - fn repartition_sorted_limit_with_filter() -> Result<()> { - let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("c", &schema).unwrap(), - options: SortOptions::default(), - }]); - let plan = sort_required_exec_with_req( - filter_exec(sort_exec(sort_key.clone(), parquet_exec(), false)), - sort_key, - ); - - let expected = &[ - "SortRequiredExec: [c@2 ASC]", - "FilterExec: c@2 = 0", - // We can use repartition here, ordering requirement by SortRequiredExec - // is still satisfied. - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - - assert_optimized!(expected, plan.clone(), true); - assert_optimized!(expected, plan, false); - - Ok(()) - } - - #[test] - fn repartition_ignores_limit() -> Result<()> { - let alias = vec![("a".to_string(), "a".to_string())]; - let plan = aggregate_exec_with_alias( - limit_exec(filter_exec(limit_exec(parquet_exec()))), - alias, - ); - - let expected = &[ - "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", - "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", - "GlobalLimitExec: skip=0, fetch=100", - "LocalLimitExec: fetch=100", - // Expect no repartition to happen for local limit - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - assert_optimized!(expected, plan.clone(), true); - assert_optimized!(expected, plan, false); - - Ok(()) - } - - #[test] - fn repartition_ignores_union() -> Result<()> { - let plan = union_exec(vec![parquet_exec(); 5]); - - let expected = &[ - "UnionExec", - // Expect no repartition of ParquetExec - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - - assert_optimized!(expected, plan.clone(), true); - assert_optimized!(expected, plan, false); - - Ok(()) - } - - #[test] - fn repartition_through_sort_preserving_merge() -> Result<()> { - // sort preserving merge with non-sorted input - let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("c", &schema).unwrap(), - options: SortOptions::default(), - }]); - let plan = sort_preserving_merge_exec(sort_key, parquet_exec()); - - // need resort as the data was not sorted correctly - let expected = &[ - "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - assert_optimized!(expected, plan.clone(), true); - assert_optimized!(expected, plan, false); - - Ok(()) - } - - #[test] - fn repartition_ignores_sort_preserving_merge() -> Result<()> { - // sort preserving merge already sorted input, - let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("c", &schema).unwrap(), - options: SortOptions::default(), - }]); - let plan = sort_preserving_merge_exec( - sort_key.clone(), - parquet_exec_multiple_sorted(vec![sort_key]), - ); - - // should not sort (as the data was already sorted) - // should not repartition, since increased parallelism is not beneficial for SortPReservingMerge - let expected = &[ - "SortPreservingMergeExec: [c@2 ASC]", - "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", - ]; - - assert_optimized!(expected, plan.clone(), true); - - let expected = &[ - "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - "CoalescePartitionsExec", - "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", - ]; - assert_optimized!(expected, plan, false); - - Ok(()) - } - - #[test] - fn repartition_ignores_sort_preserving_merge_with_union() -> Result<()> { - // 2 sorted parquet files unioned (partitions are concatenated, sort is preserved) - let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("c", &schema).unwrap(), - options: SortOptions::default(), - }]); - let input = union_exec(vec![parquet_exec_with_sort(vec![sort_key.clone()]); 2]); - let plan = sort_preserving_merge_exec(sort_key, input); - - // should not repartition / sort (as the data was already sorted) - let expected = &[ - "SortPreservingMergeExec: [c@2 ASC]", - "UnionExec", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", - ]; - - assert_optimized!(expected, plan.clone(), true); - - let expected = &[ - "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - "CoalescePartitionsExec", - "UnionExec", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", - ]; - assert_optimized!(expected, plan, false); - - Ok(()) - } - - #[test] - fn repartition_does_not_destroy_sort() -> Result<()> { - // SortRequired - // Parquet(sorted) - let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("d", &schema).unwrap(), - options: SortOptions::default(), - }]); - let plan = sort_required_exec_with_req( - filter_exec(parquet_exec_with_sort(vec![sort_key.clone()])), - sort_key, - ); - - // during repartitioning ordering is preserved - let expected = &[ - "SortRequiredExec: [d@3 ASC]", - "FilterExec: c@2 = 0", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[d@3 ASC]", - ]; - - assert_optimized!(expected, plan.clone(), true, true); - assert_optimized!(expected, plan, false, true); - - Ok(()) - } - - #[test] - fn repartition_does_not_destroy_sort_more_complex() -> Result<()> { - // model a more complicated scenario where one child of a union can be repartitioned for performance - // but the other can not be - // - // Union - // SortRequired - // Parquet(sorted) - // Filter - // Parquet(unsorted) - - let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("c", &schema).unwrap(), - options: SortOptions::default(), - }]); - let input1 = sort_required_exec_with_req( - parquet_exec_with_sort(vec![sort_key.clone()]), - sort_key, - ); - let input2 = filter_exec(parquet_exec()); - let plan = union_exec(vec![input1, input2]); - - // should not repartition below the SortRequired as that - // branch doesn't benefit from increased parallelism - let expected = &[ - "UnionExec", - // union input 1: no repartitioning - "SortRequiredExec: [c@2 ASC]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", - // union input 2: should repartition - "FilterExec: c@2 = 0", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - assert_optimized!(expected, plan.clone(), true); - assert_optimized!(expected, plan, false); - - Ok(()) - } - - #[test] - fn repartition_transitively_with_projection() -> Result<()> { - let schema = schema(); - let proj_exprs = vec![( - Arc::new(BinaryExpr::new( - col("a", &schema).unwrap(), - Operator::Plus, - col("b", &schema).unwrap(), - )) as Arc, - "sum".to_string(), - )]; - // non sorted input - let proj = Arc::new(ProjectionExec::try_new(proj_exprs, parquet_exec())?); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("sum", &proj.schema()).unwrap(), - options: SortOptions::default(), - }]); - let plan = sort_preserving_merge_exec(sort_key, proj); - - let expected = &[ - "SortPreservingMergeExec: [sum@0 ASC]", - "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", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - - assert_optimized!(expected, plan.clone(), true); - - let expected_first_sort_enforcement = &[ - "SortExec: expr=[sum@0 ASC], preserve_partitioning=[false]", - "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", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - assert_optimized!(expected_first_sort_enforcement, plan, false); - - Ok(()) - } - - #[test] - fn repartition_ignores_transitively_with_projection() -> Result<()> { - let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("c", &schema).unwrap(), - options: SortOptions::default(), - }]); - let alias = vec![ - ("a".to_string(), "a".to_string()), - ("b".to_string(), "b".to_string()), - ("c".to_string(), "c".to_string()), - ]; - // sorted input - let plan = sort_required_exec_with_req( - projection_exec_with_alias( - parquet_exec_multiple_sorted(vec![sort_key.clone()]), - alias, - ), - sort_key, - ); - - let expected = &[ - "SortRequiredExec: [c@2 ASC]", - // Since this projection is trivial, increasing parallelism is not beneficial - "ProjectionExec: expr=[a@0 as a, b@1 as b, c@2 as c]", - "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", - ]; - assert_optimized!(expected, plan.clone(), true); - assert_optimized!(expected, plan, false); - - Ok(()) - } - - #[test] - fn repartition_transitively_past_sort_with_projection() -> Result<()> { - let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("c", &schema).unwrap(), - options: SortOptions::default(), - }]); - let alias = vec![ - ("a".to_string(), "a".to_string()), - ("b".to_string(), "b".to_string()), - ("c".to_string(), "c".to_string()), - ]; - let plan = sort_preserving_merge_exec( - sort_key.clone(), - sort_exec( - sort_key, - projection_exec_with_alias(parquet_exec(), alias), - true, - ), - ); - - let expected = &[ - "SortExec: expr=[c@2 ASC], preserve_partitioning=[true]", - // Since this projection is trivial, increasing parallelism is not beneficial - "ProjectionExec: expr=[a@0 as a, b@1 as b, c@2 as c]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - assert_optimized!(expected, plan.clone(), true); - assert_optimized!(expected, plan, false); - - Ok(()) - } - - #[test] - fn repartition_transitively_past_sort_with_filter() -> Result<()> { - let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("a", &schema).unwrap(), - options: SortOptions::default(), - }]); - let plan = sort_exec(sort_key, filter_exec(parquet_exec()), false); - - let expected = &[ - "SortPreservingMergeExec: [a@0 ASC]", - "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", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - - assert_optimized!(expected, plan.clone(), true); - - let expected_first_sort_enforcement = &[ - "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - "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", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - assert_optimized!(expected_first_sort_enforcement, plan, false); - - Ok(()) - } - - #[test] - #[cfg(feature = "parquet")] - fn repartition_transitively_past_sort_with_projection_and_filter() -> Result<()> { - let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("a", &schema).unwrap(), - options: SortOptions::default(), - }]); - let plan = sort_exec( - sort_key, - projection_exec_with_alias( - filter_exec(parquet_exec()), - vec![ - ("a".to_string(), "a".to_string()), - ("b".to_string(), "b".to_string()), - ("c".to_string(), "c".to_string()), - ], - ), - false, - ); - - let expected = &[ - "SortPreservingMergeExec: [a@0 ASC]", - // Expect repartition on the input to the sort (as it can benefit from additional parallelism) - "SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", - "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", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - - assert_optimized!(expected, plan.clone(), true); - - let expected_first_sort_enforcement = &[ - "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - "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", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - assert_optimized!(expected_first_sort_enforcement, plan, false); - - Ok(()) - } - - #[test] - fn parallelization_single_partition() -> Result<()> { - let alias = vec![("a".to_string(), "a".to_string())]; - let plan_parquet = aggregate_exec_with_alias(parquet_exec(), alias.clone()); - let plan_csv = aggregate_exec_with_alias(csv_exec(), alias); - - let expected_parquet = [ - "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", - "RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2", - "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", - "ParquetExec: file_groups={2 groups: [[x:0..50], [x:50..100]]}, projection=[a, b, c, d, e]", - ]; - let expected_csv = [ - "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", - "RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2", - "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", - "CsvExec: file_groups={2 groups: [[x:0..50], [x:50..100]]}, projection=[a, b, c, d, e], has_header=false", - ]; - assert_optimized!(expected_parquet, plan_parquet, true, false, 2, true, 10); - assert_optimized!(expected_csv, plan_csv, true, false, 2, true, 10); - - Ok(()) - } - - #[test] - fn parallelization_multiple_files() -> Result<()> { - let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("a", &schema).unwrap(), - options: SortOptions::default(), - }]); - - let plan = filter_exec(parquet_exec_multiple_sorted(vec![sort_key.clone()])); - let plan = sort_required_exec_with_req(plan, sort_key); - - // The groups must have only contiguous ranges of rows from the same file - // if any group has rows from multiple files, the data is no longer sorted destroyed - // https://github.com/apache/datafusion/issues/8451 - let expected = [ - "SortRequiredExec: [a@0 ASC]", - "FilterExec: c@2 = 0", - "ParquetExec: file_groups={3 groups: [[x:0..50], [y:0..100], [x:50..100]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", ]; - let target_partitions = 3; - let repartition_size = 1; - assert_optimized!( - expected, - plan, - true, - true, - target_partitions, - true, - repartition_size, - false - ); - - let expected = [ - "SortRequiredExec: [a@0 ASC]", - "FilterExec: c@2 = 0", - "ParquetExec: file_groups={8 groups: [[x:0..25], [y:0..25], [x:25..50], [y:25..50], [x:50..75], [y:50..75], [x:75..100], [y:75..100]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", - ]; - let target_partitions = 8; - let repartition_size = 1; - assert_optimized!( - expected, - plan, - true, - true, - target_partitions, - true, - repartition_size, - false - ); - - Ok(()) - } - - #[test] - /// CsvExec on compressed csv file will not be partitioned - /// (Not able to decompress chunked csv file) - fn parallelization_compressed_csv() -> Result<()> { - let compression_types = [ - FileCompressionType::GZIP, - FileCompressionType::BZIP2, - FileCompressionType::XZ, - FileCompressionType::ZSTD, - FileCompressionType::UNCOMPRESSED, - ]; - - let expected_not_partitioned = [ - "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", - "RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2", - "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", - "RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", - "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - ]; - - let expected_partitioned = [ - "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", - "RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2", - "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", - "CsvExec: file_groups={2 groups: [[x:0..50], [x:50..100]]}, projection=[a, b, c, d, e], has_header=false", - ]; - - for compression_type in compression_types { - let expected = if compression_type.is_compressed() { - &expected_not_partitioned[..] - } else { - &expected_partitioned[..] - }; - - let plan = aggregate_exec_with_alias( - Arc::new( - CsvExec::builder( - FileScanConfig::new( - ObjectStoreUrl::parse("test:///").unwrap(), - schema(), - ) - .with_file(PartitionedFile::new("x".to_string(), 100)), - ) - .with_has_header(false) - .with_delimeter(b',') - .with_quote(b'"') - .with_escape(None) - .with_comment(None) - .with_newlines_in_values(false) - .with_file_compression_type(compression_type) - .build(), - ), - vec![("a".to_string(), "a".to_string())], - ); - assert_optimized!(expected, plan, true, false, 2, true, 10, false); - } - Ok(()) - } - - #[test] - fn parallelization_two_partitions() -> Result<()> { - let alias = vec![("a".to_string(), "a".to_string())]; - let plan_parquet = - aggregate_exec_with_alias(parquet_exec_multiple(), alias.clone()); - let plan_csv = aggregate_exec_with_alias(csv_exec_multiple(), alias); - - let expected_parquet = [ - "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", - "RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2", - "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", - // Plan already has two partitions - "ParquetExec: file_groups={2 groups: [[x:0..100], [y:0..100]]}, projection=[a, b, c, d, e]", - ]; - let expected_csv = [ - "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", - "RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2", - "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", - // Plan already has two partitions - "CsvExec: file_groups={2 groups: [[x:0..100], [y:0..100]]}, projection=[a, b, c, d, e], has_header=false", - ]; - assert_optimized!(expected_parquet, plan_parquet, true, false, 2, true, 10); - assert_optimized!(expected_csv, plan_csv, true, false, 2, true, 10); - Ok(()) - } - - #[test] - fn parallelization_two_partitions_into_four() -> Result<()> { - let alias = vec![("a".to_string(), "a".to_string())]; - let plan_parquet = - aggregate_exec_with_alias(parquet_exec_multiple(), alias.clone()); - let plan_csv = aggregate_exec_with_alias(csv_exec_multiple(), alias); - - let expected_parquet = [ - "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", - "RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4", - "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", - // Multiple source files splitted across partitions - "ParquetExec: file_groups={4 groups: [[x:0..50], [x:50..100], [y:0..50], [y:50..100]]}, projection=[a, b, c, d, e]", - ]; - let expected_csv = [ - "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", - "RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4", - "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", - // Multiple source files splitted across partitions - "CsvExec: file_groups={4 groups: [[x:0..50], [x:50..100], [y:0..50], [y:50..100]]}, projection=[a, b, c, d, e], has_header=false", - ]; - assert_optimized!(expected_parquet, plan_parquet, true, false, 4, true, 10); - assert_optimized!(expected_csv, plan_csv, true, false, 4, true, 10); - - Ok(()) - } - - #[test] - fn parallelization_sorted_limit() -> Result<()> { - let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("c", &schema).unwrap(), - options: SortOptions::default(), - }]); - let plan_parquet = limit_exec(sort_exec(sort_key.clone(), parquet_exec(), false)); - let plan_csv = limit_exec(sort_exec(sort_key, csv_exec(), false)); - - let expected_parquet = &[ - "GlobalLimitExec: skip=0, fetch=100", - "LocalLimitExec: fetch=100", - // data is sorted so can't repartition here - "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - // Doesn't parallelize for SortExec without preserve_partitioning - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - let expected_csv = &[ - "GlobalLimitExec: skip=0, fetch=100", - "LocalLimitExec: fetch=100", - // data is sorted so can't repartition here - "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - // Doesn't parallelize for SortExec without preserve_partitioning - "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - ]; - assert_optimized!(expected_parquet, plan_parquet, true); - assert_optimized!(expected_csv, plan_csv, true); - - Ok(()) - } - - #[test] - fn parallelization_limit_with_filter() -> Result<()> { - let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("c", &schema).unwrap(), - options: SortOptions::default(), - }]); - let plan_parquet = limit_exec(filter_exec(sort_exec( - sort_key.clone(), - parquet_exec(), - false, - ))); - let plan_csv = limit_exec(filter_exec(sort_exec(sort_key, csv_exec(), false))); - - let expected_parquet = &[ - "GlobalLimitExec: skip=0, fetch=100", - "CoalescePartitionsExec", - "LocalLimitExec: fetch=100", - "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", - "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]", - ]; - let expected_csv = &[ - "GlobalLimitExec: skip=0, fetch=100", - "CoalescePartitionsExec", - "LocalLimitExec: fetch=100", - "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", - "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", - ]; - assert_optimized!(expected_parquet, plan_parquet, true); - assert_optimized!(expected_csv, plan_csv, true); - - Ok(()) - } - - #[test] - fn parallelization_ignores_limit() -> Result<()> { - let alias = vec![("a".to_string(), "a".to_string())]; - let plan_parquet = aggregate_exec_with_alias( - limit_exec(filter_exec(limit_exec(parquet_exec()))), - alias.clone(), - ); - let plan_csv = aggregate_exec_with_alias( - limit_exec(filter_exec(limit_exec(csv_exec()))), - alias, - ); - - let expected_parquet = &[ - "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", - "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", - "GlobalLimitExec: skip=0, fetch=100", - // Limit doesn't benefit from input partitioning - no parallelism - "LocalLimitExec: fetch=100", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - let expected_csv = &[ - "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", - "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", - "GlobalLimitExec: skip=0, fetch=100", - // Limit doesn't benefit from input partitioning - no parallelism - "LocalLimitExec: fetch=100", - "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - ]; - assert_optimized!(expected_parquet, plan_parquet, true); - assert_optimized!(expected_csv, plan_csv, true); - - Ok(()) - } - - #[test] - fn parallelization_union_inputs() -> Result<()> { - let plan_parquet = union_exec(vec![parquet_exec(); 5]); - let plan_csv = union_exec(vec![csv_exec(); 5]); - - let expected_parquet = &[ - "UnionExec", - // Union doesn't benefit from input partitioning - no parallelism - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - let expected_csv = &[ - "UnionExec", - // Union doesn't benefit from input partitioning - no parallelism - "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - ]; - assert_optimized!(expected_parquet, plan_parquet, true); - assert_optimized!(expected_csv, plan_csv, true); - - Ok(()) - } - - #[test] - fn parallelization_prior_to_sort_preserving_merge() -> Result<()> { - let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("c", &schema).unwrap(), - options: SortOptions::default(), - }]); - // sort preserving merge already sorted input, - let plan_parquet = sort_preserving_merge_exec( - sort_key.clone(), - parquet_exec_with_sort(vec![sort_key.clone()]), - ); - let plan_csv = sort_preserving_merge_exec( - sort_key.clone(), - csv_exec_with_sort(vec![sort_key]), - ); - - // parallelization is not beneficial for SortPreservingMerge - let expected_parquet = &[ - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", - ]; - let expected_csv = &[ - "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], has_header=false", - ]; - assert_optimized!(expected_parquet, plan_parquet, true); - assert_optimized!(expected_csv, plan_csv, true); - - Ok(()) - } - - #[test] - fn parallelization_sort_preserving_merge_with_union() -> Result<()> { - let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("c", &schema).unwrap(), - options: SortOptions::default(), - }]); - // 2 sorted parquet files unioned (partitions are concatenated, sort is preserved) - let input_parquet = - union_exec(vec![parquet_exec_with_sort(vec![sort_key.clone()]); 2]); - let input_csv = union_exec(vec![csv_exec_with_sort(vec![sort_key.clone()]); 2]); - let plan_parquet = sort_preserving_merge_exec(sort_key.clone(), input_parquet); - let plan_csv = sort_preserving_merge_exec(sort_key, input_csv); - - // should not repartition (union doesn't benefit from increased parallelism) - // should not sort (as the data was already sorted) - let expected_parquet = &[ - "SortPreservingMergeExec: [c@2 ASC]", - "UnionExec", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", - ]; - let expected_csv = &[ - "SortPreservingMergeExec: [c@2 ASC]", - "UnionExec", - "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], has_header=false", - "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], has_header=false", - ]; - assert_optimized!(expected_parquet, plan_parquet, true); - assert_optimized!(expected_csv, plan_csv, true); - - Ok(()) - } - - #[test] - fn parallelization_does_not_benefit() -> Result<()> { - let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("c", &schema).unwrap(), - options: SortOptions::default(), - }]); - // SortRequired - // Parquet(sorted) - let plan_parquet = sort_required_exec_with_req( - parquet_exec_with_sort(vec![sort_key.clone()]), - sort_key.clone(), - ); - let plan_csv = sort_required_exec_with_req( - csv_exec_with_sort(vec![sort_key.clone()]), - sort_key, - ); - - // no parallelization, because SortRequiredExec doesn't benefit from increased parallelism - let expected_parquet = &[ - "SortRequiredExec: [c@2 ASC]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", - ]; - let expected_csv = &[ - "SortRequiredExec: [c@2 ASC]", - "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], has_header=false", - ]; - assert_optimized!(expected_parquet, plan_parquet, true); - assert_optimized!(expected_csv, plan_csv, true); - - Ok(()) - } - - #[test] - fn parallelization_ignores_transitively_with_projection_parquet() -> Result<()> { - // sorted input - let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("c", &schema).unwrap(), - options: SortOptions::default(), - }]); - - //Projection(a as a2, b as b2) - let alias_pairs: Vec<(String, String)> = vec![ - ("a".to_string(), "a2".to_string()), - ("c".to_string(), "c2".to_string()), - ]; - let proj_parquet = projection_exec_with_alias( - parquet_exec_with_sort(vec![sort_key]), - alias_pairs, - ); - let sort_key_after_projection = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("c2", &proj_parquet.schema()).unwrap(), - options: SortOptions::default(), - }]); - let plan_parquet = - sort_preserving_merge_exec(sort_key_after_projection, proj_parquet); - let expected = &[ - "SortPreservingMergeExec: [c2@1 ASC]", - " ProjectionExec: expr=[a@0 as a2, c@2 as c2]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", - ]; - plans_matches_expected!(expected, &plan_parquet); - - // data should not be repartitioned / resorted - let expected_parquet = &[ - "ProjectionExec: expr=[a@0 as a2, c@2 as c2]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", - ]; - assert_optimized!(expected_parquet, plan_parquet, true); - - Ok(()) - } - - #[test] - fn parallelization_ignores_transitively_with_projection_csv() -> Result<()> { - // sorted input - let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("c", &schema).unwrap(), - options: SortOptions::default(), - }]); - - //Projection(a as a2, b as b2) - let alias_pairs: Vec<(String, String)> = vec![ - ("a".to_string(), "a2".to_string()), - ("c".to_string(), "c2".to_string()), - ]; - - let proj_csv = - projection_exec_with_alias(csv_exec_with_sort(vec![sort_key]), alias_pairs); - let sort_key_after_projection = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("c2", &proj_csv.schema()).unwrap(), - options: SortOptions::default(), - }]); - let plan_csv = sort_preserving_merge_exec(sort_key_after_projection, proj_csv); - let expected = &[ - "SortPreservingMergeExec: [c2@1 ASC]", - " ProjectionExec: expr=[a@0 as a2, c@2 as c2]", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], has_header=false", - ]; - plans_matches_expected!(expected, &plan_csv); - - // data should not be repartitioned / resorted - let expected_csv = &[ - "ProjectionExec: expr=[a@0 as a2, c@2 as c2]", - "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], has_header=false", - ]; - assert_optimized!(expected_csv, plan_csv, true); - - Ok(()) - } - - #[test] - fn remove_redundant_roundrobins() -> Result<()> { - let input = parquet_exec(); - let repartition = repartition_exec(repartition_exec(input)); - let physical_plan = repartition_exec(filter_exec(repartition)); - let expected = &[ - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10", - " FilterExec: c@2 = 0", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - plans_matches_expected!(expected, &physical_plan); - - let expected = &[ - "FilterExec: c@2 = 0", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - assert_optimized!(expected, physical_plan.clone(), true); - assert_optimized!(expected, physical_plan, false); - - Ok(()) - } - - #[test] - fn remove_unnecessary_spm_after_filter() -> Result<()> { - let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("c", &schema).unwrap(), - options: SortOptions::default(), - }]); - let input = parquet_exec_multiple_sorted(vec![sort_key.clone()]); - let physical_plan = sort_preserving_merge_exec(sort_key, filter_exec(input)); - - // Original plan expects its output to be ordered by c@2 ASC. - // This is still satisfied since, after filter that column is constant. - let expected = &[ - "CoalescePartitionsExec", - "FilterExec: c@2 = 0", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2, preserve_order=true, sort_exprs=c@2 ASC", - "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", - ]; - // last flag sets config.optimizer.PREFER_EXISTING_SORT - assert_optimized!(expected, physical_plan.clone(), true, true); - assert_optimized!(expected, physical_plan, false, true); - - Ok(()) - } - - #[test] - fn preserve_ordering_through_repartition() -> Result<()> { - let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("d", &schema).unwrap(), - options: SortOptions::default(), - }]); - let input = parquet_exec_multiple_sorted(vec![sort_key.clone()]); - let physical_plan = sort_preserving_merge_exec(sort_key, filter_exec(input)); - - 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", - "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[d@3 ASC]", - ]; - // last flag sets config.optimizer.PREFER_EXISTING_SORT - assert_optimized!(expected, physical_plan.clone(), true, true); - assert_optimized!(expected, physical_plan, false, true); - - Ok(()) - } - - #[test] - fn do_not_preserve_ordering_through_repartition() -> Result<()> { - let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("a", &schema).unwrap(), - options: SortOptions::default(), - }]); - let input = parquet_exec_multiple_sorted(vec![sort_key.clone()]); - let physical_plan = sort_preserving_merge_exec(sort_key, filter_exec(input)); - - let expected = &[ - "SortPreservingMergeExec: [a@0 ASC]", - "SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", - "FilterExec: c@2 = 0", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", - "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", - ]; - - assert_optimized!(expected, physical_plan.clone(), true); - - let expected = &[ - "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - "CoalescePartitionsExec", - "FilterExec: c@2 = 0", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", - "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", - ]; - assert_optimized!(expected, physical_plan, false); - - Ok(()) - } - - #[test] - fn no_need_for_sort_after_filter() -> Result<()> { - let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("c", &schema).unwrap(), - options: SortOptions::default(), - }]); - let input = parquet_exec_multiple_sorted(vec![sort_key.clone()]); - let physical_plan = sort_preserving_merge_exec(sort_key, filter_exec(input)); - - let expected = &[ - // After CoalescePartitionsExec c is still constant. Hence c@2 ASC ordering is already satisfied. - "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", - "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", - ]; - assert_optimized!(expected, physical_plan.clone(), true); - assert_optimized!(expected, physical_plan, false); - - Ok(()) - } - - #[test] - fn do_not_preserve_ordering_through_repartition2() -> Result<()> { - let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("c", &schema).unwrap(), - options: SortOptions::default(), - }]); - let input = parquet_exec_multiple_sorted(vec![sort_key]); - - let sort_req = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("a", &schema).unwrap(), - options: SortOptions::default(), - }]); - let physical_plan = sort_preserving_merge_exec(sort_req, filter_exec(input)); - - let expected = &[ - "SortPreservingMergeExec: [a@0 ASC]", - "SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", - "FilterExec: c@2 = 0", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", - "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", - ]; - - assert_optimized!(expected, physical_plan.clone(), true); - - let expected = &[ - "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - "CoalescePartitionsExec", - "SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", - "FilterExec: c@2 = 0", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", - "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", - ]; - assert_optimized!(expected, physical_plan, false); - - Ok(()) - } - - #[test] - fn do_not_preserve_ordering_through_repartition3() -> Result<()> { - let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("c", &schema).unwrap(), - options: SortOptions::default(), - }]); - let input = parquet_exec_multiple_sorted(vec![sort_key]); - let physical_plan = filter_exec(input); - - let expected = &[ - "FilterExec: c@2 = 0", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", - "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", - ]; - assert_optimized!(expected, physical_plan.clone(), true); - assert_optimized!(expected, physical_plan, false); - - Ok(()) - } - - #[test] - fn do_not_put_sort_when_input_is_invalid() -> Result<()> { - let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("a", &schema).unwrap(), - options: SortOptions::default(), - }]); - let input = parquet_exec(); - let physical_plan = sort_required_exec_with_req(filter_exec(input), sort_key); - let expected = &[ - // Ordering requirement of sort required exec is NOT satisfied - // by existing ordering at the source. - "SortRequiredExec: [a@0 ASC]", - "FilterExec: c@2 = 0", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - assert_plan_txt!(expected, physical_plan); - - let expected = &[ - "SortRequiredExec: [a@0 ASC]", - // 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", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - - let mut config = ConfigOptions::new(); - config.execution.target_partitions = 10; - config.optimizer.enable_round_robin_repartition = true; - config.optimizer.prefer_existing_sort = false; - let dist_plan = EnforceDistribution::new().optimize(physical_plan, &config)?; - assert_plan_txt!(expected, dist_plan); - - Ok(()) - } - - #[test] - fn put_sort_when_input_is_valid() -> Result<()> { - let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("a", &schema).unwrap(), - options: SortOptions::default(), - }]); - let input = parquet_exec_multiple_sorted(vec![sort_key.clone()]); - let physical_plan = sort_required_exec_with_req(filter_exec(input), sort_key); - - let expected = &[ - // Ordering requirement of sort required exec is satisfied - // by existing ordering at the source. - "SortRequiredExec: [a@0 ASC]", - "FilterExec: c@2 = 0", - "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", - ]; - assert_plan_txt!(expected, physical_plan); - - let expected = &[ - // Since at the start of the rule ordering requirement is satisfied - // EnforceDistribution rule satisfy this requirement also. - "SortRequiredExec: [a@0 ASC]", - "FilterExec: c@2 = 0", - "ParquetExec: file_groups={10 groups: [[x:0..20], [y:0..20], [x:20..40], [y:20..40], [x:40..60], [y:40..60], [x:60..80], [y:60..80], [x:80..100], [y:80..100]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", - ]; - - let mut config = ConfigOptions::new(); - config.execution.target_partitions = 10; - config.optimizer.enable_round_robin_repartition = true; - config.optimizer.prefer_existing_sort = false; - let dist_plan = EnforceDistribution::new().optimize(physical_plan, &config)?; - assert_plan_txt!(expected, dist_plan); - - Ok(()) - } - - #[test] - fn do_not_add_unnecessary_hash() -> Result<()> { - let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("c", &schema).unwrap(), - options: SortOptions::default(), - }]); - let alias = vec![("a".to_string(), "a".to_string())]; - let input = parquet_exec_with_sort(vec![sort_key]); - let physical_plan = aggregate_exec_with_alias(input, alias); - - let expected = &[ - "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", - "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", - ]; - // Make sure target partition number is 1. In this case hash repartition is unnecessary - assert_optimized!(expected, physical_plan.clone(), true, false, 1, false, 1024); - assert_optimized!(expected, physical_plan, false, false, 1, false, 1024); - - Ok(()) - } - - #[test] - fn do_not_add_unnecessary_hash2() -> Result<()> { - let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("c", &schema).unwrap(), - options: SortOptions::default(), - }]); - let alias = vec![("a".to_string(), "a".to_string())]; - let input = parquet_exec_multiple_sorted(vec![sort_key]); - let aggregate = aggregate_exec_with_alias(input, alias.clone()); - let physical_plan = aggregate_exec_with_alias(aggregate, alias); - - let expected = &[ - "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", - // Since hash requirements of this operator is satisfied. There shouldn't be - // a hash repartition here - "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", - "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", - "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", - ]; - // Make sure target partition number is larger than 2 (e.g partition number at the source). - assert_optimized!(expected, physical_plan.clone(), true, false, 4, false, 1024); - assert_optimized!(expected, physical_plan, false, false, 4, false, 1024); - - Ok(()) - } - - #[test] - fn optimize_away_unnecessary_repartition() -> Result<()> { - let physical_plan = coalesce_partitions_exec(repartition_exec(parquet_exec())); - let expected = &[ - "CoalescePartitionsExec", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - plans_matches_expected!(expected, physical_plan.clone()); - - let expected = - &["ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]"]; - assert_optimized!(expected, physical_plan.clone(), true); - assert_optimized!(expected, physical_plan, false); - - Ok(()) - } - - #[test] - fn optimize_away_unnecessary_repartition2() -> Result<()> { - let physical_plan = filter_exec(repartition_exec(coalesce_partitions_exec( - filter_exec(repartition_exec(parquet_exec())), - ))); - let expected = &[ - "FilterExec: c@2 = 0", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " CoalescePartitionsExec", - " FilterExec: c@2 = 0", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - plans_matches_expected!(expected, physical_plan.clone()); - - let expected = &[ - "FilterExec: c@2 = 0", - "FilterExec: c@2 = 0", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - assert_optimized!(expected, physical_plan.clone(), true); - assert_optimized!(expected, physical_plan, false); - - Ok(()) - } -} diff --git a/datafusion/core/src/physical_optimizer/mod.rs b/datafusion/core/src/physical_optimizer/mod.rs index 4ceefe60676e..e6aa15a4c09d 100644 --- a/datafusion/core/src/physical_optimizer/mod.rs +++ b/datafusion/core/src/physical_optimizer/mod.rs @@ -22,7 +22,6 @@ //! //! [`ExecutionPlan`]: crate::physical_plan::ExecutionPlan -pub mod enforce_distribution; pub mod optimizer; pub mod projection_pushdown; diff --git a/datafusion/core/src/test_util/mod.rs b/datafusion/core/src/test_util/mod.rs index d608db25fe98..50e33b27e1bb 100644 --- a/datafusion/core/src/test_util/mod.rs +++ b/datafusion/core/src/test_util/mod.rs @@ -44,11 +44,7 @@ use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use arrow::record_batch::RecordBatch; use datafusion_catalog::Session; use datafusion_common::TableReference; -use datafusion_expr::utils::COUNT_STAR_EXPANSION; use datafusion_expr::{CreateExternalTable, Expr, SortExpr, TableType}; -use datafusion_functions_aggregate::count::count_udaf; -use datafusion_physical_expr::aggregate::{AggregateExprBuilder, AggregateFunctionExpr}; -use datafusion_physical_expr::{expressions, PhysicalExpr}; use async_trait::async_trait; use futures::Stream; @@ -276,57 +272,3 @@ pub fn bounded_stream(batch: RecordBatch, limit: usize) -> SendableRecordBatchSt batch, }) } - -/// Describe the type of aggregate being tested -pub enum TestAggregate { - /// Testing COUNT(*) type aggregates - CountStar, - - /// Testing for COUNT(column) aggregate - ColumnA(Arc), -} - -impl TestAggregate { - /// Create a new COUNT(*) aggregate - pub fn new_count_star() -> Self { - Self::CountStar - } - - /// Create a new COUNT(column) aggregate - pub fn new_count_column(schema: &Arc) -> Self { - Self::ColumnA(Arc::clone(schema)) - } - - /// Return appropriate expr depending if COUNT is for col or table (*) - pub fn count_expr(&self, schema: &Schema) -> AggregateFunctionExpr { - AggregateExprBuilder::new(count_udaf(), vec![self.column()]) - .schema(Arc::new(schema.clone())) - .alias(self.column_name()) - .build() - .unwrap() - } - - /// what argument would this aggregate need in the plan? - fn column(&self) -> Arc { - match self { - Self::CountStar => expressions::lit(COUNT_STAR_EXPANSION), - Self::ColumnA(s) => expressions::col("a", s).unwrap(), - } - } - - /// What name would this aggregate produce in a plan? - pub fn column_name(&self) -> &'static str { - match self { - Self::CountStar => "COUNT(*)", - Self::ColumnA(_) => "COUNT(a)", - } - } - - /// What is the expected count? - pub fn expected_count(&self) -> i64 { - match self { - TestAggregate::CountStar => 3, - TestAggregate::ColumnA(_) => 2, - } - } -} diff --git a/datafusion/core/tests/core_integration.rs b/datafusion/core/tests/core_integration.rs index e0917e6cca19..93f66282333d 100644 --- a/datafusion/core/tests/core_integration.rs +++ b/datafusion/core/tests/core_integration.rs @@ -42,7 +42,6 @@ mod custom_sources_cases; /// Run all tests that are found in the `optimizer` directory mod optimizer; -/// Run all tests that are found in the `physical_optimizer` directory mod physical_optimizer; #[cfg(test)] 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 b8a96f0f5a22..85efebf2386a 100644 --- a/datafusion/core/tests/physical_optimizer/combine_partial_final_agg.rs +++ b/datafusion/core/tests/physical_optimizer/combine_partial_final_agg.rs @@ -19,14 +19,15 @@ //! //! Note these tests are not in the same module as the optimizer pass because //! they rely on `ParquetExec` which is in the core crate. + use std::sync::Arc; +use crate::physical_optimizer::parquet_exec; + use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; -use datafusion::datasource::listing::PartitionedFile; -use datafusion::datasource::physical_plan::{FileScanConfig, ParquetExec}; use datafusion::physical_optimizer::combine_partial_final_agg::CombinePartialFinalAggregate; +use datafusion::physical_optimizer::test_utils::trim_plan_display; use datafusion_common::config::ConfigOptions; -use datafusion_execution::object_store::ObjectStoreUrl; use datafusion_functions_aggregate::count::count_udaf; use datafusion_functions_aggregate::sum::sum_udaf; use datafusion_physical_expr::aggregate::{AggregateExprBuilder, AggregateFunctionExpr}; @@ -62,13 +63,6 @@ macro_rules! assert_optimized { }; } -fn trim_plan_display(plan: &str) -> Vec<&str> { - plan.split('\n') - .map(|s| s.trim()) - .filter(|s| !s.is_empty()) - .collect() -} - fn schema() -> SchemaRef { Arc::new(Schema::new(vec![ Field::new("a", DataType::Int64, true), @@ -77,14 +71,6 @@ fn schema() -> SchemaRef { ])) } -fn parquet_exec(schema: &SchemaRef) -> Arc { - ParquetExec::builder( - FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema.clone()) - .with_file(PartitionedFile::new("x".to_string(), 100)), - ) - .build_arc() -} - fn partial_aggregate_exec( input: Arc, group_by: PhysicalGroupBy, diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs new file mode 100644 index 000000000000..ea75e7d0e5f4 --- /dev/null +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -0,0 +1,3175 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use std::fmt::Debug; +use std::ops::Deref; +use std::sync::Arc; + +use crate::physical_optimizer::parquet_exec_with_sort; + +use arrow::compute::SortOptions; +use datafusion::config::ConfigOptions; +use datafusion::datasource::file_format::file_compression_type::FileCompressionType; +use datafusion::datasource::listing::PartitionedFile; +use datafusion::datasource::object_store::ObjectStoreUrl; +use datafusion::datasource::physical_plan::{CsvExec, FileScanConfig, ParquetExec}; +use datafusion_common::error::Result; +use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; +use datafusion_common::ScalarValue; +use datafusion_expr::{JoinType, Operator}; +use datafusion_physical_expr::expressions::{BinaryExpr, Column, Literal}; +use datafusion_physical_expr::PhysicalExpr; +use datafusion_physical_expr::{ + expressions::binary, expressions::lit, LexOrdering, PhysicalSortExpr, +}; +use datafusion_physical_expr_common::sort_expr::LexRequirement; +use datafusion_physical_optimizer::enforce_distribution::*; +use datafusion_physical_optimizer::enforce_sorting::EnforceSorting; +use datafusion_physical_optimizer::output_requirements::OutputRequirements; +use datafusion_physical_optimizer::test_utils::trim_plan_display; +use datafusion_physical_optimizer::test_utils::{ + check_integrity, coalesce_partitions_exec, repartition_exec, schema, + sort_merge_join_exec, sort_preserving_merge_exec, +}; +use datafusion_physical_optimizer::PhysicalOptimizerRule; +use datafusion_physical_plan::aggregates::{ + AggregateExec, AggregateMode, PhysicalGroupBy, +}; +use datafusion_physical_plan::coalesce_batches::CoalesceBatchesExec; +use datafusion_physical_plan::execution_plan::ExecutionPlan; +use datafusion_physical_plan::expressions::col; +use datafusion_physical_plan::filter::FilterExec; +use datafusion_physical_plan::joins::utils::JoinOn; +use datafusion_physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; +use datafusion_physical_plan::projection::ProjectionExec; +use datafusion_physical_plan::sorts::sort::SortExec; +use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; +use datafusion_physical_plan::union::UnionExec; +use datafusion_physical_plan::ExecutionPlanProperties; +use datafusion_physical_plan::PlanProperties; +use datafusion_physical_plan::{displayable, DisplayAs, DisplayFormatType, Statistics}; + +/// Models operators like BoundedWindowExec that require an input +/// ordering but is easy to construct +#[derive(Debug)] +struct SortRequiredExec { + input: Arc, + expr: LexOrdering, + cache: PlanProperties, +} + +impl SortRequiredExec { + fn new_with_requirement( + input: Arc, + requirement: LexOrdering, + ) -> Self { + let cache = Self::compute_properties(&input); + Self { + input, + expr: requirement, + cache, + } + } + + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. + fn compute_properties(input: &Arc) -> PlanProperties { + PlanProperties::new( + input.equivalence_properties().clone(), // Equivalence Properties + input.output_partitioning().clone(), // Output Partitioning + input.pipeline_behavior(), // Pipeline Behavior + input.boundedness(), // Boundedness + ) + } +} + +impl DisplayAs for SortRequiredExec { + fn fmt_as( + &self, + _t: DisplayFormatType, + f: &mut std::fmt::Formatter, + ) -> std::fmt::Result { + write!(f, "SortRequiredExec: [{}]", self.expr) + } +} + +impl ExecutionPlan for SortRequiredExec { + fn name(&self) -> &'static str { + "SortRequiredExec" + } + + fn as_any(&self) -> &dyn std::any::Any { + self + } + + fn properties(&self) -> &PlanProperties { + &self.cache + } + + fn benefits_from_input_partitioning(&self) -> Vec { + vec![false] + } + + fn children(&self) -> Vec<&Arc> { + vec![&self.input] + } + + // model that it requires the output ordering of its input + fn required_input_ordering(&self) -> Vec> { + if self.expr.is_empty() { + vec![None] + } else { + vec![Some(LexRequirement::from(self.expr.clone()))] + } + } + + fn with_new_children( + self: Arc, + mut children: Vec>, + ) -> Result> { + assert_eq!(children.len(), 1); + let child = children.pop().unwrap(); + Ok(Arc::new(Self::new_with_requirement( + child, + self.expr.clone(), + ))) + } + + fn execute( + &self, + _partition: usize, + _context: Arc, + ) -> Result { + unreachable!(); + } + + fn statistics(&self) -> Result { + self.input.statistics() + } +} + +fn parquet_exec() -> Arc { + parquet_exec_with_sort(vec![]) +} + +fn parquet_exec_multiple() -> Arc { + parquet_exec_multiple_sorted(vec![]) +} + +/// Created a sorted parquet exec with multiple files +fn parquet_exec_multiple_sorted(output_ordering: Vec) -> Arc { + ParquetExec::builder( + FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema()) + .with_file_groups(vec![ + vec![PartitionedFile::new("x".to_string(), 100)], + vec![PartitionedFile::new("y".to_string(), 100)], + ]) + .with_output_ordering(output_ordering), + ) + .build_arc() +} + +fn csv_exec() -> Arc { + csv_exec_with_sort(vec![]) +} + +fn csv_exec_with_sort(output_ordering: Vec) -> Arc { + Arc::new( + CsvExec::builder( + FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema()) + .with_file(PartitionedFile::new("x".to_string(), 100)) + .with_output_ordering(output_ordering), + ) + .with_has_header(false) + .with_delimeter(b',') + .with_quote(b'"') + .with_escape(None) + .with_comment(None) + .with_newlines_in_values(false) + .with_file_compression_type(FileCompressionType::UNCOMPRESSED) + .build(), + ) +} + +fn csv_exec_multiple() -> Arc { + csv_exec_multiple_sorted(vec![]) +} + +// Created a sorted parquet exec with multiple files +fn csv_exec_multiple_sorted(output_ordering: Vec) -> Arc { + Arc::new( + CsvExec::builder( + FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema()) + .with_file_groups(vec![ + vec![PartitionedFile::new("x".to_string(), 100)], + vec![PartitionedFile::new("y".to_string(), 100)], + ]) + .with_output_ordering(output_ordering), + ) + .with_has_header(false) + .with_delimeter(b',') + .with_quote(b'"') + .with_escape(None) + .with_comment(None) + .with_newlines_in_values(false) + .with_file_compression_type(FileCompressionType::UNCOMPRESSED) + .build(), + ) +} + +fn projection_exec_with_alias( + input: Arc, + alias_pairs: Vec<(String, String)>, +) -> Arc { + let mut exprs = vec![]; + for (column, alias) in alias_pairs.iter() { + exprs.push((col(column, &input.schema()).unwrap(), alias.to_string())); + } + Arc::new(ProjectionExec::try_new(exprs, input).unwrap()) +} + +fn aggregate_exec_with_alias( + input: Arc, + alias_pairs: Vec<(String, String)>, +) -> Arc { + let schema = schema(); + let mut group_by_expr: Vec<(Arc, String)> = vec![]; + for (column, alias) in alias_pairs.iter() { + group_by_expr.push((col(column, &input.schema()).unwrap(), alias.to_string())); + } + let group_by = PhysicalGroupBy::new_single(group_by_expr.clone()); + + let final_group_by_expr = group_by_expr + .iter() + .enumerate() + .map(|(index, (_col, name))| { + ( + Arc::new(Column::new(name, index)) as Arc, + name.clone(), + ) + }) + .collect::>(); + let final_grouping = PhysicalGroupBy::new_single(final_group_by_expr); + + Arc::new( + AggregateExec::try_new( + AggregateMode::FinalPartitioned, + final_grouping, + vec![], + vec![], + Arc::new( + AggregateExec::try_new( + AggregateMode::Partial, + group_by, + vec![], + vec![], + input, + schema.clone(), + ) + .unwrap(), + ), + schema, + ) + .unwrap(), + ) +} + +fn hash_join_exec( + left: Arc, + right: Arc, + join_on: &JoinOn, + join_type: &JoinType, +) -> Arc { + datafusion_physical_optimizer::test_utils::hash_join_exec( + left, + right, + join_on.clone(), + None, + join_type, + ) + .unwrap() +} + +fn filter_exec(input: Arc) -> Arc { + let predicate = Arc::new(BinaryExpr::new( + col("c", &schema()).unwrap(), + Operator::Eq, + Arc::new(Literal::new(ScalarValue::Int64(Some(0)))), + )); + Arc::new(FilterExec::try_new(predicate, input).unwrap()) +} + +fn sort_exec( + sort_exprs: LexOrdering, + input: Arc, + preserve_partitioning: bool, +) -> Arc { + let new_sort = SortExec::new(sort_exprs, input) + .with_preserve_partitioning(preserve_partitioning); + Arc::new(new_sort) +} + +fn limit_exec(input: Arc) -> Arc { + Arc::new(GlobalLimitExec::new( + Arc::new(LocalLimitExec::new(input, 100)), + 0, + Some(100), + )) +} + +fn union_exec(input: Vec>) -> Arc { + Arc::new(UnionExec::new(input)) +} + +fn sort_required_exec_with_req( + input: Arc, + sort_exprs: LexOrdering, +) -> Arc { + Arc::new(SortRequiredExec::new_with_requirement(input, sort_exprs)) +} + +fn ensure_distribution_helper( + plan: Arc, + target_partitions: usize, + prefer_existing_sort: bool, +) -> Result> { + let distribution_context = DistributionContext::new_default(plan); + let mut config = ConfigOptions::new(); + config.execution.target_partitions = target_partitions; + config.optimizer.enable_round_robin_repartition = true; + config.optimizer.repartition_file_scans = false; + config.optimizer.repartition_file_min_size = 1024; + config.optimizer.prefer_existing_sort = prefer_existing_sort; + ensure_distribution(distribution_context, &config).map(|item| item.data.plan) +} + +/// Test whether plan matches with expected plan +macro_rules! plans_matches_expected { + ($EXPECTED_LINES: expr, $PLAN: expr) => { + let physical_plan = $PLAN; + let formatted = displayable(physical_plan.as_ref()).indent(true).to_string(); + let actual: Vec<&str> = formatted.trim().lines().collect(); + + let expected_plan_lines: Vec<&str> = $EXPECTED_LINES + .iter().map(|s| *s).collect(); + + assert_eq!( + expected_plan_lines, actual, + "\n**Original Plan Mismatch\n\nexpected:\n\n{expected_plan_lines:#?}\nactual:\n\n{actual:#?}\n\n" + ); + } +} + +/// Runs the repartition optimizer and asserts the plan against the expected +/// Arguments +/// * `EXPECTED_LINES` - Expected output plan +/// * `PLAN` - Input plan +/// * `FIRST_ENFORCE_DIST` - +/// true: (EnforceDistribution, EnforceDistribution, EnforceSorting) +/// false: else runs (EnforceSorting, EnforceDistribution, EnforceDistribution) +/// * `PREFER_EXISTING_SORT` (optional) - if true, will not repartition / resort data if it is already sorted +/// * `TARGET_PARTITIONS` (optional) - number of partitions to repartition to +/// * `REPARTITION_FILE_SCANS` (optional) - if true, will repartition file scans +/// * `REPARTITION_FILE_MIN_SIZE` (optional) - minimum file size to repartition +/// * `PREFER_EXISTING_UNION` (optional) - if true, will not attempt to convert Union to Interleave +macro_rules! assert_optimized { + ($EXPECTED_LINES: expr, $PLAN: expr, $FIRST_ENFORCE_DIST: expr) => { + assert_optimized!($EXPECTED_LINES, $PLAN, $FIRST_ENFORCE_DIST, false, 10, false, 1024, false); + }; + + ($EXPECTED_LINES: expr, $PLAN: expr, $FIRST_ENFORCE_DIST: expr, $PREFER_EXISTING_SORT: expr) => { + assert_optimized!($EXPECTED_LINES, $PLAN, $FIRST_ENFORCE_DIST, $PREFER_EXISTING_SORT, 10, false, 1024, false); + }; + + ($EXPECTED_LINES: expr, $PLAN: expr, $FIRST_ENFORCE_DIST: expr, $PREFER_EXISTING_SORT: expr, $PREFER_EXISTING_UNION: expr) => { + assert_optimized!($EXPECTED_LINES, $PLAN, $FIRST_ENFORCE_DIST, $PREFER_EXISTING_SORT, 10, false, 1024, $PREFER_EXISTING_UNION); + }; + + ($EXPECTED_LINES: expr, $PLAN: expr, $FIRST_ENFORCE_DIST: expr, $PREFER_EXISTING_SORT: expr, $TARGET_PARTITIONS: expr, $REPARTITION_FILE_SCANS: expr, $REPARTITION_FILE_MIN_SIZE: expr) => { + assert_optimized!($EXPECTED_LINES, $PLAN, $FIRST_ENFORCE_DIST, $PREFER_EXISTING_SORT, $TARGET_PARTITIONS, $REPARTITION_FILE_SCANS, $REPARTITION_FILE_MIN_SIZE, false); + }; + + ($EXPECTED_LINES: expr, $PLAN: expr, $FIRST_ENFORCE_DIST: expr, $PREFER_EXISTING_SORT: expr, $TARGET_PARTITIONS: expr, $REPARTITION_FILE_SCANS: expr, $REPARTITION_FILE_MIN_SIZE: expr, $PREFER_EXISTING_UNION: expr) => { + let expected_lines: Vec<&str> = $EXPECTED_LINES.iter().map(|s| *s).collect(); + + let mut config = ConfigOptions::new(); + config.execution.target_partitions = $TARGET_PARTITIONS; + config.optimizer.repartition_file_scans = $REPARTITION_FILE_SCANS; + config.optimizer.repartition_file_min_size = $REPARTITION_FILE_MIN_SIZE; + config.optimizer.prefer_existing_sort = $PREFER_EXISTING_SORT; + config.optimizer.prefer_existing_union = $PREFER_EXISTING_UNION; + // Use a small batch size, to trigger RoundRobin in tests + config.execution.batch_size = 1; + + // NOTE: These tests verify the joint `EnforceDistribution` + `EnforceSorting` cascade + // because they were written prior to the separation of `BasicEnforcement` into + // `EnforceSorting` and `EnforceDistribution`. + // TODO: Orthogonalize the tests here just to verify `EnforceDistribution` and create + // new tests for the cascade. + + // Add the ancillary output requirements operator at the start: + let optimizer = OutputRequirements::new_add_mode(); + let optimized = optimizer.optimize($PLAN.clone(), &config)?; + + // This file has 2 rules that use tree node, apply these rules to original plan consecutively + // After these operations tree nodes should be in a consistent state. + // This code block makes sure that these rules doesn't violate tree node integrity. + { + let adjusted = if config.optimizer.top_down_join_key_reordering { + // Run adjust_input_keys_ordering rule + let plan_requirements = + PlanWithKeyRequirements::new_default($PLAN.clone()); + let adjusted = plan_requirements + .transform_down(adjust_input_keys_ordering) + .data() + .and_then(check_integrity)?; + // TODO: End state payloads will be checked here. + adjusted.plan + } else { + // Run reorder_join_keys_to_inputs rule + $PLAN.clone().transform_up(|plan| { + Ok(Transformed::yes(reorder_join_keys_to_inputs(plan)?)) + }) + .data()? + }; + + // Then run ensure_distribution rule + DistributionContext::new_default(adjusted) + .transform_up(|distribution_context| { + ensure_distribution(distribution_context, &config) + }) + .data() + .and_then(check_integrity)?; + // TODO: End state payloads will be checked here. + } + + let optimized = if $FIRST_ENFORCE_DIST { + // Run enforce distribution rule first: + let optimizer = EnforceDistribution::new(); + let optimized = optimizer.optimize(optimized, &config)?; + // The rule should be idempotent. + // Re-running this rule shouldn't introduce unnecessary operators. + let optimizer = EnforceDistribution::new(); + let optimized = optimizer.optimize(optimized, &config)?; + // Run the enforce sorting rule: + let optimizer = EnforceSorting::new(); + let optimized = optimizer.optimize(optimized, &config)?; + optimized + } else { + // Run the enforce sorting rule first: + let optimizer = EnforceSorting::new(); + let optimized = optimizer.optimize(optimized, &config)?; + // Run enforce distribution rule: + let optimizer = EnforceDistribution::new(); + let optimized = optimizer.optimize(optimized, &config)?; + // The rule should be idempotent. + // Re-running this rule shouldn't introduce unnecessary operators. + let optimizer = EnforceDistribution::new(); + let optimized = optimizer.optimize(optimized, &config)?; + optimized + }; + + // Remove the ancillary output requirements operator when done: + let optimizer = OutputRequirements::new_remove_mode(); + let optimized = optimizer.optimize(optimized, &config)?; + + // Now format correctly + let plan = displayable(optimized.as_ref()).indent(true).to_string(); + let actual_lines = trim_plan_display(&plan); + + assert_eq!( + &expected_lines, &actual_lines, + "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", + expected_lines, actual_lines + ); + }; +} + +macro_rules! assert_plan_txt { + ($EXPECTED_LINES: expr, $PLAN: expr) => { + let expected_lines: Vec<&str> = $EXPECTED_LINES.iter().map(|s| *s).collect(); + // Now format correctly + let plan = displayable($PLAN.as_ref()).indent(true).to_string(); + let actual_lines = trim_plan_display(&plan); + + assert_eq!( + &expected_lines, &actual_lines, + "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", + expected_lines, actual_lines + ); + }; +} + +#[test] +fn multi_hash_joins() -> Result<()> { + let left = parquet_exec(); + let alias_pairs: Vec<(String, String)> = vec![ + ("a".to_string(), "a1".to_string()), + ("b".to_string(), "b1".to_string()), + ("c".to_string(), "c1".to_string()), + ("d".to_string(), "d1".to_string()), + ("e".to_string(), "e1".to_string()), + ]; + let right = projection_exec_with_alias(parquet_exec(), alias_pairs); + let join_types = vec![ + JoinType::Inner, + JoinType::Left, + JoinType::Right, + JoinType::Full, + JoinType::LeftSemi, + JoinType::LeftAnti, + JoinType::LeftMark, + JoinType::RightSemi, + JoinType::RightAnti, + ]; + + // Join on (a == b1) + let join_on = vec![( + Arc::new(Column::new_with_schema("a", &schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("b1", &right.schema()).unwrap()) as _, + )]; + + for join_type in join_types { + let join = hash_join_exec(left.clone(), right.clone(), &join_on, &join_type); + let join_plan = format!( + "HashJoinExec: mode=Partitioned, join_type={join_type}, on=[(a@0, b1@1)]" + ); + + match join_type { + JoinType::Inner + | JoinType::Left + | JoinType::Right + | JoinType::Full + | JoinType::LeftSemi + | JoinType::LeftAnti + | JoinType::LeftMark => { + // Join on (a == c) + let top_join_on = vec![( + Arc::new(Column::new_with_schema("a", &join.schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("c", &schema()).unwrap()) as _, + )]; + let top_join = hash_join_exec( + join.clone(), + parquet_exec(), + &top_join_on, + &join_type, + ); + let top_join_plan = + format!("HashJoinExec: mode=Partitioned, join_type={join_type}, on=[(a@0, c@2)]"); + + let expected = match join_type { + // Should include 3 RepartitionExecs + JoinType::Inner | JoinType::Left | JoinType::LeftSemi | JoinType::LeftAnti | JoinType::LeftMark => vec![ + 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", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", + "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", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ], + // Should include 4 RepartitionExecs + _ => vec![ + top_join_plan.as_str(), + "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", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", + "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", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ], + }; + assert_optimized!(expected, top_join.clone(), true); + assert_optimized!(expected, top_join, false); + } + JoinType::RightSemi | JoinType::RightAnti => {} + } + + match join_type { + JoinType::Inner + | JoinType::Left + | JoinType::Right + | JoinType::Full + | JoinType::RightSemi + | JoinType::RightAnti => { + // This time we use (b1 == c) for top join + // Join on (b1 == c) + let top_join_on = vec![( + Arc::new(Column::new_with_schema("b1", &join.schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("c", &schema()).unwrap()) as _, + )]; + + let top_join = + hash_join_exec(join, parquet_exec(), &top_join_on, &join_type); + let top_join_plan = match join_type { + JoinType::RightSemi | JoinType::RightAnti => + format!("HashJoinExec: mode=Partitioned, join_type={join_type}, on=[(b1@1, c@2)]"), + _ => + format!("HashJoinExec: mode=Partitioned, join_type={join_type}, on=[(b1@6, c@2)]"), + }; + + let expected = match join_type { + // Should include 3 RepartitionExecs + JoinType::Inner | JoinType::Right | JoinType::RightSemi | JoinType::RightAnti => + vec![ + 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", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", + "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", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ], + // Should include 4 RepartitionExecs + _ => + vec![ + top_join_plan.as_str(), + "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", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", + "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", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ], + }; + assert_optimized!(expected, top_join.clone(), true); + assert_optimized!(expected, top_join, false); + } + JoinType::LeftSemi | JoinType::LeftAnti | JoinType::LeftMark => {} + } + } + + Ok(()) +} + +#[test] +fn multi_joins_after_alias() -> Result<()> { + let left = parquet_exec(); + let right = parquet_exec(); + + // Join on (a == b) + let join_on = vec![( + Arc::new(Column::new_with_schema("a", &schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("b", &schema()).unwrap()) as _, + )]; + let join = hash_join_exec(left, right.clone(), &join_on, &JoinType::Inner); + + // Projection(a as a1, a as a2) + let alias_pairs: Vec<(String, String)> = vec![ + ("a".to_string(), "a1".to_string()), + ("a".to_string(), "a2".to_string()), + ]; + let projection = projection_exec_with_alias(join, alias_pairs); + + // Join on (a1 == c) + let top_join_on = vec![( + Arc::new(Column::new_with_schema("a1", &projection.schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("c", &schema()).unwrap()) as _, + )]; + + let top_join = hash_join_exec( + projection.clone(), + right.clone(), + &top_join_on, + &JoinType::Inner, + ); + + // Output partition need to respect the Alias and should not introduce additional RepartitionExec + let expected = &[ + "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a1@0, c@2)]", + "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", + "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", + "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", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + assert_optimized!(expected, top_join.clone(), true); + assert_optimized!(expected, top_join, false); + + // Join on (a2 == c) + let top_join_on = vec![( + Arc::new(Column::new_with_schema("a2", &projection.schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("c", &schema()).unwrap()) as _, + )]; + + let top_join = hash_join_exec(projection, right, &top_join_on, &JoinType::Inner); + + // Output partition need to respect the Alias and should not introduce additional RepartitionExec + let expected = &[ + "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a2@1, c@2)]", + "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", + "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", + "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", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + assert_optimized!(expected, top_join.clone(), true); + assert_optimized!(expected, top_join, false); + + Ok(()) +} + +#[test] +fn multi_joins_after_multi_alias() -> Result<()> { + let left = parquet_exec(); + let right = parquet_exec(); + + // Join on (a == b) + let join_on = vec![( + Arc::new(Column::new_with_schema("a", &schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("b", &schema()).unwrap()) as _, + )]; + + let join = hash_join_exec(left, right.clone(), &join_on, &JoinType::Inner); + + // Projection(c as c1) + let alias_pairs: Vec<(String, String)> = vec![("c".to_string(), "c1".to_string())]; + let projection = projection_exec_with_alias(join, alias_pairs); + + // Projection(c1 as a) + let alias_pairs: Vec<(String, String)> = vec![("c1".to_string(), "a".to_string())]; + let projection2 = projection_exec_with_alias(projection, alias_pairs); + + // Join on (a == c) + let top_join_on = vec![( + Arc::new(Column::new_with_schema("a", &projection2.schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("c", &schema()).unwrap()) as _, + )]; + + let top_join = hash_join_exec(projection2, right, &top_join_on, &JoinType::Inner); + + // The Column 'a' has different meaning now after the two Projections + // The original Output partition can not satisfy the Join requirements and need to add an additional RepartitionExec + let expected = &[ + "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, c@2)]", + "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", + "ProjectionExec: expr=[c1@0 as a]", + "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", + "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", + "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", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + + assert_optimized!(expected, top_join.clone(), true); + assert_optimized!(expected, top_join, false); + + Ok(()) +} + +#[test] +fn join_after_agg_alias() -> Result<()> { + // group by (a as a1) + let left = aggregate_exec_with_alias( + parquet_exec(), + vec![("a".to_string(), "a1".to_string())], + ); + // group by (a as a2) + let right = aggregate_exec_with_alias( + parquet_exec(), + vec![("a".to_string(), "a2".to_string())], + ); + + // Join on (a1 == a2) + let join_on = vec![( + Arc::new(Column::new_with_schema("a1", &left.schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("a2", &right.schema()).unwrap()) as _, + )]; + let join = hash_join_exec(left, right.clone(), &join_on, &JoinType::Inner); + + // Only two RepartitionExecs added + let expected = &[ + "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a1@0, a2@0)]", + "AggregateExec: mode=FinalPartitioned, gby=[a1@0 as a1], aggr=[]", + "RepartitionExec: partitioning=Hash([a1@0], 10), input_partitions=10", + "AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[]", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "AggregateExec: mode=FinalPartitioned, gby=[a2@0 as a2], aggr=[]", + "RepartitionExec: partitioning=Hash([a2@0], 10), input_partitions=10", + "AggregateExec: mode=Partial, gby=[a@0 as a2], aggr=[]", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + assert_optimized!(expected, join.clone(), true); + assert_optimized!(expected, join, false); + + Ok(()) +} + +#[test] +fn hash_join_key_ordering() -> Result<()> { + // group by (a as a1, b as b1) + let left = aggregate_exec_with_alias( + parquet_exec(), + vec![ + ("a".to_string(), "a1".to_string()), + ("b".to_string(), "b1".to_string()), + ], + ); + // group by (b, a) + let right = aggregate_exec_with_alias( + parquet_exec(), + vec![ + ("b".to_string(), "b".to_string()), + ("a".to_string(), "a".to_string()), + ], + ); + + // Join on (b1 == b && a1 == a) + let join_on = vec![ + ( + Arc::new(Column::new_with_schema("b1", &left.schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("b", &right.schema()).unwrap()) as _, + ), + ( + Arc::new(Column::new_with_schema("a1", &left.schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("a", &right.schema()).unwrap()) as _, + ), + ]; + let join = hash_join_exec(left, right.clone(), &join_on, &JoinType::Inner); + + // Only two RepartitionExecs added + let expected = &[ + "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(b1@1, b@0), (a1@0, a@1)]", + "ProjectionExec: expr=[a1@1 as a1, b1@0 as b1]", + "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", + "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", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + assert_optimized!(expected, join.clone(), true); + assert_optimized!(expected, join, false); + + Ok(()) +} + +#[test] +fn multi_hash_join_key_ordering() -> Result<()> { + let left = parquet_exec(); + let alias_pairs: Vec<(String, String)> = vec![ + ("a".to_string(), "a1".to_string()), + ("b".to_string(), "b1".to_string()), + ("c".to_string(), "c1".to_string()), + ]; + let right = projection_exec_with_alias(parquet_exec(), alias_pairs); + + // Join on (a == a1 and b == b1 and c == c1) + let join_on = vec![ + ( + Arc::new(Column::new_with_schema("a", &schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("a1", &right.schema()).unwrap()) as _, + ), + ( + Arc::new(Column::new_with_schema("b", &schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("b1", &right.schema()).unwrap()) as _, + ), + ( + Arc::new(Column::new_with_schema("c", &schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("c1", &right.schema()).unwrap()) as _, + ), + ]; + let bottom_left_join = + hash_join_exec(left.clone(), right.clone(), &join_on, &JoinType::Inner); + + // Projection(a as A, a as AA, b as B, c as C) + let alias_pairs: Vec<(String, String)> = vec![ + ("a".to_string(), "A".to_string()), + ("a".to_string(), "AA".to_string()), + ("b".to_string(), "B".to_string()), + ("c".to_string(), "C".to_string()), + ]; + let bottom_left_projection = + projection_exec_with_alias(bottom_left_join, alias_pairs); + + // Join on (c == c1 and b == b1 and a == a1) + let join_on = vec![ + ( + Arc::new(Column::new_with_schema("c", &schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("c1", &right.schema()).unwrap()) as _, + ), + ( + Arc::new(Column::new_with_schema("b", &schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("b1", &right.schema()).unwrap()) as _, + ), + ( + Arc::new(Column::new_with_schema("a", &schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("a1", &right.schema()).unwrap()) as _, + ), + ]; + let bottom_right_join = + hash_join_exec(left, right.clone(), &join_on, &JoinType::Inner); + + // Join on (B == b1 and C == c and AA = a1) + let top_join_on = vec![ + ( + Arc::new( + Column::new_with_schema("B", &bottom_left_projection.schema()).unwrap(), + ) as _, + Arc::new(Column::new_with_schema("b1", &bottom_right_join.schema()).unwrap()) + as _, + ), + ( + Arc::new( + Column::new_with_schema("C", &bottom_left_projection.schema()).unwrap(), + ) as _, + Arc::new(Column::new_with_schema("c", &bottom_right_join.schema()).unwrap()) + as _, + ), + ( + Arc::new( + Column::new_with_schema("AA", &bottom_left_projection.schema()).unwrap(), + ) as _, + Arc::new(Column::new_with_schema("a1", &bottom_right_join.schema()).unwrap()) + as _, + ), + ]; + + let top_join = hash_join_exec( + bottom_left_projection.clone(), + bottom_right_join, + &top_join_on, + &JoinType::Inner, + ); + + let predicate: Arc = binary( + col("c", top_join.schema().deref())?, + Operator::Gt, + lit(1i64), + top_join.schema().deref(), + )?; + + let filter_top_join: Arc = + Arc::new(FilterExec::try_new(predicate, top_join)?); + + // The bottom joins' join key ordering is adjusted based on the top join. And the top join should not introduce additional RepartitionExec + let expected = &[ + "FilterExec: c@6 > 1", + "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(B@2, b1@6), (C@3, c@2), (AA@1, a1@5)]", + "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", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "RepartitionExec: partitioning=Hash([b1@1, c1@2, a1@0], 10), input_partitions=10", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1]", + "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", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "RepartitionExec: partitioning=Hash([b1@1, c1@2, a1@0], 10), input_partitions=10", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + assert_optimized!(expected, filter_top_join.clone(), true); + assert_optimized!(expected, filter_top_join, false); + + Ok(()) +} + +#[test] +fn reorder_join_keys_to_left_input() -> Result<()> { + let left = parquet_exec(); + let alias_pairs: Vec<(String, String)> = vec![ + ("a".to_string(), "a1".to_string()), + ("b".to_string(), "b1".to_string()), + ("c".to_string(), "c1".to_string()), + ]; + let right = projection_exec_with_alias(parquet_exec(), alias_pairs); + + // Join on (a == a1 and b == b1 and c == c1) + let join_on = vec![ + ( + Arc::new(Column::new_with_schema("a", &schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("a1", &right.schema()).unwrap()) as _, + ), + ( + Arc::new(Column::new_with_schema("b", &schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("b1", &right.schema()).unwrap()) as _, + ), + ( + Arc::new(Column::new_with_schema("c", &schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("c1", &right.schema()).unwrap()) as _, + ), + ]; + + let bottom_left_join = ensure_distribution_helper( + hash_join_exec(left.clone(), right.clone(), &join_on, &JoinType::Inner), + 10, + true, + )?; + + // Projection(a as A, a as AA, b as B, c as C) + let alias_pairs: Vec<(String, String)> = vec![ + ("a".to_string(), "A".to_string()), + ("a".to_string(), "AA".to_string()), + ("b".to_string(), "B".to_string()), + ("c".to_string(), "C".to_string()), + ]; + let bottom_left_projection = + projection_exec_with_alias(bottom_left_join, alias_pairs); + + // Join on (c == c1 and b == b1 and a == a1) + let join_on = vec![ + ( + Arc::new(Column::new_with_schema("c", &schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("c1", &right.schema()).unwrap()) as _, + ), + ( + Arc::new(Column::new_with_schema("b", &schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("b1", &right.schema()).unwrap()) as _, + ), + ( + Arc::new(Column::new_with_schema("a", &schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("a1", &right.schema()).unwrap()) as _, + ), + ]; + let bottom_right_join = ensure_distribution_helper( + hash_join_exec(left, right.clone(), &join_on, &JoinType::Inner), + 10, + true, + )?; + + // Join on (B == b1 and C == c and AA = a1) + let top_join_on = vec![ + ( + Arc::new( + Column::new_with_schema("B", &bottom_left_projection.schema()).unwrap(), + ) as _, + Arc::new(Column::new_with_schema("b1", &bottom_right_join.schema()).unwrap()) + as _, + ), + ( + Arc::new( + Column::new_with_schema("C", &bottom_left_projection.schema()).unwrap(), + ) as _, + Arc::new(Column::new_with_schema("c", &bottom_right_join.schema()).unwrap()) + as _, + ), + ( + Arc::new( + Column::new_with_schema("AA", &bottom_left_projection.schema()).unwrap(), + ) as _, + Arc::new(Column::new_with_schema("a1", &bottom_right_join.schema()).unwrap()) + as _, + ), + ]; + + let join_types = vec![ + JoinType::Inner, + JoinType::Left, + JoinType::Right, + JoinType::Full, + JoinType::LeftSemi, + JoinType::LeftAnti, + JoinType::RightSemi, + JoinType::RightAnti, + ]; + + for join_type in join_types { + let top_join = hash_join_exec( + bottom_left_projection.clone(), + bottom_right_join.clone(), + &top_join_on, + &join_type, + ); + let top_join_plan = + format!("HashJoinExec: mode=Partitioned, join_type={:?}, on=[(AA@1, a1@5), (B@2, b1@6), (C@3, c@2)]", &join_type); + + let reordered = reorder_join_keys_to_inputs(top_join)?; + + // The top joins' join key ordering is adjusted based on the children inputs. + let expected = &[ + top_join_plan.as_str(), + "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", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "RepartitionExec: partitioning=Hash([a1@0, b1@1, c1@2], 10), input_partitions=10", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1]", + "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", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "RepartitionExec: partitioning=Hash([c1@2, b1@1, a1@0], 10), input_partitions=10", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + + assert_plan_txt!(expected, reordered); + } + + Ok(()) +} + +#[test] +fn reorder_join_keys_to_right_input() -> Result<()> { + let left = parquet_exec(); + let alias_pairs: Vec<(String, String)> = vec![ + ("a".to_string(), "a1".to_string()), + ("b".to_string(), "b1".to_string()), + ("c".to_string(), "c1".to_string()), + ]; + let right = projection_exec_with_alias(parquet_exec(), alias_pairs); + + // Join on (a == a1 and b == b1) + let join_on = vec![ + ( + Arc::new(Column::new_with_schema("a", &schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("a1", &right.schema()).unwrap()) as _, + ), + ( + Arc::new(Column::new_with_schema("b", &schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("b1", &right.schema()).unwrap()) as _, + ), + ]; + let bottom_left_join = ensure_distribution_helper( + hash_join_exec(left.clone(), right.clone(), &join_on, &JoinType::Inner), + 10, + true, + )?; + + // Projection(a as A, a as AA, b as B, c as C) + let alias_pairs: Vec<(String, String)> = vec![ + ("a".to_string(), "A".to_string()), + ("a".to_string(), "AA".to_string()), + ("b".to_string(), "B".to_string()), + ("c".to_string(), "C".to_string()), + ]; + let bottom_left_projection = + projection_exec_with_alias(bottom_left_join, alias_pairs); + + // Join on (c == c1 and b == b1 and a == a1) + let join_on = vec![ + ( + Arc::new(Column::new_with_schema("c", &schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("c1", &right.schema()).unwrap()) as _, + ), + ( + Arc::new(Column::new_with_schema("b", &schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("b1", &right.schema()).unwrap()) as _, + ), + ( + Arc::new(Column::new_with_schema("a", &schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("a1", &right.schema()).unwrap()) as _, + ), + ]; + let bottom_right_join = ensure_distribution_helper( + hash_join_exec(left, right.clone(), &join_on, &JoinType::Inner), + 10, + true, + )?; + + // Join on (B == b1 and C == c and AA = a1) + let top_join_on = vec![ + ( + Arc::new( + Column::new_with_schema("B", &bottom_left_projection.schema()).unwrap(), + ) as _, + Arc::new(Column::new_with_schema("b1", &bottom_right_join.schema()).unwrap()) + as _, + ), + ( + Arc::new( + Column::new_with_schema("C", &bottom_left_projection.schema()).unwrap(), + ) as _, + Arc::new(Column::new_with_schema("c", &bottom_right_join.schema()).unwrap()) + as _, + ), + ( + Arc::new( + Column::new_with_schema("AA", &bottom_left_projection.schema()).unwrap(), + ) as _, + Arc::new(Column::new_with_schema("a1", &bottom_right_join.schema()).unwrap()) + as _, + ), + ]; + + let join_types = vec![ + JoinType::Inner, + JoinType::Left, + JoinType::Right, + JoinType::Full, + JoinType::LeftSemi, + JoinType::LeftAnti, + JoinType::RightSemi, + JoinType::RightAnti, + ]; + + for join_type in join_types { + let top_join = hash_join_exec( + bottom_left_projection.clone(), + bottom_right_join.clone(), + &top_join_on, + &join_type, + ); + let top_join_plan = + format!("HashJoinExec: mode=Partitioned, join_type={:?}, on=[(C@3, c@2), (B@2, b1@6), (AA@1, a1@5)]", &join_type); + + let reordered = reorder_join_keys_to_inputs(top_join)?; + + // The top joins' join key ordering is adjusted based on the children inputs. + let expected = &[ + top_join_plan.as_str(), + "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", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "RepartitionExec: partitioning=Hash([a1@0, b1@1], 10), input_partitions=10", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1]", + "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", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "RepartitionExec: partitioning=Hash([c1@2, b1@1, a1@0], 10), input_partitions=10", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + + assert_plan_txt!(expected, reordered); + } + + Ok(()) +} + +#[test] +fn multi_smj_joins() -> Result<()> { + let left = parquet_exec(); + let alias_pairs: Vec<(String, String)> = vec![ + ("a".to_string(), "a1".to_string()), + ("b".to_string(), "b1".to_string()), + ("c".to_string(), "c1".to_string()), + ("d".to_string(), "d1".to_string()), + ("e".to_string(), "e1".to_string()), + ]; + let right = projection_exec_with_alias(parquet_exec(), alias_pairs); + + // SortMergeJoin does not support RightSemi and RightAnti join now + let join_types = vec![ + JoinType::Inner, + JoinType::Left, + JoinType::Right, + JoinType::Full, + JoinType::LeftSemi, + JoinType::LeftAnti, + ]; + + // Join on (a == b1) + let join_on = vec![( + Arc::new(Column::new_with_schema("a", &schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("b1", &right.schema()).unwrap()) as _, + )]; + + for join_type in join_types { + let join = + sort_merge_join_exec(left.clone(), right.clone(), &join_on, &join_type); + let join_plan = format!("SortMergeJoin: join_type={join_type}, on=[(a@0, b1@1)]"); + + // Top join on (a == c) + let top_join_on = vec![( + Arc::new(Column::new_with_schema("a", &join.schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("c", &schema()).unwrap()) as _, + )]; + let top_join = + sort_merge_join_exec(join.clone(), parquet_exec(), &top_join_on, &join_type); + let top_join_plan = + format!("SortMergeJoin: join_type={join_type}, on=[(a@0, c@2)]"); + + let expected = match join_type { + // Should include 6 RepartitionExecs (3 hash, 3 round-robin), 3 SortExecs + JoinType::Inner | JoinType::Left | JoinType::LeftSemi | JoinType::LeftAnti => + vec![ + top_join_plan.as_str(), + 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", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "SortExec: expr=[b1@1 ASC], preserve_partitioning=[true]", + "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", + "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", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ], + // Should include 7 RepartitionExecs (4 hash, 3 round-robin), 4 SortExecs + // Since ordering of the left child is not preserved after SortMergeJoin + // when mode is Right, RightSemi, RightAnti, Full + // - We need to add one additional SortExec after SortMergeJoin in contrast the test cases + // when mode is Inner, Left, LeftSemi, LeftAnti + // Similarly, since partitioning of the left side is not preserved + // when mode is Right, RightSemi, RightAnti, Full + // - We need to add one additional Hash Repartition after SortMergeJoin in contrast the test + // cases when mode is Inner, Left, LeftSemi, LeftAnti + _ => vec![ + top_join_plan.as_str(), + // Below 2 operators are differences introduced, when join mode is changed + "SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", + "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", + 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", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "SortExec: expr=[b1@1 ASC], preserve_partitioning=[true]", + "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", + "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", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ], + }; + assert_optimized!(expected, top_join.clone(), true, true); + + let expected_first_sort_enforcement = match join_type { + // Should include 6 RepartitionExecs (3 hash, 3 round-robin), 3 SortExecs + JoinType::Inner | JoinType::Left | JoinType::LeftSemi | JoinType::LeftAnti => + vec![ + 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", + "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", + "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", + "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ], + // Should include 8 RepartitionExecs (4 hash, 8 round-robin), 4 SortExecs + // Since ordering of the left child is not preserved after SortMergeJoin + // when mode is Right, RightSemi, RightAnti, Full + // - We need to add one additional SortExec after SortMergeJoin in contrast the test cases + // when mode is Inner, Left, LeftSemi, LeftAnti + // Similarly, since partitioning of the left side is not preserved + // when mode is Right, RightSemi, RightAnti, Full + // - We need to add one additional Hash Repartition and Roundrobin repartition after + // SortMergeJoin in contrast the test cases when mode is Inner, Left, LeftSemi, LeftAnti + _ => vec![ + 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", + "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", + "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", + "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", + "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ], + }; + assert_optimized!(expected_first_sort_enforcement, top_join, false, true); + + match join_type { + JoinType::Inner | JoinType::Left | JoinType::Right | JoinType::Full => { + // This time we use (b1 == c) for top join + // Join on (b1 == c) + let top_join_on = vec![( + Arc::new(Column::new_with_schema("b1", &join.schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("c", &schema()).unwrap()) as _, + )]; + let top_join = + sort_merge_join_exec(join, parquet_exec(), &top_join_on, &join_type); + let top_join_plan = + format!("SortMergeJoin: join_type={join_type}, on=[(b1@6, c@2)]"); + + let expected = match join_type { + // Should include 6 RepartitionExecs(3 hash, 3 round-robin) and 3 SortExecs + JoinType::Inner | JoinType::Right => vec![ + top_join_plan.as_str(), + 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", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "SortExec: expr=[b1@1 ASC], preserve_partitioning=[true]", + "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", + "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", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ], + // Should include 7 RepartitionExecs (4 hash, 3 round-robin) and 4 SortExecs + JoinType::Left | JoinType::Full => vec![ + top_join_plan.as_str(), + "SortExec: expr=[b1@6 ASC], preserve_partitioning=[true]", + "RepartitionExec: partitioning=Hash([b1@6], 10), input_partitions=10", + 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", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "SortExec: expr=[b1@1 ASC], preserve_partitioning=[true]", + "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", + "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", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ], + // this match arm cannot be reached + _ => unreachable!() + }; + assert_optimized!(expected, top_join.clone(), true, true); + + let expected_first_sort_enforcement = match join_type { + // Should include 6 RepartitionExecs (3 of them preserves order) and 3 SortExecs + JoinType::Inner | JoinType::Right => vec![ + 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", + "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", + "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", + "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ], + // Should include 8 RepartitionExecs (4 of them preserves order) and 4 SortExecs + JoinType::Left | JoinType::Full => vec![ + 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", + "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", + "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", + "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", + "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ], + // this match arm cannot be reached + _ => unreachable!() + }; + assert_optimized!(expected_first_sort_enforcement, top_join, false, true); + } + _ => {} + } + } + + Ok(()) +} + +#[test] +fn smj_join_key_ordering() -> Result<()> { + // group by (a as a1, b as b1) + let left = aggregate_exec_with_alias( + parquet_exec(), + vec![ + ("a".to_string(), "a1".to_string()), + ("b".to_string(), "b1".to_string()), + ], + ); + //Projection(a1 as a3, b1 as b3) + let alias_pairs: Vec<(String, String)> = vec![ + ("a1".to_string(), "a3".to_string()), + ("b1".to_string(), "b3".to_string()), + ]; + let left = projection_exec_with_alias(left, alias_pairs); + + // group by (b, a) + let right = aggregate_exec_with_alias( + parquet_exec(), + vec![ + ("b".to_string(), "b".to_string()), + ("a".to_string(), "a".to_string()), + ], + ); + + //Projection(a as a2, b as b2) + let alias_pairs: Vec<(String, String)> = vec![ + ("a".to_string(), "a2".to_string()), + ("b".to_string(), "b2".to_string()), + ]; + let right = projection_exec_with_alias(right, alias_pairs); + + // Join on (b3 == b2 && a3 == a2) + let join_on = vec![ + ( + Arc::new(Column::new_with_schema("b3", &left.schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("b2", &right.schema()).unwrap()) as _, + ), + ( + Arc::new(Column::new_with_schema("a3", &left.schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("a2", &right.schema()).unwrap()) as _, + ), + ]; + let join = sort_merge_join_exec(left, right.clone(), &join_on, &JoinType::Inner); + + // Only two RepartitionExecs added + let expected = &[ + "SortMergeJoin: join_type=Inner, on=[(b3@1, b2@1), (a3@0, a2@0)]", + "SortExec: expr=[b3@1 ASC, a3@0 ASC], preserve_partitioning=[true]", + "ProjectionExec: expr=[a1@0 as a3, b1@1 as b3]", + "ProjectionExec: expr=[a1@1 as a1, b1@0 as b1]", + "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", + "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", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + assert_optimized!(expected, join.clone(), true, true); + + 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", + "SortExec: expr=[b3@1 ASC, a3@0 ASC], preserve_partitioning=[false]", + "CoalescePartitionsExec", + "ProjectionExec: expr=[a1@0 as a3, b1@1 as b3]", + "ProjectionExec: expr=[a1@1 as a1, b1@0 as b1]", + "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", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "RepartitionExec: partitioning=Hash([b2@1, a2@0], 10), input_partitions=10, preserve_order=true, sort_exprs=b2@1 ASC, a2@0 ASC", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "SortExec: expr=[b2@1 ASC, a2@0 ASC], preserve_partitioning=[false]", + "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", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + assert_optimized!(expected_first_sort_enforcement, join, false, true); + + Ok(()) +} + +#[test] +fn merge_does_not_need_sort() -> Result<()> { + // see https://github.com/apache/datafusion/issues/4331 + let schema = schema(); + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { + expr: col("a", &schema).unwrap(), + options: SortOptions::default(), + }]); + + // Scan some sorted parquet files + let exec = parquet_exec_multiple_sorted(vec![sort_key.clone()]); + + // CoalesceBatchesExec to mimic behavior after a filter + let exec = Arc::new(CoalesceBatchesExec::new(exec, 4096)); + + // Merge from multiple parquet files and keep the data sorted + let exec: Arc = + Arc::new(SortPreservingMergeExec::new(sort_key, exec)); + + // The optimizer should not add an additional SortExec as the + // data is already sorted + let expected = &[ + "SortPreservingMergeExec: [a@0 ASC]", + "CoalesceBatchesExec: target_batch_size=4096", + "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", + ]; + assert_optimized!(expected, exec, true); + + // In this case preserving ordering through order preserving operators is not desirable + // (according to flag: PREFER_EXISTING_SORT) + // hence in this case ordering lost during CoalescePartitionsExec and re-introduced with + // SortExec at the top. + let expected = &[ + "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", + "CoalescePartitionsExec", + "CoalesceBatchesExec: target_batch_size=4096", + "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", + ]; + assert_optimized!(expected, exec, false); + + Ok(()) +} + +#[test] +fn union_to_interleave() -> Result<()> { + // group by (a as a1) + let left = aggregate_exec_with_alias( + parquet_exec(), + vec![("a".to_string(), "a1".to_string())], + ); + // group by (a as a2) + let right = aggregate_exec_with_alias( + parquet_exec(), + vec![("a".to_string(), "a1".to_string())], + ); + + // Union + let plan = Arc::new(UnionExec::new(vec![left, right])); + + // final agg + let plan = + aggregate_exec_with_alias(plan, vec![("a1".to_string(), "a2".to_string())]); + + // Only two RepartitionExecs added, no final RepartitionExec required + let expected = &[ + "AggregateExec: mode=FinalPartitioned, gby=[a2@0 as a2], aggr=[]", + "AggregateExec: mode=Partial, gby=[a1@0 as a2], aggr=[]", + "InterleaveExec", + "AggregateExec: mode=FinalPartitioned, gby=[a1@0 as a1], aggr=[]", + "RepartitionExec: partitioning=Hash([a1@0], 10), input_partitions=10", + "AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[]", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "AggregateExec: mode=FinalPartitioned, gby=[a1@0 as a1], aggr=[]", + "RepartitionExec: partitioning=Hash([a1@0], 10), input_partitions=10", + "AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[]", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + assert_optimized!(expected, plan.clone(), true); + assert_optimized!(expected, plan.clone(), false); + + Ok(()) +} + +#[test] +fn union_not_to_interleave() -> Result<()> { + // group by (a as a1) + let left = aggregate_exec_with_alias( + parquet_exec(), + vec![("a".to_string(), "a1".to_string())], + ); + // group by (a as a2) + let right = aggregate_exec_with_alias( + parquet_exec(), + vec![("a".to_string(), "a1".to_string())], + ); + + // Union + let plan = Arc::new(UnionExec::new(vec![left, right])); + + // final agg + let plan = + aggregate_exec_with_alias(plan, vec![("a1".to_string(), "a2".to_string())]); + + // Only two RepartitionExecs added, no final RepartitionExec required + let expected = &[ + "AggregateExec: mode=FinalPartitioned, gby=[a2@0 as a2], aggr=[]", + "RepartitionExec: partitioning=Hash([a2@0], 10), input_partitions=20", + "AggregateExec: mode=Partial, gby=[a1@0 as a2], aggr=[]", + "UnionExec", + "AggregateExec: mode=FinalPartitioned, gby=[a1@0 as a1], aggr=[]", + "RepartitionExec: partitioning=Hash([a1@0], 10), input_partitions=10", + "AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[]", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "AggregateExec: mode=FinalPartitioned, gby=[a1@0 as a1], aggr=[]", + "RepartitionExec: partitioning=Hash([a1@0], 10), input_partitions=10", + "AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[]", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + // no sort in the plan but since we need it as a parameter, make it default false + let prefer_existing_sort = false; + let first_enforce_distribution = true; + let prefer_existing_union = true; + + assert_optimized!( + expected, + plan.clone(), + first_enforce_distribution, + prefer_existing_sort, + prefer_existing_union + ); + assert_optimized!( + expected, + plan, + !first_enforce_distribution, + prefer_existing_sort, + prefer_existing_union + ); + + Ok(()) +} + +#[test] +fn added_repartition_to_single_partition() -> Result<()> { + let alias = vec![("a".to_string(), "a".to_string())]; + let plan = aggregate_exec_with_alias(parquet_exec(), alias); + + let expected = [ + "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", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + assert_optimized!(expected, plan.clone(), true); + assert_optimized!(expected, plan, false); + + Ok(()) +} + +#[test] +fn repartition_deepest_node() -> Result<()> { + let alias = vec![("a".to_string(), "a".to_string())]; + let plan = aggregate_exec_with_alias(filter_exec(parquet_exec()), alias); + + let expected = &[ + "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=[]", + "FilterExec: c@2 = 0", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + assert_optimized!(expected, plan.clone(), true); + assert_optimized!(expected, plan, false); + + Ok(()) +} + +#[test] + +fn repartition_unsorted_limit() -> Result<()> { + let plan = limit_exec(filter_exec(parquet_exec())); + + let expected = &[ + "GlobalLimitExec: skip=0, fetch=100", + "CoalescePartitionsExec", + "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", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + + assert_optimized!(expected, plan.clone(), true); + assert_optimized!(expected, plan, false); + + Ok(()) +} + +#[test] +fn repartition_sorted_limit() -> Result<()> { + let schema = schema(); + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { + expr: col("c", &schema).unwrap(), + options: SortOptions::default(), + }]); + let plan = limit_exec(sort_exec(sort_key, parquet_exec(), false)); + + let expected = &[ + "GlobalLimitExec: skip=0, fetch=100", + "LocalLimitExec: fetch=100", + // data is sorted so can't repartition here + "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + assert_optimized!(expected, plan.clone(), true); + assert_optimized!(expected, plan, false); + + Ok(()) +} + +#[test] +fn repartition_sorted_limit_with_filter() -> Result<()> { + let schema = schema(); + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { + expr: col("c", &schema).unwrap(), + options: SortOptions::default(), + }]); + let plan = sort_required_exec_with_req( + filter_exec(sort_exec(sort_key.clone(), parquet_exec(), false)), + sort_key, + ); + + let expected = &[ + "SortRequiredExec: [c@2 ASC]", + "FilterExec: c@2 = 0", + // We can use repartition here, ordering requirement by SortRequiredExec + // is still satisfied. + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + + assert_optimized!(expected, plan.clone(), true); + assert_optimized!(expected, plan, false); + + Ok(()) +} + +#[test] +fn repartition_ignores_limit() -> Result<()> { + let alias = vec![("a".to_string(), "a".to_string())]; + let plan = aggregate_exec_with_alias( + limit_exec(filter_exec(limit_exec(parquet_exec()))), + alias, + ); + + let expected = &[ + "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", + "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", + "GlobalLimitExec: skip=0, fetch=100", + "LocalLimitExec: fetch=100", + // Expect no repartition to happen for local limit + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + assert_optimized!(expected, plan.clone(), true); + assert_optimized!(expected, plan, false); + + Ok(()) +} + +#[test] +fn repartition_ignores_union() -> Result<()> { + let plan = union_exec(vec![parquet_exec(); 5]); + + let expected = &[ + "UnionExec", + // Expect no repartition of ParquetExec + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + + assert_optimized!(expected, plan.clone(), true); + assert_optimized!(expected, plan, false); + + Ok(()) +} + +#[test] +fn repartition_through_sort_preserving_merge() -> Result<()> { + // sort preserving merge with non-sorted input + let schema = schema(); + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { + expr: col("c", &schema).unwrap(), + options: SortOptions::default(), + }]); + let plan = sort_preserving_merge_exec(sort_key, parquet_exec()); + + // need resort as the data was not sorted correctly + let expected = &[ + "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + assert_optimized!(expected, plan.clone(), true); + assert_optimized!(expected, plan, false); + + Ok(()) +} + +#[test] +fn repartition_ignores_sort_preserving_merge() -> Result<()> { + // sort preserving merge already sorted input, + let schema = schema(); + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { + expr: col("c", &schema).unwrap(), + options: SortOptions::default(), + }]); + let plan = sort_preserving_merge_exec( + sort_key.clone(), + parquet_exec_multiple_sorted(vec![sort_key]), + ); + + // should not sort (as the data was already sorted) + // should not repartition, since increased parallelism is not beneficial for SortPReservingMerge + let expected = &[ + "SortPreservingMergeExec: [c@2 ASC]", + "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + ]; + + assert_optimized!(expected, plan.clone(), true); + + let expected = &[ + "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", + "CoalescePartitionsExec", + "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + ]; + assert_optimized!(expected, plan, false); + + Ok(()) +} + +#[test] +fn repartition_ignores_sort_preserving_merge_with_union() -> Result<()> { + // 2 sorted parquet files unioned (partitions are concatenated, sort is preserved) + let schema = schema(); + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { + expr: col("c", &schema).unwrap(), + options: SortOptions::default(), + }]); + let input = union_exec(vec![parquet_exec_with_sort(vec![sort_key.clone()]); 2]); + let plan = sort_preserving_merge_exec(sort_key, input); + + // should not repartition / sort (as the data was already sorted) + let expected = &[ + "SortPreservingMergeExec: [c@2 ASC]", + "UnionExec", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + ]; + + assert_optimized!(expected, plan.clone(), true); + + let expected = &[ + "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", + "CoalescePartitionsExec", + "UnionExec", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + ]; + assert_optimized!(expected, plan, false); + + Ok(()) +} + +#[test] +fn repartition_does_not_destroy_sort() -> Result<()> { + // SortRequired + // Parquet(sorted) + let schema = schema(); + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { + expr: col("d", &schema).unwrap(), + options: SortOptions::default(), + }]); + let plan = sort_required_exec_with_req( + filter_exec(parquet_exec_with_sort(vec![sort_key.clone()])), + sort_key, + ); + + // during repartitioning ordering is preserved + let expected = &[ + "SortRequiredExec: [d@3 ASC]", + "FilterExec: c@2 = 0", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[d@3 ASC]", + ]; + + assert_optimized!(expected, plan.clone(), true, true); + assert_optimized!(expected, plan, false, true); + + Ok(()) +} + +#[test] +fn repartition_does_not_destroy_sort_more_complex() -> Result<()> { + // model a more complicated scenario where one child of a union can be repartitioned for performance + // but the other can not be + // + // Union + // SortRequired + // Parquet(sorted) + // Filter + // Parquet(unsorted) + + let schema = schema(); + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { + expr: col("c", &schema).unwrap(), + options: SortOptions::default(), + }]); + let input1 = sort_required_exec_with_req( + parquet_exec_with_sort(vec![sort_key.clone()]), + sort_key, + ); + let input2 = filter_exec(parquet_exec()); + let plan = union_exec(vec![input1, input2]); + + // should not repartition below the SortRequired as that + // branch doesn't benefit from increased parallelism + let expected = &[ + "UnionExec", + // union input 1: no repartitioning + "SortRequiredExec: [c@2 ASC]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + // union input 2: should repartition + "FilterExec: c@2 = 0", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + assert_optimized!(expected, plan.clone(), true); + assert_optimized!(expected, plan, false); + + Ok(()) +} + +#[test] +fn repartition_transitively_with_projection() -> Result<()> { + let schema = schema(); + let proj_exprs = vec![( + Arc::new(BinaryExpr::new( + col("a", &schema).unwrap(), + Operator::Plus, + col("b", &schema).unwrap(), + )) as Arc, + "sum".to_string(), + )]; + // non sorted input + let proj = Arc::new(ProjectionExec::try_new(proj_exprs, parquet_exec())?); + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { + expr: col("sum", &proj.schema()).unwrap(), + options: SortOptions::default(), + }]); + let plan = sort_preserving_merge_exec(sort_key, proj); + + let expected = &[ + "SortPreservingMergeExec: [sum@0 ASC]", + "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", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + + assert_optimized!(expected, plan.clone(), true); + + let expected_first_sort_enforcement = &[ + "SortExec: expr=[sum@0 ASC], preserve_partitioning=[false]", + "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", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + assert_optimized!(expected_first_sort_enforcement, plan, false); + + Ok(()) +} + +#[test] +fn repartition_ignores_transitively_with_projection() -> Result<()> { + let schema = schema(); + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { + expr: col("c", &schema).unwrap(), + options: SortOptions::default(), + }]); + let alias = vec![ + ("a".to_string(), "a".to_string()), + ("b".to_string(), "b".to_string()), + ("c".to_string(), "c".to_string()), + ]; + // sorted input + let plan = sort_required_exec_with_req( + projection_exec_with_alias( + parquet_exec_multiple_sorted(vec![sort_key.clone()]), + alias, + ), + sort_key, + ); + + let expected = &[ + "SortRequiredExec: [c@2 ASC]", + // Since this projection is trivial, increasing parallelism is not beneficial + "ProjectionExec: expr=[a@0 as a, b@1 as b, c@2 as c]", + "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + ]; + assert_optimized!(expected, plan.clone(), true); + assert_optimized!(expected, plan, false); + + Ok(()) +} + +#[test] +fn repartition_transitively_past_sort_with_projection() -> Result<()> { + let schema = schema(); + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { + expr: col("c", &schema).unwrap(), + options: SortOptions::default(), + }]); + let alias = vec![ + ("a".to_string(), "a".to_string()), + ("b".to_string(), "b".to_string()), + ("c".to_string(), "c".to_string()), + ]; + let plan = sort_preserving_merge_exec( + sort_key.clone(), + sort_exec( + sort_key, + projection_exec_with_alias(parquet_exec(), alias), + true, + ), + ); + + let expected = &[ + "SortExec: expr=[c@2 ASC], preserve_partitioning=[true]", + // Since this projection is trivial, increasing parallelism is not beneficial + "ProjectionExec: expr=[a@0 as a, b@1 as b, c@2 as c]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + assert_optimized!(expected, plan.clone(), true); + assert_optimized!(expected, plan, false); + + Ok(()) +} + +#[test] +fn repartition_transitively_past_sort_with_filter() -> Result<()> { + let schema = schema(); + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { + expr: col("a", &schema).unwrap(), + options: SortOptions::default(), + }]); + let plan = sort_exec(sort_key, filter_exec(parquet_exec()), false); + + let expected = &[ + "SortPreservingMergeExec: [a@0 ASC]", + "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", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + + assert_optimized!(expected, plan.clone(), true); + + let expected_first_sort_enforcement = &[ + "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", + "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", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + assert_optimized!(expected_first_sort_enforcement, plan, false); + + Ok(()) +} + +#[test] +#[cfg(feature = "parquet")] +fn repartition_transitively_past_sort_with_projection_and_filter() -> Result<()> { + let schema = schema(); + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { + expr: col("a", &schema).unwrap(), + options: SortOptions::default(), + }]); + let plan = sort_exec( + sort_key, + projection_exec_with_alias( + filter_exec(parquet_exec()), + vec![ + ("a".to_string(), "a".to_string()), + ("b".to_string(), "b".to_string()), + ("c".to_string(), "c".to_string()), + ], + ), + false, + ); + + let expected = &[ + "SortPreservingMergeExec: [a@0 ASC]", + // Expect repartition on the input to the sort (as it can benefit from additional parallelism) + "SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", + "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", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + + assert_optimized!(expected, plan.clone(), true); + + let expected_first_sort_enforcement = &[ + "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", + "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", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + assert_optimized!(expected_first_sort_enforcement, plan, false); + + Ok(()) +} + +#[test] +fn parallelization_single_partition() -> Result<()> { + let alias = vec![("a".to_string(), "a".to_string())]; + let plan_parquet = aggregate_exec_with_alias(parquet_exec(), alias.clone()); + let plan_csv = aggregate_exec_with_alias(csv_exec(), alias); + + let expected_parquet = [ + "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", + "RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2", + "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", + "ParquetExec: file_groups={2 groups: [[x:0..50], [x:50..100]]}, projection=[a, b, c, d, e]", + ]; + let expected_csv = [ + "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", + "RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2", + "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", + "CsvExec: file_groups={2 groups: [[x:0..50], [x:50..100]]}, projection=[a, b, c, d, e], has_header=false", + ]; + assert_optimized!(expected_parquet, plan_parquet, true, false, 2, true, 10); + assert_optimized!(expected_csv, plan_csv, true, false, 2, true, 10); + + Ok(()) +} + +#[test] +fn parallelization_multiple_files() -> Result<()> { + let schema = schema(); + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { + expr: col("a", &schema).unwrap(), + options: SortOptions::default(), + }]); + + let plan = filter_exec(parquet_exec_multiple_sorted(vec![sort_key.clone()])); + let plan = sort_required_exec_with_req(plan, sort_key); + + // The groups must have only contiguous ranges of rows from the same file + // if any group has rows from multiple files, the data is no longer sorted destroyed + // https://github.com/apache/datafusion/issues/8451 + let expected = [ + "SortRequiredExec: [a@0 ASC]", + "FilterExec: c@2 = 0", + "ParquetExec: file_groups={3 groups: [[x:0..50], [y:0..100], [x:50..100]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", ]; + let target_partitions = 3; + let repartition_size = 1; + assert_optimized!( + expected, + plan, + true, + true, + target_partitions, + true, + repartition_size, + false + ); + + let expected = [ + "SortRequiredExec: [a@0 ASC]", + "FilterExec: c@2 = 0", + "ParquetExec: file_groups={8 groups: [[x:0..25], [y:0..25], [x:25..50], [y:25..50], [x:50..75], [y:50..75], [x:75..100], [y:75..100]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", + ]; + let target_partitions = 8; + let repartition_size = 1; + assert_optimized!( + expected, + plan, + true, + true, + target_partitions, + true, + repartition_size, + false + ); + + Ok(()) +} + +#[test] +/// CsvExec on compressed csv file will not be partitioned +/// (Not able to decompress chunked csv file) +fn parallelization_compressed_csv() -> Result<()> { + let compression_types = [ + FileCompressionType::GZIP, + FileCompressionType::BZIP2, + FileCompressionType::XZ, + FileCompressionType::ZSTD, + FileCompressionType::UNCOMPRESSED, + ]; + + let expected_not_partitioned = [ + "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", + "RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2", + "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", + "RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", + "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + ]; + + let expected_partitioned = [ + "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", + "RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2", + "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", + "CsvExec: file_groups={2 groups: [[x:0..50], [x:50..100]]}, projection=[a, b, c, d, e], has_header=false", + ]; + + for compression_type in compression_types { + let expected = if compression_type.is_compressed() { + &expected_not_partitioned[..] + } else { + &expected_partitioned[..] + }; + + let plan = aggregate_exec_with_alias( + Arc::new( + CsvExec::builder( + FileScanConfig::new( + ObjectStoreUrl::parse("test:///").unwrap(), + schema(), + ) + .with_file(PartitionedFile::new("x".to_string(), 100)), + ) + .with_has_header(false) + .with_delimeter(b',') + .with_quote(b'"') + .with_escape(None) + .with_comment(None) + .with_newlines_in_values(false) + .with_file_compression_type(compression_type) + .build(), + ), + vec![("a".to_string(), "a".to_string())], + ); + assert_optimized!(expected, plan, true, false, 2, true, 10, false); + } + Ok(()) +} + +#[test] +fn parallelization_two_partitions() -> Result<()> { + let alias = vec![("a".to_string(), "a".to_string())]; + let plan_parquet = aggregate_exec_with_alias(parquet_exec_multiple(), alias.clone()); + let plan_csv = aggregate_exec_with_alias(csv_exec_multiple(), alias); + + let expected_parquet = [ + "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", + "RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2", + "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", + // Plan already has two partitions + "ParquetExec: file_groups={2 groups: [[x:0..100], [y:0..100]]}, projection=[a, b, c, d, e]", + ]; + let expected_csv = [ + "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", + "RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2", + "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", + // Plan already has two partitions + "CsvExec: file_groups={2 groups: [[x:0..100], [y:0..100]]}, projection=[a, b, c, d, e], has_header=false", + ]; + assert_optimized!(expected_parquet, plan_parquet, true, false, 2, true, 10); + assert_optimized!(expected_csv, plan_csv, true, false, 2, true, 10); + Ok(()) +} + +#[test] +fn parallelization_two_partitions_into_four() -> Result<()> { + let alias = vec![("a".to_string(), "a".to_string())]; + let plan_parquet = aggregate_exec_with_alias(parquet_exec_multiple(), alias.clone()); + let plan_csv = aggregate_exec_with_alias(csv_exec_multiple(), alias); + + let expected_parquet = [ + "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", + "RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4", + "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", + // Multiple source files splitted across partitions + "ParquetExec: file_groups={4 groups: [[x:0..50], [x:50..100], [y:0..50], [y:50..100]]}, projection=[a, b, c, d, e]", + ]; + let expected_csv = [ + "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", + "RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4", + "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", + // Multiple source files splitted across partitions + "CsvExec: file_groups={4 groups: [[x:0..50], [x:50..100], [y:0..50], [y:50..100]]}, projection=[a, b, c, d, e], has_header=false", + ]; + assert_optimized!(expected_parquet, plan_parquet, true, false, 4, true, 10); + assert_optimized!(expected_csv, plan_csv, true, false, 4, true, 10); + + Ok(()) +} + +#[test] +fn parallelization_sorted_limit() -> Result<()> { + let schema = schema(); + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { + expr: col("c", &schema).unwrap(), + options: SortOptions::default(), + }]); + let plan_parquet = limit_exec(sort_exec(sort_key.clone(), parquet_exec(), false)); + let plan_csv = limit_exec(sort_exec(sort_key, csv_exec(), false)); + + let expected_parquet = &[ + "GlobalLimitExec: skip=0, fetch=100", + "LocalLimitExec: fetch=100", + // data is sorted so can't repartition here + "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", + // Doesn't parallelize for SortExec without preserve_partitioning + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + let expected_csv = &[ + "GlobalLimitExec: skip=0, fetch=100", + "LocalLimitExec: fetch=100", + // data is sorted so can't repartition here + "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", + // Doesn't parallelize for SortExec without preserve_partitioning + "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + ]; + assert_optimized!(expected_parquet, plan_parquet, true); + assert_optimized!(expected_csv, plan_csv, true); + + Ok(()) +} + +#[test] +fn parallelization_limit_with_filter() -> Result<()> { + let schema = schema(); + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { + expr: col("c", &schema).unwrap(), + options: SortOptions::default(), + }]); + let plan_parquet = limit_exec(filter_exec(sort_exec( + sort_key.clone(), + parquet_exec(), + false, + ))); + let plan_csv = limit_exec(filter_exec(sort_exec(sort_key, csv_exec(), false))); + + let expected_parquet = &[ + "GlobalLimitExec: skip=0, fetch=100", + "CoalescePartitionsExec", + "LocalLimitExec: fetch=100", + "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", + "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]", + ]; + let expected_csv = &[ + "GlobalLimitExec: skip=0, fetch=100", + "CoalescePartitionsExec", + "LocalLimitExec: fetch=100", + "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", + "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", + ]; + assert_optimized!(expected_parquet, plan_parquet, true); + assert_optimized!(expected_csv, plan_csv, true); + + Ok(()) +} + +#[test] +fn parallelization_ignores_limit() -> Result<()> { + let alias = vec![("a".to_string(), "a".to_string())]; + let plan_parquet = aggregate_exec_with_alias( + limit_exec(filter_exec(limit_exec(parquet_exec()))), + alias.clone(), + ); + let plan_csv = + aggregate_exec_with_alias(limit_exec(filter_exec(limit_exec(csv_exec()))), alias); + + let expected_parquet = &[ + "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", + "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", + "GlobalLimitExec: skip=0, fetch=100", + // Limit doesn't benefit from input partitioning - no parallelism + "LocalLimitExec: fetch=100", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + let expected_csv = &[ + "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", + "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", + "GlobalLimitExec: skip=0, fetch=100", + // Limit doesn't benefit from input partitioning - no parallelism + "LocalLimitExec: fetch=100", + "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + ]; + assert_optimized!(expected_parquet, plan_parquet, true); + assert_optimized!(expected_csv, plan_csv, true); + + Ok(()) +} + +#[test] +fn parallelization_union_inputs() -> Result<()> { + let plan_parquet = union_exec(vec![parquet_exec(); 5]); + let plan_csv = union_exec(vec![csv_exec(); 5]); + + let expected_parquet = &[ + "UnionExec", + // Union doesn't benefit from input partitioning - no parallelism + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + let expected_csv = &[ + "UnionExec", + // Union doesn't benefit from input partitioning - no parallelism + "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + ]; + assert_optimized!(expected_parquet, plan_parquet, true); + assert_optimized!(expected_csv, plan_csv, true); + + Ok(()) +} + +#[test] +fn parallelization_prior_to_sort_preserving_merge() -> Result<()> { + let schema = schema(); + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { + expr: col("c", &schema).unwrap(), + options: SortOptions::default(), + }]); + // sort preserving merge already sorted input, + let plan_parquet = sort_preserving_merge_exec( + sort_key.clone(), + parquet_exec_with_sort(vec![sort_key.clone()]), + ); + let plan_csv = + sort_preserving_merge_exec(sort_key.clone(), csv_exec_with_sort(vec![sort_key])); + + // parallelization is not beneficial for SortPreservingMerge + let expected_parquet = &[ + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + ]; + let expected_csv = &[ + "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], has_header=false", + ]; + assert_optimized!(expected_parquet, plan_parquet, true); + assert_optimized!(expected_csv, plan_csv, true); + + Ok(()) +} + +#[test] +fn parallelization_sort_preserving_merge_with_union() -> Result<()> { + let schema = schema(); + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { + expr: col("c", &schema).unwrap(), + options: SortOptions::default(), + }]); + // 2 sorted parquet files unioned (partitions are concatenated, sort is preserved) + let input_parquet = + union_exec(vec![parquet_exec_with_sort(vec![sort_key.clone()]); 2]); + let input_csv = union_exec(vec![csv_exec_with_sort(vec![sort_key.clone()]); 2]); + let plan_parquet = sort_preserving_merge_exec(sort_key.clone(), input_parquet); + let plan_csv = sort_preserving_merge_exec(sort_key, input_csv); + + // should not repartition (union doesn't benefit from increased parallelism) + // should not sort (as the data was already sorted) + let expected_parquet = &[ + "SortPreservingMergeExec: [c@2 ASC]", + "UnionExec", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + ]; + let expected_csv = &[ + "SortPreservingMergeExec: [c@2 ASC]", + "UnionExec", + "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], has_header=false", + "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], has_header=false", + ]; + assert_optimized!(expected_parquet, plan_parquet, true); + assert_optimized!(expected_csv, plan_csv, true); + + Ok(()) +} + +#[test] +fn parallelization_does_not_benefit() -> Result<()> { + let schema = schema(); + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { + expr: col("c", &schema).unwrap(), + options: SortOptions::default(), + }]); + // SortRequired + // Parquet(sorted) + let plan_parquet = sort_required_exec_with_req( + parquet_exec_with_sort(vec![sort_key.clone()]), + sort_key.clone(), + ); + let plan_csv = + sort_required_exec_with_req(csv_exec_with_sort(vec![sort_key.clone()]), sort_key); + + // no parallelization, because SortRequiredExec doesn't benefit from increased parallelism + let expected_parquet = &[ + "SortRequiredExec: [c@2 ASC]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + ]; + let expected_csv = &[ + "SortRequiredExec: [c@2 ASC]", + "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], has_header=false", + ]; + assert_optimized!(expected_parquet, plan_parquet, true); + assert_optimized!(expected_csv, plan_csv, true); + + Ok(()) +} + +#[test] +fn parallelization_ignores_transitively_with_projection_parquet() -> Result<()> { + // sorted input + let schema = schema(); + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { + expr: col("c", &schema).unwrap(), + options: SortOptions::default(), + }]); + + //Projection(a as a2, b as b2) + let alias_pairs: Vec<(String, String)> = vec![ + ("a".to_string(), "a2".to_string()), + ("c".to_string(), "c2".to_string()), + ]; + let proj_parquet = + projection_exec_with_alias(parquet_exec_with_sort(vec![sort_key]), alias_pairs); + let sort_key_after_projection = LexOrdering::new(vec![PhysicalSortExpr { + expr: col("c2", &proj_parquet.schema()).unwrap(), + options: SortOptions::default(), + }]); + let plan_parquet = + sort_preserving_merge_exec(sort_key_after_projection, proj_parquet); + let expected = &[ + "SortPreservingMergeExec: [c2@1 ASC]", + " ProjectionExec: expr=[a@0 as a2, c@2 as c2]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + ]; + plans_matches_expected!(expected, &plan_parquet); + + // data should not be repartitioned / resorted + let expected_parquet = &[ + "ProjectionExec: expr=[a@0 as a2, c@2 as c2]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + ]; + assert_optimized!(expected_parquet, plan_parquet, true); + + Ok(()) +} + +#[test] +fn parallelization_ignores_transitively_with_projection_csv() -> Result<()> { + // sorted input + let schema = schema(); + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { + expr: col("c", &schema).unwrap(), + options: SortOptions::default(), + }]); + + //Projection(a as a2, b as b2) + let alias_pairs: Vec<(String, String)> = vec![ + ("a".to_string(), "a2".to_string()), + ("c".to_string(), "c2".to_string()), + ]; + + let proj_csv = + projection_exec_with_alias(csv_exec_with_sort(vec![sort_key]), alias_pairs); + let sort_key_after_projection = LexOrdering::new(vec![PhysicalSortExpr { + expr: col("c2", &proj_csv.schema()).unwrap(), + options: SortOptions::default(), + }]); + let plan_csv = sort_preserving_merge_exec(sort_key_after_projection, proj_csv); + let expected = &[ + "SortPreservingMergeExec: [c2@1 ASC]", + " ProjectionExec: expr=[a@0 as a2, c@2 as c2]", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], has_header=false", + ]; + plans_matches_expected!(expected, &plan_csv); + + // data should not be repartitioned / resorted + let expected_csv = &[ + "ProjectionExec: expr=[a@0 as a2, c@2 as c2]", + "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], has_header=false", + ]; + assert_optimized!(expected_csv, plan_csv, true); + + Ok(()) +} + +#[test] +fn remove_redundant_roundrobins() -> Result<()> { + let input = parquet_exec(); + let repartition = repartition_exec(repartition_exec(input)); + let physical_plan = repartition_exec(filter_exec(repartition)); + let expected = &[ + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10", + " FilterExec: c@2 = 0", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + plans_matches_expected!(expected, &physical_plan); + + let expected = &[ + "FilterExec: c@2 = 0", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + assert_optimized!(expected, physical_plan.clone(), true); + assert_optimized!(expected, physical_plan, false); + + Ok(()) +} + +#[test] +fn remove_unnecessary_spm_after_filter() -> Result<()> { + let schema = schema(); + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { + expr: col("c", &schema).unwrap(), + options: SortOptions::default(), + }]); + let input = parquet_exec_multiple_sorted(vec![sort_key.clone()]); + let physical_plan = sort_preserving_merge_exec(sort_key, filter_exec(input)); + + // Original plan expects its output to be ordered by c@2 ASC. + // This is still satisfied since, after filter that column is constant. + let expected = &[ + "CoalescePartitionsExec", + "FilterExec: c@2 = 0", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2, preserve_order=true, sort_exprs=c@2 ASC", + "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + ]; + // last flag sets config.optimizer.PREFER_EXISTING_SORT + assert_optimized!(expected, physical_plan.clone(), true, true); + assert_optimized!(expected, physical_plan, false, true); + + Ok(()) +} + +#[test] +fn preserve_ordering_through_repartition() -> Result<()> { + let schema = schema(); + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { + expr: col("d", &schema).unwrap(), + options: SortOptions::default(), + }]); + let input = parquet_exec_multiple_sorted(vec![sort_key.clone()]); + let physical_plan = sort_preserving_merge_exec(sort_key, filter_exec(input)); + + 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", + "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[d@3 ASC]", + ]; + // last flag sets config.optimizer.PREFER_EXISTING_SORT + assert_optimized!(expected, physical_plan.clone(), true, true); + assert_optimized!(expected, physical_plan, false, true); + + Ok(()) +} + +#[test] +fn do_not_preserve_ordering_through_repartition() -> Result<()> { + let schema = schema(); + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { + expr: col("a", &schema).unwrap(), + options: SortOptions::default(), + }]); + let input = parquet_exec_multiple_sorted(vec![sort_key.clone()]); + let physical_plan = sort_preserving_merge_exec(sort_key, filter_exec(input)); + + let expected = &[ + "SortPreservingMergeExec: [a@0 ASC]", + "SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", + "FilterExec: c@2 = 0", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", + "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", + ]; + + assert_optimized!(expected, physical_plan.clone(), true); + + let expected = &[ + "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", + "CoalescePartitionsExec", + "FilterExec: c@2 = 0", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", + "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", + ]; + assert_optimized!(expected, physical_plan, false); + + Ok(()) +} + +#[test] +fn no_need_for_sort_after_filter() -> Result<()> { + let schema = schema(); + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { + expr: col("c", &schema).unwrap(), + options: SortOptions::default(), + }]); + let input = parquet_exec_multiple_sorted(vec![sort_key.clone()]); + let physical_plan = sort_preserving_merge_exec(sort_key, filter_exec(input)); + + let expected = &[ + // After CoalescePartitionsExec c is still constant. Hence c@2 ASC ordering is already satisfied. + "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", + "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + ]; + assert_optimized!(expected, physical_plan.clone(), true); + assert_optimized!(expected, physical_plan, false); + + Ok(()) +} + +#[test] +fn do_not_preserve_ordering_through_repartition2() -> Result<()> { + let schema = schema(); + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { + expr: col("c", &schema).unwrap(), + options: SortOptions::default(), + }]); + let input = parquet_exec_multiple_sorted(vec![sort_key]); + + let sort_req = LexOrdering::new(vec![PhysicalSortExpr { + expr: col("a", &schema).unwrap(), + options: SortOptions::default(), + }]); + let physical_plan = sort_preserving_merge_exec(sort_req, filter_exec(input)); + + let expected = &[ + "SortPreservingMergeExec: [a@0 ASC]", + "SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", + "FilterExec: c@2 = 0", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", + "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + ]; + + assert_optimized!(expected, physical_plan.clone(), true); + + let expected = &[ + "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", + "CoalescePartitionsExec", + "SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", + "FilterExec: c@2 = 0", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", + "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + ]; + assert_optimized!(expected, physical_plan, false); + + Ok(()) +} + +#[test] +fn do_not_preserve_ordering_through_repartition3() -> Result<()> { + let schema = schema(); + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { + expr: col("c", &schema).unwrap(), + options: SortOptions::default(), + }]); + let input = parquet_exec_multiple_sorted(vec![sort_key]); + let physical_plan = filter_exec(input); + + let expected = &[ + "FilterExec: c@2 = 0", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", + "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + ]; + assert_optimized!(expected, physical_plan.clone(), true); + assert_optimized!(expected, physical_plan, false); + + Ok(()) +} + +#[test] +fn do_not_put_sort_when_input_is_invalid() -> Result<()> { + let schema = schema(); + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { + expr: col("a", &schema).unwrap(), + options: SortOptions::default(), + }]); + let input = parquet_exec(); + let physical_plan = sort_required_exec_with_req(filter_exec(input), sort_key); + let expected = &[ + // Ordering requirement of sort required exec is NOT satisfied + // by existing ordering at the source. + "SortRequiredExec: [a@0 ASC]", + "FilterExec: c@2 = 0", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + assert_plan_txt!(expected, physical_plan); + + let expected = &[ + "SortRequiredExec: [a@0 ASC]", + // 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", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + + let mut config = ConfigOptions::new(); + config.execution.target_partitions = 10; + config.optimizer.enable_round_robin_repartition = true; + config.optimizer.prefer_existing_sort = false; + let dist_plan = EnforceDistribution::new().optimize(physical_plan, &config)?; + assert_plan_txt!(expected, dist_plan); + + Ok(()) +} + +#[test] +fn put_sort_when_input_is_valid() -> Result<()> { + let schema = schema(); + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { + expr: col("a", &schema).unwrap(), + options: SortOptions::default(), + }]); + let input = parquet_exec_multiple_sorted(vec![sort_key.clone()]); + let physical_plan = sort_required_exec_with_req(filter_exec(input), sort_key); + + let expected = &[ + // Ordering requirement of sort required exec is satisfied + // by existing ordering at the source. + "SortRequiredExec: [a@0 ASC]", + "FilterExec: c@2 = 0", + "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", + ]; + assert_plan_txt!(expected, physical_plan); + + let expected = &[ + // Since at the start of the rule ordering requirement is satisfied + // EnforceDistribution rule satisfy this requirement also. + "SortRequiredExec: [a@0 ASC]", + "FilterExec: c@2 = 0", + "ParquetExec: file_groups={10 groups: [[x:0..20], [y:0..20], [x:20..40], [y:20..40], [x:40..60], [y:40..60], [x:60..80], [y:60..80], [x:80..100], [y:80..100]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", + ]; + + let mut config = ConfigOptions::new(); + config.execution.target_partitions = 10; + config.optimizer.enable_round_robin_repartition = true; + config.optimizer.prefer_existing_sort = false; + let dist_plan = EnforceDistribution::new().optimize(physical_plan, &config)?; + assert_plan_txt!(expected, dist_plan); + + Ok(()) +} + +#[test] +fn do_not_add_unnecessary_hash() -> Result<()> { + let schema = schema(); + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { + expr: col("c", &schema).unwrap(), + options: SortOptions::default(), + }]); + let alias = vec![("a".to_string(), "a".to_string())]; + let input = parquet_exec_with_sort(vec![sort_key]); + let physical_plan = aggregate_exec_with_alias(input, alias); + + let expected = &[ + "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", + "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + ]; + // Make sure target partition number is 1. In this case hash repartition is unnecessary + assert_optimized!(expected, physical_plan.clone(), true, false, 1, false, 1024); + assert_optimized!(expected, physical_plan, false, false, 1, false, 1024); + + Ok(()) +} + +#[test] +fn do_not_add_unnecessary_hash2() -> Result<()> { + let schema = schema(); + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { + expr: col("c", &schema).unwrap(), + options: SortOptions::default(), + }]); + let alias = vec![("a".to_string(), "a".to_string())]; + let input = parquet_exec_multiple_sorted(vec![sort_key]); + let aggregate = aggregate_exec_with_alias(input, alias.clone()); + let physical_plan = aggregate_exec_with_alias(aggregate, alias); + + let expected = &[ + "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", + // Since hash requirements of this operator is satisfied. There shouldn't be + // a hash repartition here + "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", + "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", + "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + ]; + // Make sure target partition number is larger than 2 (e.g partition number at the source). + assert_optimized!(expected, physical_plan.clone(), true, false, 4, false, 1024); + assert_optimized!(expected, physical_plan, false, false, 4, false, 1024); + + Ok(()) +} + +#[test] +fn optimize_away_unnecessary_repartition() -> Result<()> { + let physical_plan = coalesce_partitions_exec(repartition_exec(parquet_exec())); + let expected = &[ + "CoalescePartitionsExec", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + plans_matches_expected!(expected, physical_plan.clone()); + + let expected = + &["ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]"]; + assert_optimized!(expected, physical_plan.clone(), true); + assert_optimized!(expected, physical_plan, false); + + Ok(()) +} + +#[test] +fn optimize_away_unnecessary_repartition2() -> Result<()> { + let physical_plan = filter_exec(repartition_exec(coalesce_partitions_exec( + filter_exec(repartition_exec(parquet_exec())), + ))); + let expected = &[ + "FilterExec: c@2 = 0", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " CoalescePartitionsExec", + " FilterExec: c@2 = 0", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + plans_matches_expected!(expected, physical_plan.clone()); + + let expected = &[ + "FilterExec: c@2 = 0", + "FilterExec: c@2 = 0", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + assert_optimized!(expected, physical_plan.clone(), true); + assert_optimized!(expected, physical_plan, false); + + Ok(()) +} diff --git a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs index 21dbe06d7ac2..4fe04753fd9c 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs @@ -15,66 +15,103 @@ // specific language governing permissions and limitations // under the License. -use datafusion::prelude::{SessionConfig, SessionContext}; -use datafusion_physical_optimizer::test_utils::{ - aggregate_exec, bounded_window_exec, check_integrity, coalesce_batches_exec, - coalesce_partitions_exec, filter_exec, global_limit_exec, hash_join_exec, limit_exec, - local_limit_exec, memory_exec, repartition_exec, sort_exec, sort_expr, - sort_expr_options, sort_merge_join_exec, sort_preserving_merge_exec, - spr_repartition_exec, union_exec, RequirementsTestExec, -}; -use datafusion_physical_plan::displayable; use std::sync::Arc; +use crate::physical_optimizer::parquet_exec; + +use datafusion_physical_plan::displayable; use arrow::compute::SortOptions; -use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; +use arrow_schema::SchemaRef; use datafusion_common::Result; use datafusion_expr::JoinType; use datafusion_physical_expr::expressions::{col, Column, NotExpr}; use datafusion_physical_optimizer::PhysicalOptimizerRule; -use datafusion_physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; - -use crate::assert_optimized; -use crate::physical_optimizer::test_util::{ - csv_exec_ordered, csv_exec_sorted, parquet_exec, parquet_exec_sorted, stream_exec_ordered, -}; -use datafusion::physical_optimizer::enforce_distribution::EnforceDistribution; use datafusion_physical_expr::Partitioning; -use datafusion_physical_expr_common::sort_expr::LexOrdering; +use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; use datafusion_physical_optimizer::enforce_sorting::{EnforceSorting,PlanWithCorrespondingCoalescePartitions,PlanWithCorrespondingSort,parallelize_sorts,ensure_sorting}; use datafusion_physical_optimizer::enforce_sorting::replace_with_order_preserving_variants::{replace_with_order_preserving_variants,OrderPreservationContext}; use datafusion_physical_optimizer::enforce_sorting::sort_pushdown::{SortPushDown, assign_initial_requirements, pushdown_sorts}; use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; use datafusion_physical_plan::repartition::RepartitionExec; -use datafusion_physical_plan::sorts::sort::SortExec; use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; use datafusion_physical_plan::{get_plan_string, ExecutionPlan}; +use datafusion_common::config::ConfigOptions; +use datafusion_common::tree_node::{TreeNode, TransformedResult}; +use datafusion_physical_optimizer::test_utils::{check_integrity,bounded_window_exec, coalesce_partitions_exec, create_test_schema, create_test_schema2, create_test_schema3, filter_exec, global_limit_exec, hash_join_exec, limit_exec, local_limit_exec, memory_exec, repartition_exec, sort_exec, sort_expr, sort_expr_options, sort_merge_join_exec, sort_preserving_merge_exec, spr_repartition_exec, stream_exec_ordered, union_exec}; +use datafusion::datasource::physical_plan::{CsvExec, FileScanConfig, ParquetExec}; +use datafusion_execution::object_store::ObjectStoreUrl; +use datafusion::datasource::listing::PartitionedFile; +use datafusion::datasource::file_format::file_compression_type::FileCompressionType; + use rstest::rstest; -use datafusion_common::tree_node::{TreeNode,TransformedResult}; -fn create_test_schema() -> Result { - let nullable_column = Field::new("nullable_col", DataType::Int32, true); - let non_nullable_column = Field::new("non_nullable_col", DataType::Int32, false); - let schema = Arc::new(Schema::new(vec![nullable_column, non_nullable_column])); - Ok(schema) +/// Create a csv exec for tests +fn csv_exec_ordered( + schema: &SchemaRef, + sort_exprs: impl IntoIterator, +) -> Arc { + let sort_exprs = sort_exprs.into_iter().collect(); + + Arc::new( + CsvExec::builder( + FileScanConfig::new( + ObjectStoreUrl::parse("test:///").unwrap(), + schema.clone(), + ) + .with_file(PartitionedFile::new("file_path".to_string(), 100)) + .with_output_ordering(vec![sort_exprs]), + ) + .with_has_header(true) + .with_delimeter(0) + .with_quote(b'"') + .with_escape(None) + .with_comment(None) + .with_newlines_in_values(false) + .with_file_compression_type(FileCompressionType::UNCOMPRESSED) + .build(), + ) } -fn create_test_schema2() -> Result { - let col_a = Field::new("col_a", DataType::Int32, true); - let col_b = Field::new("col_b", DataType::Int32, true); - let schema = Arc::new(Schema::new(vec![col_a, col_b])); - Ok(schema) +/// Created a sorted parquet exec +pub fn parquet_exec_sorted( + schema: &SchemaRef, + sort_exprs: impl IntoIterator, +) -> Arc { + let sort_exprs = sort_exprs.into_iter().collect(); + + ParquetExec::builder( + FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema.clone()) + .with_file(PartitionedFile::new("x".to_string(), 100)) + .with_output_ordering(vec![sort_exprs]), + ) + .build_arc() } -// Generate a schema which consists of 5 columns (a, b, c, d, e) -fn create_test_schema3() -> Result { - let a = Field::new("a", DataType::Int32, true); - let b = Field::new("b", DataType::Int32, false); - let c = Field::new("c", DataType::Int32, true); - let d = Field::new("d", DataType::Int32, false); - let e = Field::new("e", DataType::Int32, false); - let schema = Arc::new(Schema::new(vec![a, b, c, d, e])); - Ok(schema) +/// Create a sorted Csv exec +fn csv_exec_sorted( + schema: &SchemaRef, + sort_exprs: impl IntoIterator, +) -> Arc { + let sort_exprs = sort_exprs.into_iter().collect(); + + Arc::new( + CsvExec::builder( + FileScanConfig::new( + ObjectStoreUrl::parse("test:///").unwrap(), + schema.clone(), + ) + .with_file(PartitionedFile::new("x".to_string(), 100)) + .with_output_ordering(vec![sort_exprs]), + ) + .with_has_header(false) + .with_delimeter(0) + .with_quote(0) + .with_escape(None) + .with_comment(None) + .with_newlines_in_values(false) + .with_file_compression_type(FileCompressionType::UNCOMPRESSED) + .build(), + ) } /// Runs the sort enforcement optimizer and asserts the plan @@ -87,9 +124,8 @@ fn create_test_schema3() -> Result { /// macro_rules! assert_optimized { ($EXPECTED_PLAN_LINES: expr, $EXPECTED_OPTIMIZED_PLAN_LINES: expr, $PLAN: expr, $REPARTITION_SORTS: expr) => { - let config = SessionConfig::new().with_repartition_sorts($REPARTITION_SORTS); - let session_ctx = SessionContext::new_with_config(config); - let state = session_ctx.state(); + let mut config = ConfigOptions::new(); + config.optimizer.repartition_sorts = $REPARTITION_SORTS; // This file has 4 rules that use tree node, apply these rules as in the // EnforceSorting::optimize implementation @@ -103,7 +139,7 @@ macro_rules! assert_optimized { .and_then(check_integrity)?; // TODO: End state payloads will be checked here. - let new_plan = if state.config_options().optimizer.repartition_sorts { + let new_plan = if config.optimizer.repartition_sorts { let plan_with_coalesce_partitions = PlanWithCorrespondingCoalescePartitions::new_default(adjusted.plan); let parallel = plan_with_coalesce_partitions @@ -123,7 +159,7 @@ macro_rules! assert_optimized { plan_with_pipeline_fixer, false, true, - state.config_options(), + &config, ) }) .data() @@ -153,7 +189,7 @@ macro_rules! assert_optimized { // Run the actual optimizer let optimized_physical_plan = - EnforceSorting::new().optimize(physical_plan, state.config_options())?; + EnforceSorting::new().optimize(physical_plan,&config)?; // Get string representation of the plan let actual = get_plan_string(&optimized_physical_plan); @@ -165,275 +201,6 @@ macro_rules! assert_optimized { }; } -#[tokio::test] -async fn test_remove_unnecessary_sort() -> Result<()> { - let schema = create_test_schema()?; - let source = memory_exec(&schema); - let input = sort_exec(vec![sort_expr("non_nullable_col", &schema)], source); - let physical_plan = sort_exec(vec![sort_expr("nullable_col", &schema)], input); - - let expected_input = [ - "SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " SortExec: expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", - ]; - let expected_optimized = [ - "SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", - ]; - assert_optimized!(expected_input, expected_optimized, physical_plan, true); - - Ok(()) -} - -#[tokio::test] -async fn test_remove_unnecessary_sort_window_multilayer() -> Result<()> { - let schema = create_test_schema()?; - let source = memory_exec(&schema); - - let sort_exprs = vec![sort_expr_options( - "non_nullable_col", - &source.schema(), - SortOptions { - descending: true, - nulls_first: true, - }, - )]; - let sort = sort_exec(sort_exprs.clone(), source); - // Add dummy layer propagating Sort above, to test whether sort can be removed from multi layer before - let coalesce_batches = coalesce_batches_exec(sort); - - let window_agg = - bounded_window_exec("non_nullable_col", sort_exprs, coalesce_batches); - - let sort_exprs = vec![sort_expr_options( - "non_nullable_col", - &window_agg.schema(), - SortOptions { - descending: false, - nulls_first: false, - }, - )]; - - let sort = sort_exec(sort_exprs.clone(), window_agg); - - // Add dummy layer propagating Sort above, to test whether sort can be removed from multi layer before - let filter = filter_exec( - Arc::new(NotExpr::new( - col("non_nullable_col", schema.as_ref()).unwrap(), - )), - sort, - ); - - let physical_plan = bounded_window_exec("non_nullable_col", sort_exprs, filter); - - 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]", - " FilterExec: NOT non_nullable_col@1", - " SortExec: expr=[non_nullable_col@1 ASC NULLS LAST], preserve_partitioning=[false]", - " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", - " CoalesceBatchesExec: target_batch_size=128", - " SortExec: expr=[non_nullable_col@1 DESC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]"]; - - let expected_optimized = ["WindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(NULL), is_causal: false }]", - " FilterExec: NOT non_nullable_col@1", - " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", - " CoalesceBatchesExec: target_batch_size=128", - " SortExec: expr=[non_nullable_col@1 DESC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]"]; - assert_optimized!(expected_input, expected_optimized, physical_plan, true); - - Ok(()) -} - -#[tokio::test] -async fn test_add_required_sort() -> Result<()> { - let schema = create_test_schema()?; - let source = memory_exec(&schema); - - let sort_exprs = vec![sort_expr("nullable_col", &schema)]; - - let physical_plan = sort_preserving_merge_exec(sort_exprs, source); - - let expected_input = [ - "SortPreservingMergeExec: [nullable_col@0 ASC]", - " MemoryExec: partitions=1, partition_sizes=[0]", - ]; - let expected_optimized = [ - "SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", - ]; - assert_optimized!(expected_input, expected_optimized, physical_plan, true); - - Ok(()) -} - -#[tokio::test] -async fn test_remove_unnecessary_sort1() -> Result<()> { - let schema = create_test_schema()?; - let source = memory_exec(&schema); - let sort_exprs = vec![sort_expr("nullable_col", &schema)]; - let sort = sort_exec(sort_exprs.clone(), source); - let spm = sort_preserving_merge_exec(sort_exprs, sort); - - let sort_exprs = vec![sort_expr("nullable_col", &schema)]; - let sort = sort_exec(sort_exprs.clone(), spm); - let physical_plan = sort_preserving_merge_exec(sort_exprs, sort); - let expected_input = [ - "SortPreservingMergeExec: [nullable_col@0 ASC]", - " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " SortPreservingMergeExec: [nullable_col@0 ASC]", - " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", - ]; - let expected_optimized = [ - "SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", - ]; - assert_optimized!(expected_input, expected_optimized, physical_plan, true); - - Ok(()) -} - -#[tokio::test] -async fn test_remove_unnecessary_sort2() -> Result<()> { - let schema = create_test_schema()?; - let source = memory_exec(&schema); - let sort_exprs = vec![sort_expr("non_nullable_col", &schema)]; - let sort = sort_exec(sort_exprs.clone(), source); - let spm = sort_preserving_merge_exec(sort_exprs, sort); - - let sort_exprs = vec![ - sort_expr("nullable_col", &schema), - sort_expr("non_nullable_col", &schema), - ]; - let sort2 = sort_exec(sort_exprs.clone(), spm); - let spm2 = sort_preserving_merge_exec(sort_exprs, sort2); - - let sort_exprs = vec![sort_expr("nullable_col", &schema)]; - let sort3 = sort_exec(sort_exprs, spm2); - 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", - " 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]", - " SortPreservingMergeExec: [non_nullable_col@1 ASC]", - " SortExec: expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", - ]; - - let expected_optimized = [ - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[0]", - ]; - assert_optimized!(expected_input, expected_optimized, physical_plan, true); - - Ok(()) -} - -#[tokio::test] -async fn test_remove_unnecessary_sort3() -> Result<()> { - let schema = create_test_schema()?; - let source = memory_exec(&schema); - let sort_exprs = vec![sort_expr("non_nullable_col", &schema)]; - let sort = sort_exec(sort_exprs.clone(), source); - let spm = sort_preserving_merge_exec(sort_exprs, sort); - - let sort_exprs = LexOrdering::new(vec![ - sort_expr("nullable_col", &schema), - sort_expr("non_nullable_col", &schema), - ]); - let repartition_exec = repartition_exec(spm); - let sort2 = Arc::new( - SortExec::new(sort_exprs.clone(), repartition_exec) - .with_preserve_partitioning(true), - ) as _; - let spm2 = sort_preserving_merge_exec(sort_exprs, sort2); - - let physical_plan = aggregate_exec(spm2); - - // When removing a `SortPreservingMergeExec`, make sure that partitioning - // requirements are not violated. In some cases, we may need to replace - // it with a `CoalescePartitionsExec` instead of directly removing it. - let expected_input = [ - "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", - " SortPreservingMergeExec: [non_nullable_col@1 ASC]", - " SortExec: expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", - ]; - - let expected_optimized = [ - "AggregateExec: mode=Final, gby=[], aggr=[]", - " CoalescePartitionsExec", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[0]", - ]; - assert_optimized!(expected_input, expected_optimized, physical_plan, true); - - Ok(()) -} - -#[tokio::test] -async fn test_remove_unnecessary_sort4() -> Result<()> { - let schema = create_test_schema()?; - let source1 = repartition_exec(memory_exec(&schema)); - - let source2 = repartition_exec(memory_exec(&schema)); - let union = union_exec(vec![source1, source2]); - - let sort_exprs = LexOrdering::new(vec![sort_expr("non_nullable_col", &schema)]); - // let sort = sort_exec(sort_exprs.clone(), union); - let sort = Arc::new( - SortExec::new(sort_exprs.clone(), union).with_preserve_partitioning(true), - ) as _; - let spm = sort_preserving_merge_exec(sort_exprs, sort); - - let filter = filter_exec( - Arc::new(NotExpr::new( - col("non_nullable_col", schema.as_ref()).unwrap(), - )), - spm, - ); - - let sort_exprs = vec![ - sort_expr("nullable_col", &schema), - sort_expr("non_nullable_col", &schema), - ]; - let physical_plan = sort_exec(sort_exprs, filter); - - // When removing a `SortPreservingMergeExec`, make sure that partitioning - // requirements are not violated. In some cases, we may need to replace - // it with a `CoalescePartitionsExec` instead of directly removing it. - let expected_input = ["SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " FilterExec: NOT non_nullable_col@1", - " SortPreservingMergeExec: [non_nullable_col@1 ASC]", - " SortExec: expr=[non_nullable_col@1 ASC], preserve_partitioning=[true]", - " UnionExec", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[0]", - " RepartitionExec: partitioning=RoundRobinBatch(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", - " MemoryExec: partitions=1, partition_sizes=[0]", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[0]"]; - assert_optimized!(expected_input, expected_optimized, physical_plan, true); - - Ok(()) -} - #[tokio::test] async fn test_remove_unnecessary_sort5() -> Result<()> { let left_schema = create_test_schema2()?; @@ -450,169 +217,13 @@ async fn test_remove_unnecessary_sort5() -> Result<()> { let physical_plan = sort_exec(vec![sort_expr("a", &join.schema())], join); let expected_input = ["SortExec: expr=[a@2 ASC], preserve_partitioning=[false]", - " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(col_a@0, c@2)]", - " MemoryExec: partitions=1, partition_sizes=[0]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]"]; + " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(col_a@0, c@2)]", + " MemoryExec: partitions=1, partition_sizes=[0]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]"]; let expected_optimized = ["HashJoinExec: mode=Partitioned, join_type=Inner, on=[(col_a@0, c@2)]", - " MemoryExec: partitions=1, partition_sizes=[0]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]"]; - assert_optimized!(expected_input, expected_optimized, physical_plan, true); - - Ok(()) -} - -#[tokio::test] -async fn test_remove_unnecessary_sort6() -> Result<()> { - let schema = create_test_schema()?; - let source = memory_exec(&schema); - let input = Arc::new( - SortExec::new( - LexOrdering::new(vec![sort_expr("non_nullable_col", &schema)]), - source, - ) - .with_fetch(Some(2)), - ); - let physical_plan = sort_exec( - vec![ - sort_expr("non_nullable_col", &schema), - sort_expr("nullable_col", &schema), - ], - input, - ); - - let expected_input = [ - "SortExec: expr=[non_nullable_col@1 ASC, nullable_col@0 ASC], preserve_partitioning=[false]", - " SortExec: TopK(fetch=2), expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", - ]; - let expected_optimized = [ - "SortExec: TopK(fetch=2), expr=[non_nullable_col@1 ASC, nullable_col@0 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", - ]; - assert_optimized!(expected_input, expected_optimized, physical_plan, true); - - Ok(()) -} - -#[tokio::test] -async fn test_remove_unnecessary_sort7() -> Result<()> { - let schema = create_test_schema()?; - let source = memory_exec(&schema); - let input = Arc::new(SortExec::new( - LexOrdering::new(vec![ - sort_expr("non_nullable_col", &schema), - sort_expr("nullable_col", &schema), - ]), - source, - )); - - let physical_plan = Arc::new( - SortExec::new( - LexOrdering::new(vec![sort_expr("non_nullable_col", &schema)]), - input, - ) - .with_fetch(Some(2)), - ) as Arc; - - let expected_input = [ - "SortExec: TopK(fetch=2), expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", - " SortExec: expr=[non_nullable_col@1 ASC, nullable_col@0 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", - ]; - let expected_optimized = [ - "GlobalLimitExec: skip=0, fetch=2", - " SortExec: expr=[non_nullable_col@1 ASC, nullable_col@0 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", - ]; - assert_optimized!(expected_input, expected_optimized, physical_plan, true); - - Ok(()) -} - -#[tokio::test] -async fn test_remove_unnecessary_sort8() -> Result<()> { - let schema = create_test_schema()?; - let source = memory_exec(&schema); - let input = Arc::new(SortExec::new( - LexOrdering::new(vec![sort_expr("non_nullable_col", &schema)]), - source, - )); - let limit = Arc::new(LocalLimitExec::new(input, 2)); - let physical_plan = sort_exec( - vec![ - sort_expr("non_nullable_col", &schema), - sort_expr("nullable_col", &schema), - ], - limit, - ); - - let expected_input = [ - "SortExec: expr=[non_nullable_col@1 ASC, nullable_col@0 ASC], preserve_partitioning=[false]", - " LocalLimitExec: fetch=2", - " SortExec: expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", - ]; - let expected_optimized = [ - "LocalLimitExec: fetch=2", - " SortExec: TopK(fetch=2), expr=[non_nullable_col@1 ASC, nullable_col@0 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", - ]; - assert_optimized!(expected_input, expected_optimized, physical_plan, true); - - Ok(()) -} - -#[tokio::test] -async fn test_do_not_pushdown_through_limit() -> Result<()> { - let schema = create_test_schema()?; - let source = memory_exec(&schema); - // let input = sort_exec(vec![sort_expr("non_nullable_col", &schema)], source); - let input = Arc::new(SortExec::new( - LexOrdering::new(vec![sort_expr("non_nullable_col", &schema)]), - source, - )); - let limit = Arc::new(GlobalLimitExec::new(input, 0, Some(5))) as _; - let physical_plan = sort_exec(vec![sort_expr("nullable_col", &schema)], limit); - - let expected_input = [ - "SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " GlobalLimitExec: skip=0, fetch=5", - " SortExec: expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", - ]; - let expected_optimized = [ - "SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " GlobalLimitExec: skip=0, fetch=5", - " SortExec: expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", - ]; - assert_optimized!(expected_input, expected_optimized, physical_plan, true); - - Ok(()) -} - -#[tokio::test] -async fn test_remove_unnecessary_spm1() -> Result<()> { - let schema = create_test_schema()?; - let source = memory_exec(&schema); - let input = - sort_preserving_merge_exec(vec![sort_expr("non_nullable_col", &schema)], source); - let input2 = - sort_preserving_merge_exec(vec![sort_expr("non_nullable_col", &schema)], input); - let physical_plan = - sort_preserving_merge_exec(vec![sort_expr("nullable_col", &schema)], input2); - - let expected_input = [ - "SortPreservingMergeExec: [nullable_col@0 ASC]", - " SortPreservingMergeExec: [non_nullable_col@1 ASC]", - " SortPreservingMergeExec: [non_nullable_col@1 ASC]", - " MemoryExec: partitions=1, partition_sizes=[0]", - ]; - let expected_optimized = [ - "SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", " MemoryExec: partitions=1, partition_sizes=[0]", - ]; + " ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -638,75 +249,24 @@ async fn test_do_not_remove_sort_with_limit() -> Result<()> { 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", - " UnionExec", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", - " GlobalLimitExec: skip=0, fetch=100", - " LocalLimitExec: fetch=100", - " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", + " UnionExec", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", + " GlobalLimitExec: skip=0, fetch=100", + " LocalLimitExec: fetch=100", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; // We should keep the bottom `SortExec`. let expected_optimized = ["SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", - " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[true]", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", - " UnionExec", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", - " GlobalLimitExec: skip=0, fetch=100", - " LocalLimitExec: fetch=100", - " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; - assert_optimized!(expected_input, expected_optimized, physical_plan, true); - - Ok(()) -} - -#[tokio::test] -async fn test_change_wrong_sorting() -> Result<()> { - let schema = create_test_schema()?; - let source = memory_exec(&schema); - let sort_exprs = vec![ - sort_expr("nullable_col", &schema), - sort_expr("non_nullable_col", &schema), - ]; - let sort = sort_exec(vec![sort_exprs[0].clone()], source); - let physical_plan = sort_preserving_merge_exec(sort_exprs, sort); - let expected_input = [ - "SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", - " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", - ]; - let expected_optimized = [ - "SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", - ]; - assert_optimized!(expected_input, expected_optimized, physical_plan, true); - - Ok(()) -} - -#[tokio::test] -async fn test_change_wrong_sorting2() -> Result<()> { - let schema = create_test_schema()?; - let source = memory_exec(&schema); - let sort_exprs = vec![ - sort_expr("nullable_col", &schema), - sort_expr("non_nullable_col", &schema), - ]; - let spm1 = sort_preserving_merge_exec(sort_exprs.clone(), source); - let sort2 = sort_exec(vec![sort_exprs[0].clone()], spm1); - let physical_plan = sort_preserving_merge_exec(vec![sort_exprs[1].clone()], sort2); - - let expected_input = [ - "SortPreservingMergeExec: [non_nullable_col@1 ASC]", - " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", - " MemoryExec: partitions=1, partition_sizes=[0]", - ]; - let expected_optimized = [ - "SortExec: expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", - ]; + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[true]", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", + " UnionExec", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", + " GlobalLimitExec: skip=0, fetch=100", + " LocalLimitExec: fetch=100", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -727,12 +287,12 @@ async fn test_union_inputs_sorted() -> Result<()> { // one input to the union is already sorted, one is not. let expected_input = vec![ - "SortPreservingMergeExec: [nullable_col@0 ASC]", - " UnionExec", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", - " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - ]; + "SortPreservingMergeExec: [nullable_col@0 ASC]", + " UnionExec", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", + " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + ]; // should not add a sort at the output of the union, input plan should not be changed let expected_optimized = expected_input.clone(); assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -759,12 +319,12 @@ async fn test_union_inputs_different_sorted() -> Result<()> { // one input to the union is already sorted, one is not. let expected_input = vec![ - "SortPreservingMergeExec: [nullable_col@0 ASC]", - " UnionExec", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC, non_nullable_col@1 ASC]", - " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - ]; + "SortPreservingMergeExec: [nullable_col@0 ASC]", + " UnionExec", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC, non_nullable_col@1 ASC]", + " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + ]; // should not add a sort at the output of the union, input plan should not be changed let expected_optimized = expected_input.clone(); assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -793,17 +353,17 @@ async fn test_union_inputs_different_sorted2() -> Result<()> { // First ParquetExec has output ordering(nullable_col@0 ASC). However, it doesn't satisfy the // required ordering of SortPreservingMergeExec. let expected_input = ["SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", - " UnionExec", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", - " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; + " UnionExec", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; let expected_optimized = ["SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", - " UnionExec", - " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", - " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; + " UnionExec", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -832,20 +392,20 @@ async fn test_union_inputs_different_sorted3() -> Result<()> { // Second input to the union is already Sorted (matches with the required ordering by the SortPreservingMergeExec above). // Third input to the union is not Sorted (SortExec is matches required ordering by the SortPreservingMergeExec above). let expected_input = ["SortPreservingMergeExec: [nullable_col@0 ASC]", - " UnionExec", - " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", - " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; + " UnionExec", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", + " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; // should adjust sorting in the first input of the union such that it is not unnecessarily fine let expected_optimized = ["SortPreservingMergeExec: [nullable_col@0 ASC]", - " UnionExec", - " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", - " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; + " UnionExec", + " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", + " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -874,20 +434,20 @@ async fn test_union_inputs_different_sorted4() -> Result<()> { // `UnionExec` satisfy the ordering, OR add a single sort after // the `UnionExec` (both of which are equally good for this example). let expected_input = ["SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", - " UnionExec", - " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", - " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; + " UnionExec", + " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", + " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; let expected_optimized = ["SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", - " UnionExec", - " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", - " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; + " UnionExec", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -924,17 +484,17 @@ async fn test_union_inputs_different_sorted5() -> Result<()> { // example below. However, we should be able to change the unnecessarily // fine `SortExec`s below with required `SortExec`s that are absolutely necessary. let expected_input = ["SortPreservingMergeExec: [nullable_col@0 ASC]", - " UnionExec", - " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 DESC NULLS LAST], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; + " UnionExec", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 DESC NULLS LAST], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; let expected_optimized = ["SortPreservingMergeExec: [nullable_col@0 ASC]", - " UnionExec", - " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; + " UnionExec", + " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -967,23 +527,23 @@ async fn test_union_inputs_different_sorted6() -> Result<()> { // The final plan should be valid AND the ordering of the third child // shouldn't be finer than necessary. let expected_input = ["SortPreservingMergeExec: [nullable_col@0 ASC]", - " UnionExec", - " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", - " SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; + " UnionExec", + " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", + " SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; // Should adjust the requirement in the third input of the union so // that it is not unnecessarily fine. let expected_optimized = ["SortPreservingMergeExec: [nullable_col@0 ASC]", - " UnionExec", - " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", - " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[true]", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; + " UnionExec", + " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", + " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[true]", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -1007,18 +567,18 @@ async fn test_union_inputs_different_sorted7() -> Result<()> { // Union has unnecessarily fine ordering below it. We should be able to replace them with absolutely necessary ordering. let expected_input = ["SortPreservingMergeExec: [nullable_col@0 ASC]", - " UnionExec", - " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; + " UnionExec", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; // Union preserves the inputs ordering and we should not change any of the SortExecs under UnionExec let expected_output = ["SortPreservingMergeExec: [nullable_col@0 ASC]", - " UnionExec", - " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; + " UnionExec", + " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; assert_optimized!(expected_input, expected_output, physical_plan, true); Ok(()) @@ -1059,15 +619,15 @@ async fn test_union_inputs_different_sorted8() -> Result<()> { // The `UnionExec` doesn't preserve any of the inputs ordering in the // example below. let expected_input = ["UnionExec", - " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " SortExec: expr=[nullable_col@0 DESC NULLS LAST, non_nullable_col@1 DESC NULLS LAST], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " SortExec: expr=[nullable_col@0 DESC NULLS LAST, non_nullable_col@1 DESC NULLS LAST], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; // Since `UnionExec` doesn't preserve ordering in the plan above. // We shouldn't keep SortExecs in the plan. let expected_optimized = ["UnionExec", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -1105,19 +665,19 @@ async fn test_window_multi_path_sort() -> Result<()> { // corresponding SortExecs together. Also, the inputs of these `SortExec`s // are not necessarily the same to be able to remove them. 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: [nullable_col@0 DESC NULLS LAST]", - " UnionExec", - " SortExec: expr=[nullable_col@0 DESC NULLS LAST], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC, non_nullable_col@1 ASC]", - " SortExec: expr=[nullable_col@0 DESC NULLS LAST], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]"]; + "BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", + " SortPreservingMergeExec: [nullable_col@0 DESC NULLS LAST]", + " UnionExec", + " SortExec: expr=[nullable_col@0 DESC NULLS LAST], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC, non_nullable_col@1 ASC]", + " SortExec: expr=[nullable_col@0 DESC NULLS LAST], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]"]; let expected_optimized = [ - "WindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(NULL), is_causal: false }]", - " SortPreservingMergeExec: [nullable_col@0 ASC]", - " UnionExec", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC, non_nullable_col@1 ASC]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]"]; + "WindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(NULL), is_causal: false }]", + " SortPreservingMergeExec: [nullable_col@0 ASC]", + " UnionExec", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC, non_nullable_col@1 ASC]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -1144,17 +704,17 @@ async fn test_window_multi_path_sort2() -> Result<()> { // The `WindowAggExec` can get its required sorting from the leaf nodes directly. // The unnecessary SortExecs should be removed 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: [nullable_col@0 ASC, non_nullable_col@1 ASC]", - " UnionExec", - " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", - " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]"]; + " SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", + " UnionExec", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]"]; let expected_optimized = ["BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", - " SortPreservingMergeExec: [nullable_col@0 ASC]", - " UnionExec", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]"]; + " SortPreservingMergeExec: [nullable_col@0 ASC]", + " UnionExec", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -1192,21 +752,21 @@ async fn test_union_inputs_different_sorted_with_limit() -> Result<()> { // Should not change the unnecessarily fine `SortExec`s because there is `LimitExec` let expected_input = ["SortPreservingMergeExec: [nullable_col@0 ASC]", - " UnionExec", - " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " GlobalLimitExec: skip=0, fetch=100", - " LocalLimitExec: fetch=100", - " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 DESC NULLS LAST], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; + " UnionExec", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " GlobalLimitExec: skip=0, fetch=100", + " LocalLimitExec: fetch=100", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 DESC NULLS LAST], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; let expected_optimized = ["SortPreservingMergeExec: [nullable_col@0 ASC]", - " UnionExec", - " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " GlobalLimitExec: skip=0, fetch=100", - " LocalLimitExec: fetch=100", - " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 DESC NULLS LAST], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; + " UnionExec", + " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " GlobalLimitExec: skip=0, fetch=100", + " LocalLimitExec: fetch=100", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 DESC NULLS LAST], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -1250,9 +810,9 @@ async fn test_sort_merge_join_order_by_left() -> Result<()> { " SortMergeJoin: join_type={join_type}, on=[(nullable_col@0, col_a@0)]" ); let expected_input = ["SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", - join_plan2.as_str(), - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]"]; + join_plan2.as_str(), + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]"]; let expected_optimized = match join_type { JoinType::Inner | JoinType::Left @@ -1260,23 +820,23 @@ async fn test_sort_merge_join_order_by_left() -> Result<()> { | JoinType::LeftAnti => { // can push down the sort requirements and save 1 SortExec vec![ - join_plan.as_str(), - " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " SortExec: expr=[col_a@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]", - ] + join_plan.as_str(), + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " SortExec: expr=[col_a@0 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]", + ] } _ => { // can not push down the sort requirements vec![ - "SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - join_plan2.as_str(), - " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " SortExec: expr=[col_a@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]", - ] + "SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", + join_plan2.as_str(), + " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " SortExec: expr=[col_a@0 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]", + ] } }; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -1325,30 +885,30 @@ async fn test_sort_merge_join_order_by_right() -> Result<()> { " SortMergeJoin: join_type={join_type}, on=[(nullable_col@0, col_a@0)]" ); let expected_input = [spm_plan, - join_plan2.as_str(), - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]"]; + join_plan2.as_str(), + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]"]; let expected_optimized = match join_type { JoinType::Inner | JoinType::Right | JoinType::RightAnti => { // can push down the sort requirements and save 1 SortExec vec![ - join_plan.as_str(), - " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " SortExec: expr=[col_a@0 ASC, col_b@1 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]", - ] + join_plan.as_str(), + " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " SortExec: expr=[col_a@0 ASC, col_b@1 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]", + ] } _ => { // can not push down the sort requirements for Left and Full join. vec![ - "SortExec: expr=[col_a@2 ASC, col_b@3 ASC], preserve_partitioning=[false]", - join_plan2.as_str(), - " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " SortExec: expr=[col_a@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]", - ] + "SortExec: expr=[col_a@2 ASC, col_b@3 ASC], preserve_partitioning=[false]", + join_plan2.as_str(), + " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " SortExec: expr=[col_a@0 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]", + ] } }; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -1380,17 +940,17 @@ async fn test_sort_merge_join_complex_order_by() -> Result<()> { let physical_plan = sort_preserving_merge_exec(sort_exprs1, join.clone()); let expected_input = ["SortPreservingMergeExec: [col_b@3 ASC, col_a@2 ASC]", - " SortMergeJoin: join_type=Inner, on=[(nullable_col@0, col_a@0)]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]"]; + " SortMergeJoin: join_type=Inner, on=[(nullable_col@0, col_a@0)]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]"]; // can not push down the sort requirements, need to add SortExec let expected_optimized = ["SortExec: expr=[col_b@3 ASC, col_a@2 ASC], preserve_partitioning=[false]", - " SortMergeJoin: join_type=Inner, on=[(nullable_col@0, col_a@0)]", - " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " SortExec: expr=[col_a@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]"]; + " SortMergeJoin: join_type=Inner, on=[(nullable_col@0, col_a@0)]", + " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " SortExec: expr=[col_a@0 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); // order by (nullable_col, col_b, col_a) @@ -1402,50 +962,17 @@ async fn test_sort_merge_join_complex_order_by() -> Result<()> { let physical_plan = sort_preserving_merge_exec(sort_exprs2, join); let expected_input = ["SortPreservingMergeExec: [nullable_col@0 ASC, col_b@3 ASC, col_a@2 ASC]", - " SortMergeJoin: join_type=Inner, on=[(nullable_col@0, col_a@0)]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]"]; + " SortMergeJoin: join_type=Inner, on=[(nullable_col@0, col_a@0)]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]"]; // can not push down the sort requirements, need to add SortExec let expected_optimized = ["SortExec: expr=[nullable_col@0 ASC, col_b@3 ASC, col_a@2 ASC], preserve_partitioning=[false]", - " SortMergeJoin: join_type=Inner, on=[(nullable_col@0, col_a@0)]", - " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " SortExec: expr=[col_a@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]"]; - assert_optimized!(expected_input, expected_optimized, physical_plan, true); - - Ok(()) -} - -#[tokio::test] -async fn test_multiple_sort_window_exec() -> Result<()> { - let schema = create_test_schema()?; - let source = memory_exec(&schema); - - let sort_exprs1 = vec![sort_expr("nullable_col", &schema)]; - let sort_exprs2 = vec![ - sort_expr("nullable_col", &schema), - sort_expr("non_nullable_col", &schema), - ]; - - let sort1 = sort_exec(sort_exprs1.clone(), source); - let window_agg1 = bounded_window_exec("non_nullable_col", sort_exprs1.clone(), sort1); - let window_agg2 = bounded_window_exec("non_nullable_col", sort_exprs2, window_agg1); - // let filter_exec = sort_exec; - let physical_plan = bounded_window_exec("non_nullable_col", sort_exprs1, window_agg2); - - 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]", - " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", - " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", - " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]"]; - - let expected_optimized = ["BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", - " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", - " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", - " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]"]; + " SortMergeJoin: join_type=Inner, on=[(nullable_col@0, col_a@0)]", + " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " SortExec: expr=[col_a@0 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -1472,109 +999,15 @@ async fn test_multilayer_coalesce_partitions() -> Result<()> { // we should be able to parallelize Sorting also (given that executors in between don't require) // single partition. 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", - " 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", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; - assert_optimized!(expected_input, expected_optimized, physical_plan, true); - - Ok(()) -} - -#[tokio::test] -// With new change in SortEnforcement EnforceSorting->EnforceDistribution->EnforceSorting -// should produce same result with EnforceDistribution+EnforceSorting -// This enables us to use EnforceSorting possibly before EnforceDistribution -// Given that it will be called at least once after last EnforceDistribution. The reason is that -// EnforceDistribution may invalidate ordering invariant. -async fn test_commutativity() -> Result<()> { - let schema = create_test_schema()?; - - let session_ctx = SessionContext::new(); - let state = session_ctx.state(); - - let memory_exec = memory_exec(&schema); - let sort_exprs = LexOrdering::new(vec![sort_expr("nullable_col", &schema)]); - let window = bounded_window_exec("nullable_col", sort_exprs.clone(), memory_exec); - let repartition = repartition_exec(window); - - let orig_plan = - Arc::new(SortExec::new(sort_exprs, repartition)) as Arc; - 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", - " 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]", - ]; - assert_eq!( - expected_input, actual, - "\n**Original Plan Mismatch\n\nexpected:\n\n{expected_input:#?}\nactual:\n\n{actual:#?}\n\n" - ); - - let mut plan = orig_plan.clone(); - let rules = vec![ - Arc::new(EnforceDistribution::new()) as Arc, - Arc::new(EnforceSorting::new()) as Arc, - ]; - for rule in rules { - plan = rule.optimize(plan, state.config_options())?; - } - let first_plan = plan.clone(); - - let mut plan = orig_plan.clone(); - let rules = vec![ - Arc::new(EnforceSorting::new()) as Arc, - Arc::new(EnforceDistribution::new()) as Arc, - Arc::new(EnforceSorting::new()) as Arc, - ]; - for rule in rules { - plan = rule.optimize(plan, state.config_options())?; - } - let second_plan = plan.clone(); - - assert_eq!(get_plan_string(&first_plan), get_plan_string(&second_plan)); - Ok(()) -} - -#[tokio::test] -async fn test_coalesce_propagate() -> Result<()> { - let schema = create_test_schema()?; - let source = memory_exec(&schema); - let repartition = repartition_exec(source); - let coalesce_partitions = Arc::new(CoalescePartitionsExec::new(repartition)); - let repartition = repartition_exec(coalesce_partitions); - let sort_exprs = LexOrdering::new(vec![sort_expr("nullable_col", &schema)]); - // Add local sort - let sort = Arc::new( - SortExec::new(sort_exprs.clone(), repartition).with_preserve_partitioning(true), - ) as _; - let spm = sort_preserving_merge_exec(sort_exprs.clone(), sort); - let sort = sort_exec(sort_exprs, spm); - - let physical_plan = sort.clone(); - // Sort Parallelize rule should end Coalesce + Sort linkage when Sort is Global Sort - // Also input plan is not valid as it is. We need to add SortExec before SortPreservingMergeExec. - 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]", + " FilterExec: NOT non_nullable_col@1", + " CoalescePartitionsExec", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " CoalescePartitionsExec", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[0]", - ]; - let expected_optimized = [ - "SortPreservingMergeExec: [nullable_col@0 ASC]", + " 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]", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[0]", - ]; + " FilterExec: NOT non_nullable_col@1", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -1594,15 +1027,15 @@ async fn test_with_lost_ordering_bounded() -> Result<()> { let physical_plan = sort_exec(vec![sort_expr("a", &schema)], coalesce_partitions); 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", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], has_header=false"]; + " CoalescePartitionsExec", + " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], has_header=false"]; let expected_optimized = ["SortPreservingMergeExec: [a@0 ASC]", - " SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", - " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], has_header=false"]; + " SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", + " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], has_header=false"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -1631,43 +1064,43 @@ async fn test_with_lost_ordering_unbounded_bounded( // Expected inputs unbounded and bounded let expected_input_unbounded = 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", - " StreamingTableExec: partition_sizes=1, projection=[a, b, c, d, e], infinite_source=true, output_ordering=[a@0 ASC]", - ]; + "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", + " 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", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], has_header=true", - ]; + "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", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], has_header=true", + ]; // Expected unbounded result (same for with and without flag) 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", - " StreamingTableExec: partition_sizes=1, projection=[a, b, c, d, e], infinite_source=true, output_ordering=[a@0 ASC]", - ]; + "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", + " StreamingTableExec: partition_sizes=1, projection=[a, b, c, d, e], infinite_source=true, output_ordering=[a@0 ASC]", + ]; // Expected bounded results with and without flag let expected_optimized_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", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], has_header=true", - ]; + "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", + " 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", - " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], has_header=true", - ]; + "SortPreservingMergeExec: [a@0 ASC]", + " SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", + " RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " CsvExec: file_groups={1 group: [[file_path]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC], has_header=true", + ]; let (expected_input, expected_optimized, expected_optimized_sort_parallelize) = if source_unbounded { ( @@ -1708,13 +1141,13 @@ async fn test_do_not_pushdown_through_spm() -> Result<()> { let physical_plan = sort_exec(vec![sort_expr("b", &schema)], spm); let expected_input = ["SortExec: expr=[b@1 ASC], preserve_partitioning=[false]", - " SortPreservingMergeExec: [a@0 ASC, b@1 ASC]", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC, b@1 ASC], has_header=false",]; + " SortPreservingMergeExec: [a@0 ASC, b@1 ASC]", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC, b@1 ASC], has_header=false",]; let expected_optimized = ["SortExec: expr=[b@1 ASC], preserve_partitioning=[false]", - " SortPreservingMergeExec: [a@0 ASC, b@1 ASC]", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC, b@1 ASC], has_header=false",]; + " SortPreservingMergeExec: [a@0 ASC, b@1 ASC]", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC, b@1 ASC], has_header=false",]; assert_optimized!(expected_input, expected_optimized, physical_plan, false); Ok(()) @@ -1737,13 +1170,13 @@ async fn test_pushdown_through_spm() -> Result<()> { ); let expected_input = ["SortExec: expr=[a@0 ASC, b@1 ASC, c@2 ASC], preserve_partitioning=[false]", - " SortPreservingMergeExec: [a@0 ASC, b@1 ASC]", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC, b@1 ASC], has_header=false",]; + " SortPreservingMergeExec: [a@0 ASC, b@1 ASC]", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC, b@1 ASC], has_header=false",]; let expected_optimized = ["SortPreservingMergeExec: [a@0 ASC, b@1 ASC]", - " SortExec: expr=[a@0 ASC, b@1 ASC, c@2 ASC], preserve_partitioning=[true]", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC, b@1 ASC], has_header=false",]; + " SortExec: expr=[a@0 ASC, b@1 ASC, c@2 ASC], preserve_partitioning=[true]", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC, b@1 ASC], has_header=false",]; assert_optimized!(expected_input, expected_optimized, physical_plan, false); Ok(()) @@ -1762,77 +1195,25 @@ async fn test_window_multi_layer_requirement() -> Result<()> { let physical_plan = bounded_window_exec("a", sort_exprs, spm); 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", - " 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", - ]; + "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", + " 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", + ]; let expected_optimized = [ - "BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", - " SortExec: expr=[a@0 ASC, b@1 ASC], preserve_partitioning=[false]", - " CoalescePartitionsExec", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - ]; + "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", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + ]; assert_optimized!(expected_input, expected_optimized, physical_plan, false); Ok(()) } - -#[tokio::test] -async fn test_replace_with_partial_sort() -> Result<()> { - let schema = create_test_schema3()?; - let input_sort_exprs = vec![sort_expr("a", &schema)]; - let unbounded_input = stream_exec_ordered(&schema, input_sort_exprs); - - let physical_plan = sort_exec( - vec![sort_expr("a", &schema), sort_expr("c", &schema)], - unbounded_input, - ); - - let expected_input = [ - "SortExec: expr=[a@0 ASC, c@2 ASC], preserve_partitioning=[false]", - " StreamingTableExec: partition_sizes=1, projection=[a, b, c, d, e], infinite_source=true, output_ordering=[a@0 ASC]" - ]; - let expected_optimized = [ - "PartialSortExec: expr=[a@0 ASC, c@2 ASC], common_prefix_length=[1]", - " StreamingTableExec: partition_sizes=1, projection=[a, b, c, d, e], infinite_source=true, output_ordering=[a@0 ASC]", - ]; - assert_optimized!(expected_input, expected_optimized, physical_plan, true); - Ok(()) -} - -#[tokio::test] -async fn test_replace_with_partial_sort2() -> Result<()> { - let schema = create_test_schema3()?; - let input_sort_exprs = vec![sort_expr("a", &schema), sort_expr("c", &schema)]; - let unbounded_input = stream_exec_ordered(&schema, input_sort_exprs); - - let physical_plan = sort_exec( - vec![ - sort_expr("a", &schema), - sort_expr("c", &schema), - sort_expr("d", &schema), - ], - unbounded_input, - ); - - let expected_input = [ - "SortExec: expr=[a@0 ASC, c@2 ASC, d@3 ASC], preserve_partitioning=[false]", - " StreamingTableExec: partition_sizes=1, projection=[a, b, c, d, e], infinite_source=true, output_ordering=[a@0 ASC, c@2 ASC]" - ]; - // let optimized - let expected_optimized = [ - "PartialSortExec: expr=[a@0 ASC, c@2 ASC, d@3 ASC], common_prefix_length=[2]", - " StreamingTableExec: partition_sizes=1, projection=[a, b, c, d, e], infinite_source=true, output_ordering=[a@0 ASC, c@2 ASC]", - ]; - assert_optimized!(expected_input, expected_optimized, physical_plan, true); - Ok(()) -} - #[tokio::test] async fn test_not_replaced_with_partial_sort_for_bounded_input() -> Result<()> { let schema = create_test_schema3()?; @@ -1848,97 +1229,10 @@ async fn test_not_replaced_with_partial_sort_for_bounded_input() -> Result<()> { parquet_input, ); let expected_input = [ - "SortExec: expr=[a@0 ASC, b@1 ASC, c@2 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[b@1 ASC, c@2 ASC]" - ]; - let expected_no_change = expected_input; - assert_optimized!(expected_input, expected_no_change, physical_plan, false); - Ok(()) -} - -#[tokio::test] -async fn test_not_replaced_with_partial_sort_for_unbounded_input() -> Result<()> { - let schema = create_test_schema3()?; - let input_sort_exprs = vec![sort_expr("b", &schema), sort_expr("c", &schema)]; - let unbounded_input = stream_exec_ordered(&schema, input_sort_exprs); - - let physical_plan = sort_exec( - vec![ - sort_expr("a", &schema), - sort_expr("b", &schema), - sort_expr("c", &schema), - ], - unbounded_input, - ); - let expected_input = [ - "SortExec: expr=[a@0 ASC, b@1 ASC, c@2 ASC], preserve_partitioning=[false]", - " StreamingTableExec: partition_sizes=1, projection=[a, b, c, d, e], infinite_source=true, output_ordering=[b@1 ASC, c@2 ASC]" - ]; - let expected_no_change = expected_input; - assert_optimized!(expected_input, expected_no_change, physical_plan, true); - Ok(()) -} - -#[tokio::test] -async fn test_push_with_required_input_ordering_prohibited() -> Result<()> { - // SortExec: expr=[b] <-- can't push this down - // RequiredInputOrder expr=[a] <-- this requires input sorted by a, and preserves the input order - // SortExec: expr=[a] - // MemoryExec - let schema = create_test_schema3()?; - let sort_exprs_a = LexOrdering::new(vec![sort_expr("a", &schema)]); - let sort_exprs_b = LexOrdering::new(vec![sort_expr("b", &schema)]); - let plan = memory_exec(&schema); - let plan = sort_exec(sort_exprs_a.clone(), plan); - let plan = RequirementsTestExec::new(plan) - .with_required_input_ordering(sort_exprs_a) - .with_maintains_input_order(true) - .into_arc(); - let plan = sort_exec(sort_exprs_b, plan); - - let expected_input = [ - "SortExec: expr=[b@1 ASC], preserve_partitioning=[false]", - " RequiredInputOrderingExec", - " SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", + "SortExec: expr=[a@0 ASC, b@1 ASC, c@2 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[b@1 ASC, c@2 ASC]" ]; - // should not be able to push shorts let expected_no_change = expected_input; - assert_optimized!(expected_input, expected_no_change, plan, true); - Ok(()) -} - -// test when the required input ordering is satisfied so could push through -#[tokio::test] -async fn test_push_with_required_input_ordering_allowed() -> Result<()> { - // SortExec: expr=[a,b] <-- can push this down (as it is compatible with the required input ordering) - // RequiredInputOrder expr=[a] <-- this requires input sorted by a, and preserves the input order - // SortExec: expr=[a] - // MemoryExec - let schema = create_test_schema3()?; - let sort_exprs_a = LexOrdering::new(vec![sort_expr("a", &schema)]); - let sort_exprs_ab = - LexOrdering::new(vec![sort_expr("a", &schema), sort_expr("b", &schema)]); - let plan = memory_exec(&schema); - let plan = sort_exec(sort_exprs_a.clone(), plan); - let plan = RequirementsTestExec::new(plan) - .with_required_input_ordering(sort_exprs_a) - .with_maintains_input_order(true) - .into_arc(); - let plan = sort_exec(sort_exprs_ab, plan); - - let expected_input = [ - "SortExec: expr=[a@0 ASC, b@1 ASC], preserve_partitioning=[false]", - " RequiredInputOrderingExec", - " SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", - ]; - // should able to push shorts - let expected = [ - "RequiredInputOrderingExec", - " SortExec: expr=[a@0 ASC, b@1 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", - ]; - assert_optimized!(expected_input, expected, plan, true); + assert_optimized!(expected_input, expected_no_change, physical_plan, false); Ok(()) } diff --git a/datafusion/core/tests/physical_optimizer/limited_distinct_aggregation.rs b/datafusion/core/tests/physical_optimizer/limited_distinct_aggregation.rs index 46a56fe1fb75..4373dc40de38 100644 --- a/datafusion/core/tests/physical_optimizer/limited_distinct_aggregation.rs +++ b/datafusion/core/tests/physical_optimizer/limited_distinct_aggregation.rs @@ -15,100 +15,28 @@ // specific language governing permissions and limitations // under the License. -//! Tests for [`LimitedDistinctAggregation`] physical optimizer rule -//! -//! Note these tests are not in the same module as the optimizer pass because -//! they rely on `ParquetExec` which is in the core crate. -use super::test_util::{parquet_exec_with_sort, schema, trim_plan_display}; +//! Integration tests for [`LimitedDistinctAggregation`] physical optimizer rule use std::sync::Arc; -use arrow::{ - array::Int32Array, - compute::SortOptions, - datatypes::{DataType, Field, Schema}, - record_batch::RecordBatch, - util::pretty::pretty_format_batches, -}; -use arrow_schema::SchemaRef; -use datafusion::{prelude::SessionContext, test_util::TestAggregate}; +use crate::physical_optimizer::parquet_exec_with_sort; + +use arrow::{compute::SortOptions, util::pretty::pretty_format_batches}; +use datafusion::prelude::SessionContext; use datafusion_common::Result; use datafusion_execution::config::SessionConfig; -use datafusion_expr::Operator; -use datafusion_physical_expr::{ - expressions::{cast, col}, - PhysicalExpr, PhysicalSortExpr, -}; +use datafusion_physical_expr::{expressions::col, PhysicalSortExpr}; use datafusion_physical_expr_common::sort_expr::LexOrdering; -use datafusion_physical_optimizer::{ - limited_distinct_aggregation::LimitedDistinctAggregation, PhysicalOptimizerRule, +use datafusion_physical_optimizer::test_utils::{ + assert_plan_matches_expected, build_group_by, mock_data, schema, }; use datafusion_physical_plan::{ - aggregates::{AggregateExec, AggregateMode, PhysicalGroupBy}, - collect, displayable, expressions, + aggregates::{AggregateExec, AggregateMode}, + collect, limit::{GlobalLimitExec, LocalLimitExec}, - memory::MemoryExec, ExecutionPlan, }; -fn mock_data() -> Result> { - let schema = Arc::new(Schema::new(vec![ - Field::new("a", DataType::Int32, true), - Field::new("b", DataType::Int32, true), - ])); - - let batch = RecordBatch::try_new( - Arc::clone(&schema), - vec![ - Arc::new(Int32Array::from(vec![ - Some(1), - Some(2), - None, - Some(1), - Some(4), - Some(5), - ])), - Arc::new(Int32Array::from(vec![ - Some(1), - None, - Some(6), - Some(2), - Some(8), - Some(9), - ])), - ], - )?; - - Ok(Arc::new(MemoryExec::try_new( - &[vec![batch]], - Arc::clone(&schema), - None, - )?)) -} - -fn assert_plan_matches_expected( - plan: &Arc, - expected: &[&str], -) -> Result<()> { - let expected_lines: Vec<&str> = expected.to_vec(); - let session_ctx = SessionContext::new(); - let state = session_ctx.state(); - - let optimized = LimitedDistinctAggregation::new() - .optimize(Arc::clone(plan), state.config_options())?; - - let optimized_result = displayable(optimized.as_ref()).indent(true).to_string(); - let actual_lines = trim_plan_display(&optimized_result); - - assert_eq!( - &expected_lines, &actual_lines, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected_lines, actual_lines - ); - - Ok(()) -} - async fn assert_results_match_expected( plan: Arc, expected: &str, @@ -121,14 +49,6 @@ async fn assert_results_match_expected( Ok(()) } -pub fn build_group_by(input_schema: &SchemaRef, columns: Vec) -> PhysicalGroupBy { - let mut group_by_expr: Vec<(Arc, String)> = vec![]; - for column in columns.iter() { - group_by_expr.push((col(column, input_schema).unwrap(), column.to_string())); - } - PhysicalGroupBy::new_single(group_by_expr.clone()) -} - #[tokio::test] async fn test_partial_final() -> Result<()> { let source = mock_data()?; @@ -311,103 +231,6 @@ async fn test_distinct_cols_different_than_group_by_cols() -> Result<()> { Ok(()) } -#[test] -fn test_no_group_by() -> Result<()> { - let source = mock_data()?; - let schema = source.schema(); - - // `SELECT FROM MemoryExec LIMIT 10;`, Single AggregateExec - let single_agg = AggregateExec::try_new( - AggregateMode::Single, - build_group_by(&schema, vec![]), - vec![], /* aggr_expr */ - vec![], /* filter_expr */ - source, /* input */ - schema, /* input_schema */ - )?; - let limit_exec = LocalLimitExec::new( - Arc::new(single_agg), - 10, // fetch - ); - // expected not to push the limit to the AggregateExec - let expected = [ - "LocalLimitExec: fetch=10", - "AggregateExec: mode=Single, gby=[], aggr=[]", - "MemoryExec: partitions=1, partition_sizes=[1]", - ]; - let plan: Arc = Arc::new(limit_exec); - assert_plan_matches_expected(&plan, &expected)?; - Ok(()) -} - -#[test] -fn test_has_aggregate_expression() -> Result<()> { - let source = mock_data()?; - let schema = source.schema(); - let agg = TestAggregate::new_count_star(); - - // `SELECT FROM MemoryExec LIMIT 10;`, Single AggregateExec - let single_agg = AggregateExec::try_new( - AggregateMode::Single, - build_group_by(&schema, vec!["a".to_string()]), - vec![Arc::new(agg.count_expr(&schema))], /* aggr_expr */ - vec![None], /* filter_expr */ - source, /* input */ - schema.clone(), /* input_schema */ - )?; - let limit_exec = LocalLimitExec::new( - Arc::new(single_agg), - 10, // fetch - ); - // expected not to push the limit to the AggregateExec - let expected = [ - "LocalLimitExec: fetch=10", - "AggregateExec: mode=Single, gby=[a@0 as a], aggr=[COUNT(*)]", - "MemoryExec: partitions=1, partition_sizes=[1]", - ]; - let plan: Arc = Arc::new(limit_exec); - assert_plan_matches_expected(&plan, &expected)?; - Ok(()) -} - -#[test] -fn test_has_filter() -> Result<()> { - let source = mock_data()?; - let schema = source.schema(); - - // `SELECT a FROM MemoryExec WHERE a > 1 GROUP BY a LIMIT 10;`, Single AggregateExec - // the `a > 1` filter is applied in the AggregateExec - let filter_expr = Some(expressions::binary( - col("a", &schema)?, - Operator::Gt, - cast(expressions::lit(1u32), &schema, DataType::Int32)?, - &schema, - )?); - let agg = TestAggregate::new_count_star(); - let single_agg = AggregateExec::try_new( - AggregateMode::Single, - build_group_by(&schema.clone(), vec!["a".to_string()]), - vec![Arc::new(agg.count_expr(&schema))], /* aggr_expr */ - vec![filter_expr], /* filter_expr */ - source, /* input */ - schema.clone(), /* input_schema */ - )?; - let limit_exec = LocalLimitExec::new( - Arc::new(single_agg), - 10, // fetch - ); - // expected not to push the limit to the AggregateExec - // TODO(msirek): open an issue for `filter_expr` of `AggregateExec` not printing out - let expected = [ - "LocalLimitExec: fetch=10", - "AggregateExec: mode=Single, gby=[a@0 as a], aggr=[COUNT(*)]", - "MemoryExec: partitions=1, partition_sizes=[1]", - ]; - let plan: Arc = Arc::new(limit_exec); - assert_plan_matches_expected(&plan, &expected)?; - Ok(()) -} - #[test] fn test_has_order_by() -> Result<()> { let sort_key = LexOrdering::new(vec![PhysicalSortExpr { @@ -433,10 +256,10 @@ fn test_has_order_by() -> Result<()> { ); // expected not to push the limit to the AggregateExec let expected = [ - "LocalLimitExec: fetch=10", - "AggregateExec: mode=Single, gby=[a@0 as a], aggr=[], ordering_mode=Sorted", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", - ]; + "LocalLimitExec: fetch=10", + "AggregateExec: mode=Single, gby=[a@0 as a], aggr=[], ordering_mode=Sorted", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", + ]; let plan: Arc = Arc::new(limit_exec); assert_plan_matches_expected(&plan, &expected)?; Ok(()) diff --git a/datafusion/core/tests/physical_optimizer/mod.rs b/datafusion/core/tests/physical_optimizer/mod.rs index 55f50a682b4a..e4128d8fb8ef 100644 --- a/datafusion/core/tests/physical_optimizer/mod.rs +++ b/datafusion/core/tests/physical_optimizer/mod.rs @@ -15,9 +15,41 @@ // specific language governing permissions and limitations // under the License. +//! Physical Optimizer integration tests + mod combine_partial_final_agg; +mod enforce_distribution; mod enforce_sorting; mod limited_distinct_aggregation; mod replace_with_order_preserving_variants; mod sanity_checker; -pub(crate) mod test_util; + +use std::sync::Arc; + +use arrow_schema::SchemaRef; +use datafusion::datasource::listing::PartitionedFile; +use datafusion::datasource::physical_plan::{FileScanConfig, ParquetExec}; +use datafusion_execution::object_store::ObjectStoreUrl; +use datafusion_physical_expr_common::sort_expr::LexOrdering; +use datafusion_physical_optimizer::test_utils::schema; + +/// Create a non sorted parquet exec +pub fn parquet_exec(schema: &SchemaRef) -> Arc { + ParquetExec::builder( + FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema.clone()) + .with_file(PartitionedFile::new("x".to_string(), 100)), + ) + .build_arc() +} + +/// Create a single parquet file that is sorted +pub(crate) fn parquet_exec_with_sort( + output_ordering: Vec, +) -> Arc { + ParquetExec::builder( + FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema()) + .with_file(PartitionedFile::new("x".to_string(), 100)) + .with_output_ordering(output_ordering), + ) + .build_arc() +} diff --git a/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs b/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs index e102ab5041b7..912683083738 100644 --- a/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs +++ b/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs @@ -17,9 +17,12 @@ use std::sync::Arc; -use datafusion::prelude::{SessionConfig, SessionContext}; +use datafusion::prelude::SessionContext; +use arrow::array::{ArrayRef, Int32Array}; +use arrow::compute::SortOptions; +use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; +use arrow::record_batch::RecordBatch; use datafusion_execution::TaskContext; - use datafusion_physical_plan::coalesce_batches::CoalesceBatchesExec; use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; use datafusion_physical_plan::collect; @@ -32,27 +35,53 @@ use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeE use datafusion_physical_plan::{ displayable, get_plan_string, ExecutionPlan, Partitioning, }; - -use arrow::array::{ArrayRef, Int32Array}; -use arrow::compute::SortOptions; -use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; -use arrow::record_batch::RecordBatch; - use datafusion_common::tree_node::{TransformedResult, TreeNode}; - use datafusion_common::Result; - use datafusion_expr::{JoinType, Operator}; - use datafusion_physical_expr::expressions::{self, col, Column}; - use datafusion_physical_expr::PhysicalSortExpr; - use datafusion_physical_optimizer::test_utils::check_integrity; - use datafusion_physical_optimizer::enforce_sorting::replace_with_order_preserving_variants::{replace_with_order_preserving_variants, OrderPreservationContext}; - -use crate::physical_optimizer::test_util::stream_exec_ordered_with_projection; +use datafusion_common::Result; +use datafusion_expr::{JoinType, Operator}; +use datafusion_physical_expr::expressions::{self, col, Column}; +use datafusion_physical_expr::PhysicalSortExpr; +use datafusion_physical_optimizer::test_utils::{check_integrity, stream_exec_ordered_with_projection}; +use datafusion_physical_optimizer::enforce_sorting::replace_with_order_preserving_variants::{replace_with_order_preserving_variants, OrderPreservationContext}; +use datafusion_common::config::ConfigOptions; use object_store::memory::InMemory; use object_store::ObjectStore; use rstest::rstest; use url::Url; +/// Runs the `replace_with_order_preserving_variants` sub-rule and asserts +/// the plan against the original and expected plans. +/// +/// # Parameters +/// +/// * `$EXPECTED_PLAN_LINES`: Expected input plan. +/// * `EXPECTED_OPTIMIZED_PLAN_LINES`: Optimized plan when the flag +/// `prefer_existing_sort` is `false`. +/// * `EXPECTED_PREFER_SORT_ON_OPTIMIZED_PLAN_LINES`: Optimized plan when +/// the flag `prefer_existing_sort` is `true`. +/// * `$PLAN`: The plan to optimize. +macro_rules! assert_optimized_prefer_sort_on_off { + ($EXPECTED_PLAN_LINES: expr, $EXPECTED_OPTIMIZED_PLAN_LINES: expr, $EXPECTED_PREFER_SORT_ON_OPTIMIZED_PLAN_LINES: expr, $PLAN: expr, $PREFER_EXISTING_SORT: expr, $SOURCE_UNBOUNDED: expr) => { + if $PREFER_EXISTING_SORT { + assert_optimized!( + $EXPECTED_PLAN_LINES, + $EXPECTED_PREFER_SORT_ON_OPTIMIZED_PLAN_LINES, + $PLAN, + $PREFER_EXISTING_SORT, + $SOURCE_UNBOUNDED + ); + } else { + assert_optimized!( + $EXPECTED_PLAN_LINES, + $EXPECTED_OPTIMIZED_PLAN_LINES, + $PLAN, + $PREFER_EXISTING_SORT, + $SOURCE_UNBOUNDED + ); + } + }; +} + /// Runs the `replace_with_order_preserving_variants` sub-rule and asserts /// the plan against the original and expected plans for both bounded and /// unbounded cases. @@ -93,39 +122,6 @@ macro_rules! assert_optimized_in_all_boundedness_situations { }; } -/// Runs the `replace_with_order_preserving_variants` sub-rule and asserts -/// the plan against the original and expected plans. -/// -/// # Parameters -/// -/// * `$EXPECTED_PLAN_LINES`: Expected input plan. -/// * `EXPECTED_OPTIMIZED_PLAN_LINES`: Optimized plan when the flag -/// `prefer_existing_sort` is `false`. -/// * `EXPECTED_PREFER_SORT_ON_OPTIMIZED_PLAN_LINES`: Optimized plan when -/// the flag `prefer_existing_sort` is `true`. -/// * `$PLAN`: The plan to optimize. -macro_rules! assert_optimized_prefer_sort_on_off { - ($EXPECTED_PLAN_LINES: expr, $EXPECTED_OPTIMIZED_PLAN_LINES: expr, $EXPECTED_PREFER_SORT_ON_OPTIMIZED_PLAN_LINES: expr, $PLAN: expr, $PREFER_EXISTING_SORT: expr, $SOURCE_UNBOUNDED: expr) => { - if $PREFER_EXISTING_SORT { - assert_optimized!( - $EXPECTED_PLAN_LINES, - $EXPECTED_PREFER_SORT_ON_OPTIMIZED_PLAN_LINES, - $PLAN, - $PREFER_EXISTING_SORT, - $SOURCE_UNBOUNDED - ); - } else { - assert_optimized!( - $EXPECTED_PLAN_LINES, - $EXPECTED_OPTIMIZED_PLAN_LINES, - $PLAN, - $PREFER_EXISTING_SORT, - $SOURCE_UNBOUNDED - ); - } - }; -} - /// Runs the `replace_with_order_preserving_variants` sub-rule and asserts /// the plan against the original and expected plans. /// @@ -153,9 +149,10 @@ macro_rules! assert_optimized { let expected_optimized_lines: Vec<&str> = $EXPECTED_OPTIMIZED_PLAN_LINES.iter().map(|s| *s).collect(); // Run the rule top-down - let config = SessionConfig::new().with_prefer_existing_sort($PREFER_EXISTING_SORT); + let mut config = ConfigOptions::new(); + config.optimizer.prefer_existing_sort=$PREFER_EXISTING_SORT; let plan_with_pipeline_fixer = OrderPreservationContext::new_default(physical_plan); - let parallel = plan_with_pipeline_fixer.transform_up(|plan_with_pipeline_fixer| replace_with_order_preserving_variants(plan_with_pipeline_fixer, false, false, config.options())).data().and_then(check_integrity)?; + let parallel = plan_with_pipeline_fixer.transform_up(|plan_with_pipeline_fixer| replace_with_order_preserving_variants(plan_with_pipeline_fixer, false, false, &config)).data().and_then(check_integrity)?; let optimized_physical_plan = parallel.plan; // Get string representation of the plan @@ -1121,9 +1118,6 @@ async fn test_with_multiple_child_trees( Ok(()) } -// End test cases -// Start test helpers - fn sort_expr(name: &str, schema: &Schema) -> PhysicalSortExpr { let sort_opts = SortOptions { nulls_first: false, diff --git a/datafusion/core/tests/physical_optimizer/sanity_checker.rs b/datafusion/core/tests/physical_optimizer/sanity_checker.rs index 538f0e443ddb..7636df9b3db9 100644 --- a/datafusion/core/tests/physical_optimizer/sanity_checker.rs +++ b/datafusion/core/tests/physical_optimizer/sanity_checker.rs @@ -15,58 +15,128 @@ // specific language governing permissions and limitations // under the License. -//! Tests for [`SanityCheckPlan`] physical optimizer rule -//! -//! Note these tests are not in the same module as the optimizer pass because -//! they rely on `ParquetExec` which is in the core crate. - -use crate::physical_optimizer::test_util::{ - BinaryTestCase, QueryCase, SourceType, UnaryTestCase, -}; -use arrow::compute::SortOptions; -use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; -use datafusion_common::config::ConfigOptions; -use datafusion_common::Result; -use datafusion_expr::JoinType; -use datafusion_physical_expr::expressions::col; -use datafusion_physical_expr::Partitioning; -use datafusion_physical_optimizer::test_utils::{ - bounded_window_exec, global_limit_exec, local_limit_exec, memory_exec, - repartition_exec, sort_exec, sort_expr_options, sort_merge_join_exec, -}; -use datafusion_physical_optimizer::{sanity_checker::*, PhysicalOptimizerRule}; -use datafusion_physical_plan::displayable; -use datafusion_physical_plan::repartition::RepartitionExec; -use datafusion_physical_plan::ExecutionPlan; use std::sync::Arc; -fn create_test_schema() -> SchemaRef { - Arc::new(Schema::new(vec![Field::new("c9", DataType::Int32, true)])) +use datafusion::datasource::stream::{FileStreamProvider, StreamConfig, StreamTable}; +use datafusion::prelude::{CsvReadOptions, SessionContext}; +use datafusion_common::Result; + +use async_trait::async_trait; + +async fn register_current_csv( + ctx: &SessionContext, + table_name: &str, + infinite: bool, +) -> Result<()> { + let testdata = datafusion::test_util::arrow_test_data(); + let schema = datafusion::test_util::aggr_test_schema(); + let path = format!("{testdata}/csv/aggregate_test_100.csv"); + + match infinite { + true => { + let source = FileStreamProvider::new_file(schema, path.into()); + let config = StreamConfig::new(Arc::new(source)); + ctx.register_table(table_name, Arc::new(StreamTable::new(Arc::new(config))))?; + } + false => { + ctx.register_csv(table_name, &path, CsvReadOptions::new().schema(&schema)) + .await?; + } + } + + Ok(()) +} + +#[derive(Eq, PartialEq, Debug)] +pub enum SourceType { + Unbounded, + Bounded, } -fn create_test_schema2() -> SchemaRef { - Arc::new(Schema::new(vec![ - Field::new("a", DataType::Int32, true), - Field::new("b", DataType::Int32, true), - ])) +#[async_trait] +pub trait SqlTestCase { + async fn register_table(&self, ctx: &SessionContext) -> Result<()>; + fn expect_fail(&self) -> bool; } -/// Check if sanity checker should accept or reject plans. -fn assert_sanity_check(plan: &Arc, is_sane: bool) { - let sanity_checker = SanityCheckPlan::new(); - let opts = ConfigOptions::default(); - assert_eq!( - sanity_checker.optimize(plan.clone(), &opts).is_ok(), - is_sane - ); +/// [UnaryTestCase] is designed for single input [ExecutionPlan]s. +pub struct UnaryTestCase { + pub source_type: SourceType, + pub expect_fail: bool, } -/// Check if the plan we created is as expected by comparing the plan -/// formatted as a string. -fn assert_plan(plan: &dyn ExecutionPlan, expected_lines: Vec<&str>) { - let plan_str = displayable(plan).indent(true).to_string(); - let actual_lines: Vec<&str> = plan_str.trim().lines().collect(); - assert_eq!(actual_lines, expected_lines); +#[async_trait] +impl SqlTestCase for UnaryTestCase { + async fn register_table(&self, ctx: &SessionContext) -> Result<()> { + let table_is_infinite = self.source_type == SourceType::Unbounded; + register_current_csv(ctx, "test", table_is_infinite).await?; + Ok(()) + } + + fn expect_fail(&self) -> bool { + self.expect_fail + } +} + +/// [BinaryTestCase] is designed for binary input [ExecutionPlan]s. +pub struct BinaryTestCase { + pub source_types: (SourceType, SourceType), + pub expect_fail: bool, +} + +#[async_trait] +impl SqlTestCase for BinaryTestCase { + async fn register_table(&self, ctx: &SessionContext) -> Result<()> { + let left_table_is_infinite = self.source_types.0 == SourceType::Unbounded; + let right_table_is_infinite = self.source_types.1 == SourceType::Unbounded; + register_current_csv(ctx, "left", left_table_is_infinite).await?; + register_current_csv(ctx, "right", right_table_is_infinite).await?; + Ok(()) + } + + fn expect_fail(&self) -> bool { + self.expect_fail + } +} + +pub struct QueryCase { + pub sql: String, + pub cases: Vec>, + pub error_operator: String, +} + +impl QueryCase { + /// Run the test cases + pub async fn run(&self) -> Result<()> { + for case in &self.cases { + let ctx = SessionContext::new(); + case.register_table(&ctx).await?; + let error = if case.expect_fail() { + Some(&self.error_operator) + } else { + None + }; + self.run_case(ctx, error).await?; + } + Ok(()) + } + + async fn run_case(&self, ctx: SessionContext, error: Option<&String>) -> Result<()> { + let dataframe = ctx.sql(self.sql.as_str()).await?; + let plan = dataframe.create_physical_plan().await; + if let Some(error) = error { + let plan_error = plan.unwrap_err(); + assert!( + plan_error.to_string().contains(error.as_str()), + "plan_error: {:?} doesn't contain message: {:?}", + plan_error, + error.as_str() + ); + } else { + assert!(plan.is_ok()) + } + Ok(()) + } } #[tokio::test] @@ -291,246 +361,3 @@ async fn test_analyzer() -> Result<()> { case.run().await?; Ok(()) } - -#[tokio::test] -/// Tests that plan is valid when the sort requirements are satisfied. -async fn test_bounded_window_agg_sort_requirement() -> Result<()> { - let schema = create_test_schema(); - let source = memory_exec(&schema); - let sort_exprs = vec![sort_expr_options( - "c9", - &source.schema(), - SortOptions { - descending: false, - nulls_first: false, - }, - )]; - let sort = sort_exec(sort_exprs.clone(), source); - let bw = bounded_window_exec("c9", sort_exprs, sort); - assert_plan(bw.as_ref(), vec![ - "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=[c9@0 ASC NULLS LAST], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]" - ]); - assert_sanity_check(&bw, true); - Ok(()) -} - -#[tokio::test] -/// Tests that plan is invalid when the sort requirements are not satisfied. -async fn test_bounded_window_agg_no_sort_requirement() -> Result<()> { - let schema = create_test_schema(); - let source = memory_exec(&schema); - let sort_exprs = vec![sort_expr_options( - "c9", - &source.schema(), - SortOptions { - descending: false, - nulls_first: false, - }, - )]; - let bw = bounded_window_exec("c9", sort_exprs, source); - assert_plan(bw.as_ref(), vec![ - "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]" - ]); - // Order requirement of the `BoundedWindowAggExec` is not satisfied. We expect to receive error during sanity check. - assert_sanity_check(&bw, false); - Ok(()) -} - -#[tokio::test] -/// A valid when a single partition requirement -/// is satisfied. -async fn test_global_limit_single_partition() -> Result<()> { - let schema = create_test_schema(); - let source = memory_exec(&schema); - let limit = global_limit_exec(source); - - assert_plan( - limit.as_ref(), - vec![ - "GlobalLimitExec: skip=0, fetch=100", - " MemoryExec: partitions=1, partition_sizes=[0]", - ], - ); - assert_sanity_check(&limit, true); - Ok(()) -} - -#[tokio::test] -/// An invalid plan when a single partition requirement -/// is not satisfied. -async fn test_global_limit_multi_partition() -> Result<()> { - let schema = create_test_schema(); - let source = memory_exec(&schema); - let limit = global_limit_exec(repartition_exec(source)); - - assert_plan( - limit.as_ref(), - vec![ - "GlobalLimitExec: skip=0, fetch=100", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[0]", - ], - ); - // Distribution requirement of the `GlobalLimitExec` is not satisfied. We expect to receive error during sanity check. - assert_sanity_check(&limit, false); - Ok(()) -} - -#[tokio::test] -/// A plan with no requirements should satisfy. -async fn test_local_limit() -> Result<()> { - let schema = create_test_schema(); - let source = memory_exec(&schema); - let limit = local_limit_exec(source); - - assert_plan( - limit.as_ref(), - vec![ - "LocalLimitExec: fetch=100", - " MemoryExec: partitions=1, partition_sizes=[0]", - ], - ); - assert_sanity_check(&limit, true); - Ok(()) -} - -#[tokio::test] -/// Valid plan with multiple children satisfy both order and distribution. -async fn test_sort_merge_join_satisfied() -> Result<()> { - let schema1 = create_test_schema(); - let schema2 = create_test_schema2(); - let source1 = memory_exec(&schema1); - let source2 = memory_exec(&schema2); - let sort_opts = SortOptions::default(); - let sort_exprs1 = vec![sort_expr_options("c9", &source1.schema(), sort_opts)]; - 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 left_jcol = col("c9", &left.schema()).unwrap(); - let right_jcol = col("a", &right.schema()).unwrap(); - let left = Arc::new(RepartitionExec::try_new( - left, - Partitioning::Hash(vec![left_jcol.clone()], 10), - )?); - - let right = Arc::new(RepartitionExec::try_new( - right, - Partitioning::Hash(vec![right_jcol.clone()], 10), - )?); - - let join_on = vec![(left_jcol as _, right_jcol as _)]; - let join_ty = JoinType::Inner; - let smj = sort_merge_join_exec(left, right, &join_on, &join_ty); - - assert_plan( - smj.as_ref(), - vec![ - "SortMergeJoin: join_type=Inner, on=[(c9@0, a@0)]", - " 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=Hash([a@0], 10), input_partitions=1", - " SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", - ], - ); - assert_sanity_check(&smj, true); - Ok(()) -} - -#[tokio::test] -/// Invalid case when the order is not satisfied by the 2nd -/// child. -async fn test_sort_merge_join_order_missing() -> Result<()> { - let schema1 = create_test_schema(); - let schema2 = create_test_schema2(); - let source1 = memory_exec(&schema1); - let right = memory_exec(&schema2); - let sort_exprs1 = vec![sort_expr_options( - "c9", - &source1.schema(), - SortOptions::default(), - )]; - let left = sort_exec(sort_exprs1, source1); - // Missing sort of the right child here.. - let left_jcol = col("c9", &left.schema()).unwrap(); - let right_jcol = col("a", &right.schema()).unwrap(); - let left = Arc::new(RepartitionExec::try_new( - left, - Partitioning::Hash(vec![left_jcol.clone()], 10), - )?); - - let right = Arc::new(RepartitionExec::try_new( - right, - Partitioning::Hash(vec![right_jcol.clone()], 10), - )?); - - let join_on = vec![(left_jcol as _, right_jcol as _)]; - let join_ty = JoinType::Inner; - let smj = sort_merge_join_exec(left, right, &join_on, &join_ty); - - assert_plan( - smj.as_ref(), - vec![ - "SortMergeJoin: join_type=Inner, on=[(c9@0, a@0)]", - " 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=Hash([a@0], 10), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[0]", - ], - ); - // Order requirement for the `SortMergeJoin` is not satisfied for right child. We expect to receive error during sanity check. - assert_sanity_check(&smj, false); - Ok(()) -} - -#[tokio::test] -/// Invalid case when the distribution is not satisfied by the 2nd -/// child. -async fn test_sort_merge_join_dist_missing() -> Result<()> { - let schema1 = create_test_schema(); - let schema2 = create_test_schema2(); - let source1 = memory_exec(&schema1); - let source2 = memory_exec(&schema2); - let sort_opts = SortOptions::default(); - let sort_exprs1 = vec![sort_expr_options("c9", &source1.schema(), sort_opts)]; - 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( - right, - Partitioning::RoundRobinBatch(10), - )?); - let left_jcol = col("c9", &left.schema()).unwrap(); - let right_jcol = col("a", &right.schema()).unwrap(); - let left = Arc::new(RepartitionExec::try_new( - left, - Partitioning::Hash(vec![left_jcol.clone()], 10), - )?); - - // Missing hash partitioning on right child. - - let join_on = vec![(left_jcol as _, right_jcol as _)]; - let join_ty = JoinType::Inner; - let smj = sort_merge_join_exec(left, right, &join_on, &join_ty); - - assert_plan( - smj.as_ref(), - vec![ - "SortMergeJoin: join_type=Inner, on=[(c9@0, a@0)]", - " 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", - " SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", - ], - ); - // Distribution requirement for the `SortMergeJoin` is not satisfied for right child (has round-robin partitioning). We expect to receive error during sanity check. - assert_sanity_check(&smj, false); - Ok(()) -} diff --git a/datafusion/core/tests/physical_optimizer/test_util.rs b/datafusion/core/tests/physical_optimizer/test_util.rs deleted file mode 100644 index 3c3da36199b1..000000000000 --- a/datafusion/core/tests/physical_optimizer/test_util.rs +++ /dev/null @@ -1,319 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -//! Test utilities for physical optimizer tests - -use std::sync::Arc; - -use async_trait::async_trait; -use datafusion::datasource::stream::{FileStreamProvider, StreamConfig, StreamTable}; -use datafusion::error::Result; -use datafusion::prelude::{CsvReadOptions, SessionContext}; - -use arrow_schema::{DataType, Field, Schema, SchemaRef}; -use datafusion::datasource::file_format::file_compression_type::FileCompressionType; -use datafusion::datasource::physical_plan::CsvExec; -use datafusion::datasource::{ - listing::PartitionedFile, - physical_plan::{FileScanConfig, ParquetExec}, -}; -use datafusion_execution::object_store::ObjectStoreUrl; -use datafusion_execution::{SendableRecordBatchStream, TaskContext}; -use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; -use datafusion_physical_plan::streaming::{PartitionStream, StreamingTableExec}; -use datafusion_physical_plan::ExecutionPlan; - -/// create a single parquet file that is sorted -pub(crate) fn parquet_exec_with_sort( - output_ordering: Vec, -) -> Arc { - ParquetExec::builder( - FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema()) - .with_file(PartitionedFile::new("x".to_string(), 100)) - .with_output_ordering(output_ordering), - ) - .build_arc() -} - -pub(crate) fn schema() -> SchemaRef { - Arc::new(Schema::new(vec![ - Field::new("a", DataType::Int64, true), - Field::new("b", DataType::Int64, true), - Field::new("c", DataType::Int64, true), - Field::new("d", DataType::Int32, true), - Field::new("e", DataType::Boolean, true), - ])) -} - -pub(crate) fn trim_plan_display(plan: &str) -> Vec<&str> { - plan.split('\n') - .map(|s| s.trim()) - .filter(|s| !s.is_empty()) - .collect() -} - -async fn register_current_csv( - ctx: &SessionContext, - table_name: &str, - infinite: bool, -) -> Result<()> { - let testdata = datafusion::test_util::arrow_test_data(); - let schema = datafusion::test_util::aggr_test_schema(); - let path = format!("{testdata}/csv/aggregate_test_100.csv"); - - match infinite { - true => { - let source = FileStreamProvider::new_file(schema, path.into()); - let config = StreamConfig::new(Arc::new(source)); - ctx.register_table(table_name, Arc::new(StreamTable::new(Arc::new(config))))?; - } - false => { - ctx.register_csv(table_name, &path, CsvReadOptions::new().schema(&schema)) - .await?; - } - } - - Ok(()) -} - -#[derive(Eq, PartialEq, Debug)] -pub enum SourceType { - Unbounded, - Bounded, -} - -#[async_trait] -pub trait SqlTestCase { - async fn register_table(&self, ctx: &SessionContext) -> Result<()>; - fn expect_fail(&self) -> bool; -} - -/// [UnaryTestCase] is designed for single input [ExecutionPlan]s. -pub struct UnaryTestCase { - pub source_type: SourceType, - pub expect_fail: bool, -} - -#[async_trait] -impl SqlTestCase for UnaryTestCase { - async fn register_table(&self, ctx: &SessionContext) -> Result<()> { - let table_is_infinite = self.source_type == SourceType::Unbounded; - register_current_csv(ctx, "test", table_is_infinite).await?; - Ok(()) - } - - fn expect_fail(&self) -> bool { - self.expect_fail - } -} -/// [BinaryTestCase] is designed for binary input [ExecutionPlan]s. -pub struct BinaryTestCase { - pub source_types: (SourceType, SourceType), - pub expect_fail: bool, -} - -#[async_trait] -impl SqlTestCase for BinaryTestCase { - async fn register_table(&self, ctx: &SessionContext) -> Result<()> { - let left_table_is_infinite = self.source_types.0 == SourceType::Unbounded; - let right_table_is_infinite = self.source_types.1 == SourceType::Unbounded; - register_current_csv(ctx, "left", left_table_is_infinite).await?; - register_current_csv(ctx, "right", right_table_is_infinite).await?; - Ok(()) - } - - fn expect_fail(&self) -> bool { - self.expect_fail - } -} - -pub struct QueryCase { - pub sql: String, - pub cases: Vec>, - pub error_operator: String, -} - -impl QueryCase { - /// Run the test cases - pub async fn run(&self) -> Result<()> { - for case in &self.cases { - let ctx = SessionContext::new(); - case.register_table(&ctx).await?; - let error = if case.expect_fail() { - Some(&self.error_operator) - } else { - None - }; - self.run_case(ctx, error).await?; - } - Ok(()) - } - async fn run_case(&self, ctx: SessionContext, error: Option<&String>) -> Result<()> { - let dataframe = ctx.sql(self.sql.as_str()).await?; - let plan = dataframe.create_physical_plan().await; - if let Some(error) = error { - let plan_error = plan.unwrap_err(); - assert!( - plan_error.to_string().contains(error.as_str()), - "plan_error: {:?} doesn't contain message: {:?}", - plan_error, - error.as_str() - ); - } else { - assert!(plan.is_ok()) - } - Ok(()) - } -} - -/// Create a non sorted parquet exec -pub fn parquet_exec(schema: &SchemaRef) -> Arc { - ParquetExec::builder( - FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema.clone()) - .with_file(PartitionedFile::new("x".to_string(), 100)), - ) - .build_arc() -} - -// Created a sorted parquet exec -pub fn parquet_exec_sorted( - schema: &SchemaRef, - sort_exprs: impl IntoIterator, -) -> Arc { - let sort_exprs = sort_exprs.into_iter().collect(); - - ParquetExec::builder( - FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema.clone()) - .with_file(PartitionedFile::new("x".to_string(), 100)) - .with_output_ordering(vec![sort_exprs]), - ) - .build_arc() -} - -/// Created a sorted Csv exec -pub fn csv_exec_sorted( - schema: &SchemaRef, - sort_exprs: impl IntoIterator, -) -> Arc { - let sort_exprs = sort_exprs.into_iter().collect(); - - Arc::new( - CsvExec::builder( - FileScanConfig::new( - ObjectStoreUrl::parse("test:///").unwrap(), - schema.clone(), - ) - .with_file(PartitionedFile::new("x".to_string(), 100)) - .with_output_ordering(vec![sort_exprs]), - ) - .with_has_header(false) - .with_delimeter(0) - .with_quote(0) - .with_escape(None) - .with_comment(None) - .with_newlines_in_values(false) - .with_file_compression_type(FileCompressionType::UNCOMPRESSED) - .build(), - ) -} - -// construct a stream partition for test purposes -#[derive(Debug)] -pub(crate) struct TestStreamPartition { - pub schema: SchemaRef, -} - -impl PartitionStream for TestStreamPartition { - fn schema(&self) -> &SchemaRef { - &self.schema - } - fn execute(&self, _ctx: Arc) -> SendableRecordBatchStream { - unreachable!() - } -} - -/// Create an unbounded stream exec -pub fn stream_exec_ordered( - schema: &SchemaRef, - sort_exprs: impl IntoIterator, -) -> Arc { - let sort_exprs = sort_exprs.into_iter().collect(); - - Arc::new( - StreamingTableExec::try_new( - schema.clone(), - vec![Arc::new(TestStreamPartition { - schema: schema.clone(), - }) as _], - None, - vec![sort_exprs], - true, - None, - ) - .unwrap(), - ) -} - -// Creates a stream exec source for the test purposes -pub fn stream_exec_ordered_with_projection( - schema: &SchemaRef, - sort_exprs: impl IntoIterator, -) -> Arc { - let sort_exprs = sort_exprs.into_iter().collect(); - let projection: Vec = vec![0, 2, 3]; - - Arc::new( - StreamingTableExec::try_new( - schema.clone(), - vec![Arc::new(TestStreamPartition { - schema: schema.clone(), - }) as _], - Some(&projection), - vec![sort_exprs], - true, - None, - ) - .unwrap(), - ) -} - -/// Create a csv exec for tests -pub fn csv_exec_ordered( - schema: &SchemaRef, - sort_exprs: impl IntoIterator, -) -> Arc { - let sort_exprs = sort_exprs.into_iter().collect(); - - Arc::new( - CsvExec::builder( - FileScanConfig::new( - ObjectStoreUrl::parse("test:///").unwrap(), - schema.clone(), - ) - .with_file(PartitionedFile::new("file_path".to_string(), 100)) - .with_output_ordering(vec![sort_exprs]), - ) - .with_has_header(true) - .with_delimeter(0) - .with_quote(b'"') - .with_escape(None) - .with_comment(None) - .with_newlines_in_values(false) - .with_file_compression_type(FileCompressionType::UNCOMPRESSED) - .build(), - ) -} diff --git a/datafusion/physical-optimizer/Cargo.toml b/datafusion/physical-optimizer/Cargo.toml index c6c75a0e0939..a40827bda209 100644 --- a/datafusion/physical-optimizer/Cargo.toml +++ b/datafusion/physical-optimizer/Cargo.toml @@ -41,6 +41,7 @@ datafusion-common = { workspace = true, default-features = true } datafusion-execution = { workspace = true } datafusion-expr = { workspace = true } datafusion-expr-common = { workspace = true, default-features = true } +datafusion-functions-aggregate = { workspace = true } datafusion-physical-expr = { workspace = true } datafusion-physical-expr-common = { workspace = true } datafusion-physical-plan = { workspace = true } @@ -52,7 +53,6 @@ url = { workspace = true } [dev-dependencies] datafusion-expr = { workspace = true } -datafusion-functions-aggregate = { workspace = true } datafusion-functions-nested = { workspace = true } rstest = { workspace = true } tokio = { workspace = true } diff --git a/datafusion/physical-optimizer/src/aggregate_statistics.rs b/datafusion/physical-optimizer/src/aggregate_statistics.rs index a00bc4b1d571..68bfd63b9a19 100644 --- a/datafusion/physical-optimizer/src/aggregate_statistics.rs +++ b/datafusion/physical-optimizer/src/aggregate_statistics.rs @@ -152,14 +152,9 @@ mod tests { use crate::aggregate_statistics::AggregateStatistics; use crate::PhysicalOptimizerRule; use datafusion_common::config::ConfigOptions; - use datafusion_common::utils::expr::COUNT_STAR_EXPANSION; use datafusion_execution::TaskContext; - use datafusion_functions_aggregate::count::count_udaf; - use datafusion_physical_expr::aggregate::AggregateExprBuilder; - use datafusion_physical_expr::PhysicalExpr; use datafusion_physical_plan::aggregates::AggregateExec; use datafusion_physical_plan::projection::ProjectionExec; - use datafusion_physical_plan::udaf::AggregateFunctionExpr; use datafusion_physical_plan::ExecutionPlan; use std::sync::Arc; @@ -172,6 +167,7 @@ mod tests { use datafusion_physical_plan::filter::FilterExec; use datafusion_physical_plan::memory::MemoryExec; + use crate::test_utils::TestAggregate; use arrow::array::Int32Array; use arrow::datatypes::{DataType, Field, Schema}; use arrow::record_batch::RecordBatch; @@ -179,60 +175,6 @@ mod tests { use datafusion_physical_expr::expressions::{self, cast}; use datafusion_physical_plan::aggregates::AggregateMode; - /// Describe the type of aggregate being tested - pub enum TestAggregate { - /// Testing COUNT(*) type aggregates - CountStar, - - /// Testing for COUNT(column) aggregate - ColumnA(Arc), - } - - impl TestAggregate { - /// Create a new COUNT(*) aggregate - pub fn new_count_star() -> Self { - Self::CountStar - } - - /// Create a new COUNT(column) aggregate - pub fn new_count_column(schema: &Arc) -> Self { - Self::ColumnA(Arc::clone(schema)) - } - - /// Return appropriate expr depending if COUNT is for col or table (*) - pub fn count_expr(&self, schema: &Schema) -> AggregateFunctionExpr { - AggregateExprBuilder::new(count_udaf(), vec![self.column()]) - .schema(Arc::new(schema.clone())) - .alias(self.column_name()) - .build() - .unwrap() - } - - /// what argument would this aggregate need in the plan? - fn column(&self) -> Arc { - match self { - Self::CountStar => expressions::lit(COUNT_STAR_EXPANSION), - Self::ColumnA(s) => expressions::col("a", s).unwrap(), - } - } - - /// What name would this aggregate produce in a plan? - pub fn column_name(&self) -> &'static str { - match self { - Self::CountStar => "COUNT(*)", - Self::ColumnA(_) => "COUNT(a)", - } - } - - /// What is the expected count? - pub fn expected_count(&self) -> i64 { - match self { - TestAggregate::CountStar => 3, - TestAggregate::ColumnA(_) => 2, - } - } - } - /// Mock data using a MemoryExec which has an exact count statistic fn mock_data() -> Result> { let schema = Arc::new(Schema::new(vec![ diff --git a/datafusion/physical-optimizer/src/enforce_distribution.rs b/datafusion/physical-optimizer/src/enforce_distribution.rs new file mode 100644 index 000000000000..2f37c672bcda --- /dev/null +++ b/datafusion/physical-optimizer/src/enforce_distribution.rs @@ -0,0 +1,1407 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +//! EnforceDistribution optimizer rule inspects the physical plan with respect +//! to distribution requirements and adds [`RepartitionExec`]s to satisfy them +//! when necessary. If increasing parallelism is beneficial (and also desirable +//! according to the configuration), this rule increases partition counts in +//! the physical plan. + +use std::fmt::Debug; +use std::sync::Arc; + +use crate::optimizer::PhysicalOptimizerRule; +use crate::output_requirements::OutputRequirementExec; +use crate::utils::{ + add_sort_above_with_check, is_coalesce_partitions, is_repartition, + is_sort_preserving_merge, +}; + +use arrow::compute::SortOptions; +use datafusion_common::config::ConfigOptions; +use datafusion_common::error::Result; +use datafusion_common::stats::Precision; +use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; +use datafusion_expr::logical_plan::JoinType; +use datafusion_physical_expr::expressions::{Column, NoOp}; +use datafusion_physical_expr::utils::map_columns_before_projection; +use datafusion_physical_expr::{ + physical_exprs_equal, EquivalenceProperties, PhysicalExpr, PhysicalExprRef, +}; +use datafusion_physical_expr_common::sort_expr::LexOrdering; +use datafusion_physical_plan::aggregates::{ + AggregateExec, AggregateMode, PhysicalGroupBy, +}; +use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; +use datafusion_physical_plan::execution_plan::EmissionType; +use datafusion_physical_plan::joins::{ + CrossJoinExec, HashJoinExec, PartitionMode, SortMergeJoinExec, +}; +use datafusion_physical_plan::projection::ProjectionExec; +use datafusion_physical_plan::repartition::RepartitionExec; +use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; +use datafusion_physical_plan::tree_node::PlanContext; +use datafusion_physical_plan::union::{can_interleave, InterleaveExec, UnionExec}; +use datafusion_physical_plan::windows::WindowAggExec; +use datafusion_physical_plan::windows::{get_best_fitting_window, BoundedWindowAggExec}; +use datafusion_physical_plan::ExecutionPlanProperties; +use datafusion_physical_plan::{Distribution, ExecutionPlan, Partitioning}; + +use itertools::izip; + +/// 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. +/// +/// For example, given an input such as: +/// +/// +/// ```text +/// ┌─────────────────────────────────┐ +/// │ │ +/// │ ExecutionPlan │ +/// │ │ +/// └─────────────────────────────────┘ +/// ▲ ▲ +/// │ │ +/// ┌─────┘ └─────┐ +/// │ │ +/// │ │ +/// │ │ +/// ┌───────────┐ ┌───────────┐ +/// │ │ │ │ +/// │ batch A1 │ │ batch B1 │ +/// │ │ │ │ +/// ├───────────┤ ├───────────┤ +/// │ │ │ │ +/// │ batch A2 │ │ batch B2 │ +/// │ │ │ │ +/// ├───────────┤ ├───────────┤ +/// │ │ │ │ +/// │ batch A3 │ │ batch B3 │ +/// │ │ │ │ +/// └───────────┘ └───────────┘ +/// +/// Input Input +/// A B +/// ``` +/// +/// This rule will attempt to add a `RepartitionExec` to increase parallelism +/// (to 3, in this case) and create the following arrangement: +/// +/// ```text +/// ┌─────────────────────────────────┐ +/// │ │ +/// │ ExecutionPlan │ +/// │ │ +/// └─────────────────────────────────┘ +/// ▲ ▲ ▲ Input now has 3 +/// │ │ │ partitions +/// ┌───────┘ │ └───────┐ +/// │ │ │ +/// │ │ │ +/// ┌───────────┐ ┌───────────┐ ┌───────────┐ +/// │ │ │ │ │ │ +/// │ batch A1 │ │ batch A3 │ │ batch B3 │ +/// │ │ │ │ │ │ +/// ├───────────┤ ├───────────┤ ├───────────┤ +/// │ │ │ │ │ │ +/// │ batch B2 │ │ batch B1 │ │ batch A2 │ +/// │ │ │ │ │ │ +/// └───────────┘ └───────────┘ └───────────┘ +/// ▲ ▲ ▲ +/// │ │ │ +/// └─────────┐ │ ┌──────────┘ +/// │ │ │ +/// │ │ │ +/// ┌─────────────────────────────────┐ batches are +/// │ RepartitionExec(3) │ repartitioned +/// │ RoundRobin │ +/// │ │ +/// └─────────────────────────────────┘ +/// ▲ ▲ +/// │ │ +/// ┌─────┘ └─────┐ +/// │ │ +/// │ │ +/// │ │ +/// ┌───────────┐ ┌───────────┐ +/// │ │ │ │ +/// │ batch A1 │ │ batch B1 │ +/// │ │ │ │ +/// ├───────────┤ ├───────────┤ +/// │ │ │ │ +/// │ batch A2 │ │ batch B2 │ +/// │ │ │ │ +/// ├───────────┤ ├───────────┤ +/// │ │ │ │ +/// │ batch A3 │ │ batch B3 │ +/// │ │ │ │ +/// └───────────┘ └───────────┘ +/// +/// +/// Input Input +/// A B +/// ``` +/// +/// The `EnforceDistribution` rule +/// - is idempotent; i.e. it can be applied multiple times, each time producing +/// the same result. +/// - always produces a valid plan in terms of distribution requirements. Its +/// input plan can be valid or invalid with respect to distribution requirements, +/// but the output plan will always be valid. +/// - produces a valid plan in terms of ordering requirements, *if* its input is +/// a valid plan in terms of ordering requirements. If the input plan is invalid, +/// this rule does not attempt to fix it as doing so is the responsibility of the +/// `EnforceSorting` rule. +/// +/// Note that distribution requirements are met in the strictest way. This may +/// result in more than strictly necessary [`RepartitionExec`]s in the plan, but +/// meeting the requirements in the strictest way may help avoid possible data +/// skew in joins. +/// +/// For example for a hash join with keys (a, b, c), the required Distribution(a, b, c) +/// can be satisfied by several alternative partitioning ways: (a, b, c), (a, b), +/// (a, c), (b, c), (a), (b), (c) and ( ). +/// +/// This rule only chooses the exact match and satisfies the Distribution(a, b, c) +/// by a HashPartition(a, b, c). +#[derive(Default, Debug)] +pub struct EnforceDistribution {} + +impl EnforceDistribution { + #[allow(missing_docs)] + pub fn new() -> Self { + Self {} + } +} + +impl PhysicalOptimizerRule for EnforceDistribution { + fn optimize( + &self, + plan: Arc, + config: &ConfigOptions, + ) -> Result> { + let top_down_join_key_reordering = config.optimizer.top_down_join_key_reordering; + + let adjusted = if top_down_join_key_reordering { + // Run a top-down process to adjust input key ordering recursively + let plan_requirements = PlanWithKeyRequirements::new_default(plan); + let adjusted = plan_requirements + .transform_down(adjust_input_keys_ordering) + .data()?; + adjusted.plan + } else { + // Run a bottom-up process + plan.transform_up(|plan| { + Ok(Transformed::yes(reorder_join_keys_to_inputs(plan)?)) + }) + .data()? + }; + + let distribution_context = DistributionContext::new_default(adjusted); + // Distribution enforcement needs to be applied bottom-up. + let distribution_context = distribution_context + .transform_up(|distribution_context| { + ensure_distribution(distribution_context, config) + }) + .data()?; + Ok(distribution_context.plan) + } + + fn name(&self) -> &str { + "EnforceDistribution" + } + + fn schema_check(&self) -> bool { + true + } +} + +#[derive(Debug, Clone)] +struct JoinKeyPairs { + left_keys: Vec>, + right_keys: Vec>, +} + +/// Keeps track of parent required key orderings. +pub type PlanWithKeyRequirements = PlanContext>>; + +/// When the physical planner creates the Joins, the ordering of join keys is from the original query. +/// That might not match with the output partitioning of the join node's children +/// A Top-Down process will use this method to adjust children's output partitioning based on the parent key reordering requirements: +/// +/// Example: +/// TopJoin on (a, b, c) +/// bottom left join on(b, a, c) +/// bottom right join on(c, b, a) +/// +/// Will be adjusted to: +/// TopJoin on (a, b, c) +/// bottom left join on(a, b, c) +/// bottom right join on(a, b, c) +/// +/// Example: +/// TopJoin on (a, b, c) +/// Agg1 group by (b, a, c) +/// Agg2 group by (c, b, a) +/// +/// Will be adjusted to: +/// TopJoin on (a, b, c) +/// Projection(b, a, c) +/// Agg1 group by (a, b, c) +/// Projection(c, b, a) +/// Agg2 group by (a, b, c) +/// +/// Following is the explanation of the reordering process: +/// +/// 1) If the current plan is Partitioned HashJoin, SortMergeJoin, check whether the requirements can be satisfied by adjusting join keys ordering: +/// Requirements can not be satisfied, clear the current requirements, generate new requirements(to pushdown) based on the current join keys, return the unchanged plan. +/// Requirements is already satisfied, clear the current requirements, generate new requirements(to pushdown) based on the current join keys, return the unchanged plan. +/// Requirements can be satisfied by adjusting keys ordering, clear the current requirements, generate new requirements(to pushdown) based on the adjusted join keys, return the changed plan. +/// +/// 2) If the current plan is Aggregation, check whether the requirements can be satisfied by adjusting group by keys ordering: +/// Requirements can not be satisfied, clear all the requirements, return the unchanged plan. +/// Requirements is already satisfied, clear all the requirements, return the unchanged plan. +/// Requirements can be satisfied by adjusting keys ordering, clear all the requirements, return the changed plan. +/// +/// 3) If the current plan is RepartitionExec, CoalescePartitionsExec or WindowAggExec, clear all the requirements, return the unchanged plan +/// 4) If the current plan is Projection, transform the requirements to the columns before the Projection and push down requirements +/// 5) For other types of operators, by default, pushdown the parent requirements to children. +/// +pub fn adjust_input_keys_ordering( + mut requirements: PlanWithKeyRequirements, +) -> Result> { + let plan = Arc::clone(&requirements.plan); + + if let Some(HashJoinExec { + left, + right, + on, + filter, + join_type, + projection, + mode, + null_equals_null, + .. + }) = plan.as_any().downcast_ref::() + { + match mode { + PartitionMode::Partitioned => { + let join_constructor = |new_conditions: ( + Vec<(PhysicalExprRef, PhysicalExprRef)>, + Vec, + )| { + HashJoinExec::try_new( + Arc::clone(left), + Arc::clone(right), + new_conditions.0, + filter.clone(), + join_type, + // TODO: although projection is not used in the join here, because projection pushdown is after enforce_distribution. Maybe we need to handle it later. Same as filter. + projection.clone(), + PartitionMode::Partitioned, + *null_equals_null, + ) + .map(|e| Arc::new(e) as _) + }; + return reorder_partitioned_join_keys( + requirements, + on, + &[], + &join_constructor, + ) + .map(Transformed::yes); + } + PartitionMode::CollectLeft => { + // Push down requirements to the right side + requirements.children[1].data = match join_type { + JoinType::Inner | JoinType::Right => shift_right_required( + &requirements.data, + left.schema().fields().len(), + ) + .unwrap_or_default(), + JoinType::RightSemi | JoinType::RightAnti => { + requirements.data.clone() + } + JoinType::Left + | JoinType::LeftSemi + | JoinType::LeftAnti + | JoinType::Full + | JoinType::LeftMark => vec![], + }; + } + PartitionMode::Auto => { + // Can not satisfy, clear the current requirements and generate new empty requirements + requirements.data.clear(); + } + } + } else if let Some(CrossJoinExec { left, .. }) = + plan.as_any().downcast_ref::() + { + let left_columns_len = left.schema().fields().len(); + // Push down requirements to the right side + requirements.children[1].data = + shift_right_required(&requirements.data, left_columns_len) + .unwrap_or_default(); + } else if let Some(SortMergeJoinExec { + left, + right, + on, + filter, + join_type, + sort_options, + null_equals_null, + .. + }) = plan.as_any().downcast_ref::() + { + let join_constructor = |new_conditions: ( + Vec<(PhysicalExprRef, PhysicalExprRef)>, + Vec, + )| { + SortMergeJoinExec::try_new( + Arc::clone(left), + Arc::clone(right), + new_conditions.0, + filter.clone(), + *join_type, + new_conditions.1, + *null_equals_null, + ) + .map(|e| Arc::new(e) as _) + }; + return reorder_partitioned_join_keys( + requirements, + on, + sort_options, + &join_constructor, + ) + .map(Transformed::yes); + } else if let Some(aggregate_exec) = plan.as_any().downcast_ref::() { + if !requirements.data.is_empty() { + if aggregate_exec.mode() == &AggregateMode::FinalPartitioned { + return reorder_aggregate_keys(requirements, aggregate_exec) + .map(Transformed::yes); + } else { + requirements.data.clear(); + } + } else { + // Keep everything unchanged + return Ok(Transformed::no(requirements)); + } + } else if let Some(proj) = plan.as_any().downcast_ref::() { + let expr = proj.expr(); + // For Projection, we need to transform the requirements to the columns before the Projection + // And then to push down the requirements + // Construct a mapping from new name to the original Column + let new_required = map_columns_before_projection(&requirements.data, expr); + if new_required.len() == requirements.data.len() { + requirements.children[0].data = new_required; + } else { + // Can not satisfy, clear the current requirements and generate new empty requirements + requirements.data.clear(); + } + } else if plan.as_any().downcast_ref::().is_some() + || plan + .as_any() + .downcast_ref::() + .is_some() + || plan.as_any().downcast_ref::().is_some() + { + requirements.data.clear(); + } else { + // By default, push down the parent requirements to children + for child in requirements.children.iter_mut() { + child.data.clone_from(&requirements.data); + } + } + Ok(Transformed::yes(requirements)) +} + +pub fn reorder_partitioned_join_keys( + mut join_plan: PlanWithKeyRequirements, + on: &[(PhysicalExprRef, PhysicalExprRef)], + sort_options: &[SortOptions], + join_constructor: &F, +) -> Result +where + F: Fn( + (Vec<(PhysicalExprRef, PhysicalExprRef)>, Vec), + ) -> Result>, +{ + let parent_required = &join_plan.data; + let join_key_pairs = extract_join_keys(on); + let eq_properties = join_plan.plan.equivalence_properties(); + + let ( + JoinKeyPairs { + left_keys, + right_keys, + }, + positions, + ) = try_reorder(join_key_pairs, parent_required, eq_properties); + + if let Some(positions) = positions { + if !positions.is_empty() { + let new_join_on = new_join_conditions(&left_keys, &right_keys); + let new_sort_options = (0..sort_options.len()) + .map(|idx| sort_options[positions[idx]]) + .collect(); + join_plan.plan = join_constructor((new_join_on, new_sort_options))?; + } + } + + join_plan.children[0].data = left_keys; + join_plan.children[1].data = right_keys; + Ok(join_plan) +} + +pub fn reorder_aggregate_keys( + mut agg_node: PlanWithKeyRequirements, + agg_exec: &AggregateExec, +) -> Result { + let parent_required = &agg_node.data; + let output_columns = agg_exec + .group_expr() + .expr() + .iter() + .enumerate() + .map(|(index, (_, name))| Column::new(name, index)) + .collect::>(); + + let output_exprs = output_columns + .iter() + .map(|c| Arc::new(c.clone()) as _) + .collect::>(); + + if parent_required.len() == output_exprs.len() + && agg_exec.group_expr().null_expr().is_empty() + && !physical_exprs_equal(&output_exprs, parent_required) + { + if let Some(positions) = expected_expr_positions(&output_exprs, parent_required) { + if let Some(agg_exec) = + agg_exec.input().as_any().downcast_ref::() + { + if matches!(agg_exec.mode(), &AggregateMode::Partial) { + let group_exprs = agg_exec.group_expr().expr(); + let new_group_exprs = positions + .into_iter() + .map(|idx| group_exprs[idx].clone()) + .collect(); + let partial_agg = Arc::new(AggregateExec::try_new( + AggregateMode::Partial, + PhysicalGroupBy::new_single(new_group_exprs), + agg_exec.aggr_expr().to_vec(), + agg_exec.filter_expr().to_vec(), + Arc::clone(agg_exec.input()), + Arc::clone(&agg_exec.input_schema), + )?); + // Build new group expressions that correspond to the output + // of the "reordered" aggregator: + let group_exprs = partial_agg.group_expr().expr(); + let new_group_by = PhysicalGroupBy::new_single( + partial_agg + .output_group_expr() + .into_iter() + .enumerate() + .map(|(idx, expr)| (expr, group_exprs[idx].1.clone())) + .collect(), + ); + let new_final_agg = Arc::new(AggregateExec::try_new( + AggregateMode::FinalPartitioned, + new_group_by, + agg_exec.aggr_expr().to_vec(), + agg_exec.filter_expr().to_vec(), + Arc::clone(&partial_agg) as _, + agg_exec.input_schema(), + )?); + + agg_node.plan = Arc::clone(&new_final_agg) as _; + agg_node.data.clear(); + agg_node.children = vec![PlanWithKeyRequirements::new( + partial_agg as _, + vec![], + agg_node.children.swap_remove(0).children, + )]; + + // Need to create a new projection to change the expr ordering back + let agg_schema = new_final_agg.schema(); + let mut proj_exprs = output_columns + .iter() + .map(|col| { + let name = col.name(); + let index = agg_schema.index_of(name)?; + Ok((Arc::new(Column::new(name, index)) as _, name.to_owned())) + }) + .collect::>>()?; + let agg_fields = agg_schema.fields(); + for (idx, field) in + agg_fields.iter().enumerate().skip(output_columns.len()) + { + let name = field.name(); + let plan = Arc::new(Column::new(name, idx)) as _; + proj_exprs.push((plan, name.clone())) + } + return ProjectionExec::try_new(proj_exprs, new_final_agg).map(|p| { + PlanWithKeyRequirements::new(Arc::new(p), vec![], vec![agg_node]) + }); + } + } + } + } + Ok(agg_node) +} + +fn shift_right_required( + parent_required: &[Arc], + left_columns_len: usize, +) -> Option>> { + let new_right_required = parent_required + .iter() + .filter_map(|r| { + r.as_any().downcast_ref::().and_then(|col| { + col.index() + .checked_sub(left_columns_len) + .map(|index| Arc::new(Column::new(col.name(), index)) as _) + }) + }) + .collect::>(); + + // if the parent required are all coming from the right side, the requirements can be pushdown + (new_right_required.len() == parent_required.len()).then_some(new_right_required) +} + +/// When the physical planner creates the Joins, the ordering of join keys is from the original query. +/// That might not match with the output partitioning of the join node's children +/// This method will try to change the ordering of the join keys to match with the +/// partitioning of the join nodes' children. If it can not match with both sides, it will try to +/// match with one, either the left side or the right side. +/// +/// Example: +/// TopJoin on (a, b, c) +/// bottom left join on(b, a, c) +/// bottom right join on(c, b, a) +/// +/// Will be adjusted to: +/// TopJoin on (b, a, c) +/// bottom left join on(b, a, c) +/// bottom right join on(c, b, a) +/// +/// Compared to the Top-Down reordering process, this Bottom-Up approach is much simpler, but might not reach a best result. +/// The Bottom-Up approach will be useful in future if we plan to support storage partition-wised Joins. +/// In that case, the datasources/tables might be pre-partitioned and we can't adjust the key ordering of the datasources +/// and then can't apply the Top-Down reordering process. +pub fn reorder_join_keys_to_inputs( + plan: Arc, +) -> Result> { + let plan_any = plan.as_any(); + if let Some(HashJoinExec { + left, + right, + on, + filter, + join_type, + projection, + mode, + null_equals_null, + .. + }) = plan_any.downcast_ref::() + { + if matches!(mode, PartitionMode::Partitioned) { + let (join_keys, positions) = reorder_current_join_keys( + extract_join_keys(on), + Some(left.output_partitioning()), + Some(right.output_partitioning()), + left.equivalence_properties(), + right.equivalence_properties(), + ); + if positions.is_some_and(|idxs| !idxs.is_empty()) { + let JoinKeyPairs { + left_keys, + right_keys, + } = join_keys; + let new_join_on = new_join_conditions(&left_keys, &right_keys); + return Ok(Arc::new(HashJoinExec::try_new( + Arc::clone(left), + Arc::clone(right), + new_join_on, + filter.clone(), + join_type, + projection.clone(), + PartitionMode::Partitioned, + *null_equals_null, + )?)); + } + } + } else if let Some(SortMergeJoinExec { + left, + right, + on, + filter, + join_type, + sort_options, + null_equals_null, + .. + }) = plan_any.downcast_ref::() + { + let (join_keys, positions) = reorder_current_join_keys( + extract_join_keys(on), + Some(left.output_partitioning()), + Some(right.output_partitioning()), + left.equivalence_properties(), + right.equivalence_properties(), + ); + if let Some(positions) = positions { + if !positions.is_empty() { + let JoinKeyPairs { + left_keys, + right_keys, + } = join_keys; + let new_join_on = new_join_conditions(&left_keys, &right_keys); + let new_sort_options = (0..sort_options.len()) + .map(|idx| sort_options[positions[idx]]) + .collect(); + return SortMergeJoinExec::try_new( + Arc::clone(left), + Arc::clone(right), + new_join_on, + filter.clone(), + *join_type, + new_sort_options, + *null_equals_null, + ) + .map(|smj| Arc::new(smj) as _); + } + } + } + Ok(plan) +} + +/// Reorder the current join keys ordering based on either left partition or right partition +fn reorder_current_join_keys( + join_keys: JoinKeyPairs, + left_partition: Option<&Partitioning>, + right_partition: Option<&Partitioning>, + left_equivalence_properties: &EquivalenceProperties, + right_equivalence_properties: &EquivalenceProperties, +) -> (JoinKeyPairs, Option>) { + match (left_partition, right_partition) { + (Some(Partitioning::Hash(left_exprs, _)), _) => { + match try_reorder(join_keys, left_exprs, left_equivalence_properties) { + (join_keys, None) => reorder_current_join_keys( + join_keys, + None, + right_partition, + left_equivalence_properties, + right_equivalence_properties, + ), + result => result, + } + } + (_, Some(Partitioning::Hash(right_exprs, _))) => { + try_reorder(join_keys, right_exprs, right_equivalence_properties) + } + _ => (join_keys, None), + } +} + +fn try_reorder( + join_keys: JoinKeyPairs, + expected: &[Arc], + equivalence_properties: &EquivalenceProperties, +) -> (JoinKeyPairs, Option>) { + let eq_groups = equivalence_properties.eq_group(); + let mut normalized_expected = vec![]; + let mut normalized_left_keys = vec![]; + let mut normalized_right_keys = vec![]; + if join_keys.left_keys.len() != expected.len() { + return (join_keys, None); + } + if physical_exprs_equal(expected, &join_keys.left_keys) + || physical_exprs_equal(expected, &join_keys.right_keys) + { + return (join_keys, Some(vec![])); + } else if !equivalence_properties.eq_group().is_empty() { + normalized_expected = expected + .iter() + .map(|e| eq_groups.normalize_expr(Arc::clone(e))) + .collect(); + + normalized_left_keys = join_keys + .left_keys + .iter() + .map(|e| eq_groups.normalize_expr(Arc::clone(e))) + .collect(); + + normalized_right_keys = join_keys + .right_keys + .iter() + .map(|e| eq_groups.normalize_expr(Arc::clone(e))) + .collect(); + + if physical_exprs_equal(&normalized_expected, &normalized_left_keys) + || physical_exprs_equal(&normalized_expected, &normalized_right_keys) + { + return (join_keys, Some(vec![])); + } + } + + let Some(positions) = expected_expr_positions(&join_keys.left_keys, expected) + .or_else(|| expected_expr_positions(&join_keys.right_keys, expected)) + .or_else(|| expected_expr_positions(&normalized_left_keys, &normalized_expected)) + .or_else(|| { + expected_expr_positions(&normalized_right_keys, &normalized_expected) + }) + else { + return (join_keys, None); + }; + + let mut new_left_keys = vec![]; + let mut new_right_keys = vec![]; + for pos in positions.iter() { + new_left_keys.push(Arc::clone(&join_keys.left_keys[*pos])); + new_right_keys.push(Arc::clone(&join_keys.right_keys[*pos])); + } + let pairs = JoinKeyPairs { + left_keys: new_left_keys, + right_keys: new_right_keys, + }; + + (pairs, Some(positions)) +} + +/// Return the expected expressions positions. +/// For example, the current expressions are ['c', 'a', 'a', b'], the expected expressions are ['b', 'c', 'a', 'a'], +/// +/// This method will return a Vec [3, 0, 1, 2] +fn expected_expr_positions( + current: &[Arc], + expected: &[Arc], +) -> Option> { + if current.is_empty() || expected.is_empty() { + return None; + } + let mut indexes: Vec = vec![]; + let mut current = current.to_vec(); + for expr in expected.iter() { + // Find the position of the expected expr in the current expressions + if let Some(expected_position) = current.iter().position(|e| e.eq(expr)) { + current[expected_position] = Arc::new(NoOp::new()); + indexes.push(expected_position); + } else { + return None; + } + } + Some(indexes) +} + +fn extract_join_keys(on: &[(PhysicalExprRef, PhysicalExprRef)]) -> JoinKeyPairs { + let (left_keys, right_keys) = on + .iter() + .map(|(l, r)| (Arc::clone(l) as _, Arc::clone(r) as _)) + .unzip(); + JoinKeyPairs { + left_keys, + right_keys, + } +} + +fn new_join_conditions( + new_left_keys: &[Arc], + new_right_keys: &[Arc], +) -> Vec<(PhysicalExprRef, PhysicalExprRef)> { + new_left_keys + .iter() + .zip(new_right_keys.iter()) + .map(|(l_key, r_key)| (Arc::clone(l_key), Arc::clone(r_key))) + .collect() +} + +/// Adds RoundRobin repartition operator to the plan increase parallelism. +/// +/// # Arguments +/// +/// * `input`: Current node. +/// * `n_target`: desired target partition number, if partition number of the +/// current executor is less than this value. Partition number will be increased. +/// +/// # Returns +/// +/// A [`Result`] object that contains new execution plan where the desired +/// partition number is achieved by adding a RoundRobin repartition. +fn add_roundrobin_on_top( + input: DistributionContext, + n_target: usize, +) -> Result { + // Adding repartition is helpful: + if input.plan.output_partitioning().partition_count() < n_target { + // When there is an existing ordering, we preserve ordering + // during repartition. This will be un-done in the future + // If any of the following conditions is true + // - Preserving ordering is not helpful in terms of satisfying ordering requirements + // - Usage of order preserving variants is not desirable + // (determined by flag `config.optimizer.prefer_existing_sort`) + let partitioning = Partitioning::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) + } +} + +/// Adds a hash repartition operator: +/// - to increase parallelism, and/or +/// - to satisfy requirements of the subsequent operators. +/// +/// Repartition(Hash) is added on top of operator `input`. +/// +/// # Arguments +/// +/// * `input`: Current node. +/// * `hash_exprs`: Stores Physical Exprs that are used during hashing. +/// * `n_target`: desired target partition number, if partition number of the +/// current executor is less than this value. Partition number will be increased. +/// +/// # Returns +/// +/// A [`Result`] object that contains new execution plan where the desired +/// distribution is satisfied by adding a Hash repartition. +fn add_hash_on_top( + input: DistributionContext, + hash_exprs: Vec>, + n_target: usize, +) -> Result { + // Early return if hash repartition is unnecessary + // `RepartitionExec: partitioning=Hash([...], 1), input_partitions=1` is unnecessary. + if n_target == 1 && input.plan.output_partitioning().partition_count() == 1 { + return Ok(input); + } + + let dist = Distribution::HashPartitioned(hash_exprs); + let satisfied = input + .plan + .output_partitioning() + .satisfy(&dist, input.plan.equivalence_properties()); + + // Add hash repartitioning when: + // - The hash distribution requirement is not satisfied, or + // - We can increase parallelism by adding hash partitioning. + if !satisfied || n_target > input.plan.output_partitioning().partition_count() { + // When there is an existing ordering, we preserve ordering during + // repartition. This will be rolled back 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 (per the flag + // `config.optimizer.prefer_existing_sort`). + let partitioning = dist.create_partitioning(n_target); + let repartition = + RepartitionExec::try_new(Arc::clone(&input.plan), partitioning)? + .with_preserve_order(); + let plan = Arc::new(repartition) as _; + + return Ok(DistributionContext::new(plan, true, vec![input])); + } + + Ok(input) +} + +/// Adds a [`SortPreservingMergeExec`] operator on top of input executor +/// to satisfy single distribution requirement. +/// +/// # Arguments +/// +/// * `input`: Current node. +/// +/// # Returns +/// +/// Updated node with an execution plan, where desired single +/// distribution is satisfied by adding [`SortPreservingMergeExec`]. +fn add_spm_on_top(input: DistributionContext) -> DistributionContext { + // Add SortPreservingMerge only when partition count is larger than 1. + if input.plan.output_partitioning().partition_count() > 1 { + // When there is an existing ordering, we preserve ordering + // when decreasing partitions. 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.bounded_order_preserving_variants`) + let should_preserve_ordering = input.plan.output_ordering().is_some(); + + let new_plan = if should_preserve_ordering { + Arc::new(SortPreservingMergeExec::new( + input + .plan + .output_ordering() + .unwrap_or(&LexOrdering::default()) + .clone(), + Arc::clone(&input.plan), + )) as _ + } else { + Arc::new(CoalescePartitionsExec::new(Arc::clone(&input.plan))) as _ + }; + + DistributionContext::new(new_plan, true, vec![input]) + } else { + input + } +} + +/// Updates the physical plan inside [`DistributionContext`] so that distribution +/// changing operators are removed from the top. If they are necessary, they will +/// be added in subsequent stages. +/// +/// Assume that following plan is given: +/// ```text +/// "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10", +/// " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", +/// " ParquetExec: file_groups={2 groups: \[\[x], \[y]]}, projection=\[a, b, c, d, e], output_ordering=\[a@0 ASC]", +/// ``` +/// +/// Since `RepartitionExec`s change the distribution, this function removes +/// them and returns following plan: +/// +/// ```text +/// "ParquetExec: file_groups={2 groups: \[\[x], \[y]]}, projection=\[a, b, c, d, e], output_ordering=\[a@0 ASC]", +/// ``` +fn remove_dist_changing_operators( + mut distribution_context: DistributionContext, +) -> Result { + while is_repartition(&distribution_context.plan) + || is_coalesce_partitions(&distribution_context.plan) + || is_sort_preserving_merge(&distribution_context.plan) + { + // All of above operators have a single child. First child is only child. + // Remove any distribution changing operators at the beginning: + distribution_context = distribution_context.children.swap_remove(0); + // Note that they will be re-inserted later on if necessary or helpful. + } + + Ok(distribution_context) +} + +/// Updates the [`DistributionContext`] if preserving ordering while changing partitioning is not helpful or desirable. +/// +/// Assume that following plan is given: +/// ```text +/// "SortPreservingMergeExec: \[a@0 ASC]" +/// " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10, preserve_order=true", +/// " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2, preserve_order=true", +/// " ParquetExec: file_groups={2 groups: \[\[x], \[y]]}, projection=\[a, b, c, d, e], output_ordering=\[a@0 ASC]", +/// ``` +/// +/// This function converts plan above to the following: +/// +/// ```text +/// "CoalescePartitionsExec" +/// " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10", +/// " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", +/// " ParquetExec: file_groups={2 groups: \[\[x], \[y]]}, projection=\[a, b, c, d, e], output_ordering=\[a@0 ASC]", +/// ``` +fn replace_order_preserving_variants( + mut context: DistributionContext, +) -> Result { + context.children = context + .children + .into_iter() + .map(|child| { + if child.data { + replace_order_preserving_variants(child) + } else { + Ok(child) + } + }) + .collect::>>()?; + + if is_sort_preserving_merge(&context.plan) { + let child_plan = Arc::clone(&context.children[0].plan); + context.plan = Arc::new(CoalescePartitionsExec::new(child_plan)); + return Ok(context); + } else if let Some(repartition) = + 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(), + )?); + return Ok(context); + } + } + + context.update_plan_from_children() +} + +/// A struct to keep track of repartition requirements for each child node. +struct RepartitionRequirementStatus { + /// The distribution requirement for the node. + requirement: Distribution, + /// Designates whether round robin partitioning is theoretically beneficial; + /// i.e. the operator can actually utilize parallelism. + roundrobin_beneficial: bool, + /// Designates whether round robin partitioning is beneficial according to + /// the statistical information we have on the number of rows. + roundrobin_beneficial_stats: bool, + /// Designates whether hash partitioning is necessary. + hash_necessary: bool, +} + +/// Calculates the `RepartitionRequirementStatus` for each children to generate +/// consistent and sensible (in terms of performance) distribution requirements. +/// As an example, a hash join's left (build) child might produce +/// +/// ```text +/// RepartitionRequirementStatus { +/// .., +/// hash_necessary: true +/// } +/// ``` +/// +/// while its right (probe) child might have very few rows and produce: +/// +/// ```text +/// RepartitionRequirementStatus { +/// .., +/// hash_necessary: false +/// } +/// ``` +/// +/// These statuses are not consistent as all children should agree on hash +/// partitioning. This function aligns the statuses to generate consistent +/// hash partitions for each children. After alignment, the right child's +/// status would turn into: +/// +/// ```text +/// RepartitionRequirementStatus { +/// .., +/// hash_necessary: true +/// } +/// ``` +fn get_repartition_requirement_status( + plan: &Arc, + batch_size: usize, + should_use_estimates: bool, +) -> Result> { + let mut needs_alignment = false; + let children = plan.children(); + let rr_beneficial = plan.benefits_from_input_partitioning(); + let requirements = plan.required_input_distribution(); + let mut repartition_status_flags = vec![]; + for (child, requirement, roundrobin_beneficial) in + izip!(children.into_iter(), requirements, rr_beneficial) + { + // Decide whether adding a round robin is beneficial depending on + // the statistical information we have on the number of rows: + let roundrobin_beneficial_stats = match child.statistics()?.num_rows { + Precision::Exact(n_rows) => n_rows > batch_size, + Precision::Inexact(n_rows) => !should_use_estimates || (n_rows > batch_size), + Precision::Absent => true, + }; + let is_hash = matches!(requirement, Distribution::HashPartitioned(_)); + // Hash re-partitioning is necessary when the input has more than one + // partitions: + let multi_partitions = child.output_partitioning().partition_count() > 1; + let roundrobin_sensible = roundrobin_beneficial && roundrobin_beneficial_stats; + needs_alignment |= is_hash && (multi_partitions || roundrobin_sensible); + repartition_status_flags.push(( + is_hash, + RepartitionRequirementStatus { + requirement, + roundrobin_beneficial, + roundrobin_beneficial_stats, + hash_necessary: is_hash && multi_partitions, + }, + )); + } + // Align hash necessary flags for hash partitions to generate consistent + // hash partitions at each children: + if needs_alignment { + // When there is at least one hash requirement that is necessary or + // beneficial according to statistics, make all children require hash + // repartitioning: + for (is_hash, status) in &mut repartition_status_flags { + if *is_hash { + status.hash_necessary = true; + } + } + } + Ok(repartition_status_flags + .into_iter() + .map(|(_, status)| status) + .collect()) +} + +/// This function checks whether we need to add additional data exchange +/// operators to satisfy distribution requirements. Since this function +/// takes care of such requirements, we should avoid manually adding data +/// exchange operators in other places. +pub fn ensure_distribution( + dist_context: DistributionContext, + config: &ConfigOptions, +) -> Result> { + let dist_context = update_children(dist_context)?; + + if dist_context.plan.children().is_empty() { + return Ok(Transformed::no(dist_context)); + } + + let target_partitions = config.execution.target_partitions; + // When `false`, round robin repartition will not be added to increase parallelism + let enable_round_robin = config.optimizer.enable_round_robin_repartition; + let repartition_file_scans = config.optimizer.repartition_file_scans; + let batch_size = config.execution.batch_size; + let should_use_estimates = config + .execution + .use_row_number_estimates_to_optimize_partitioning; + let unbounded_and_pipeline_friendly = dist_context.plan.boundedness().is_unbounded() + && matches!( + dist_context.plan.pipeline_behavior(), + EmissionType::Incremental | EmissionType::Both + ); + // Use order preserving variants either of the conditions true + // - it is desired according to config + // - when plan is unbounded + // - when it is pipeline friendly (can incrementally produce results) + let order_preserving_variants_desirable = + unbounded_and_pipeline_friendly || config.optimizer.prefer_existing_sort; + + // Remove unnecessary repartition from the physical plan if any + let DistributionContext { + mut plan, + data, + children, + } = remove_dist_changing_operators(dist_context)?; + + if let Some(exec) = plan.as_any().downcast_ref::() { + if let Some(updated_window) = get_best_fitting_window( + exec.window_expr(), + exec.input(), + &exec.partition_keys, + )? { + plan = updated_window; + } + } else if let Some(exec) = plan.as_any().downcast_ref::() { + if let Some(updated_window) = get_best_fitting_window( + exec.window_expr(), + exec.input(), + &exec.partition_keys, + )? { + plan = updated_window; + } + }; + + let repartition_status_flags = + get_repartition_requirement_status(&plan, batch_size, should_use_estimates)?; + // This loop iterates over all the children to: + // - Increase parallelism for every child if it is beneficial. + // - Satisfy the distribution requirements of every child, if it is not + // already satisfied. + // We store the updated children in `new_children`. + let children = izip!( + children.into_iter(), + plan.required_input_ordering(), + plan.maintains_input_order(), + repartition_status_flags.into_iter() + ) + .map( + |( + mut child, + required_input_ordering, + maintains, + RepartitionRequirementStatus { + requirement, + roundrobin_beneficial, + roundrobin_beneficial_stats, + hash_necessary, + }, + )| { + let add_roundrobin = enable_round_robin + // Operator benefits from partitioning (e.g. filter): + && roundrobin_beneficial + && roundrobin_beneficial_stats + // Unless partitioning increases the partition count, it is not beneficial: + && child.plan.output_partitioning().partition_count() < target_partitions; + + // When `repartition_file_scans` is set, attempt to increase + // parallelism at the source. + if repartition_file_scans && roundrobin_beneficial_stats { + if let Some(new_child) = + child.plan.repartitioned(target_partitions, config)? + { + child.plan = new_child; + } + } + + // Satisfy the distribution requirement if it is unmet. + match &requirement { + Distribution::SinglePartition => { + child = add_spm_on_top(child); + } + Distribution::HashPartitioned(exprs) => { + if add_roundrobin { + // Add round-robin repartitioning on top of the operator + // to increase parallelism. + child = add_roundrobin_on_top(child, target_partitions)?; + } + // When inserting hash is necessary to satisfy hash requirement, insert hash repartition. + if hash_necessary { + child = + add_hash_on_top(child, exprs.to_vec(), target_partitions)?; + } + } + Distribution::UnspecifiedDistribution => { + if add_roundrobin { + // Add round-robin repartitioning on top of the operator + // to increase parallelism. + child = add_roundrobin_on_top(child, target_partitions)?; + } + } + }; + + // There is an ordering requirement of the operator: + if let Some(required_input_ordering) = required_input_ordering { + // Either: + // - Ordering requirement cannot be satisfied by preserving ordering through repartitions, or + // - using order preserving variant is not desirable. + let ordering_satisfied = child + .plan + .equivalence_properties() + .ordering_satisfy_requirement(&required_input_ordering); + if (!ordering_satisfied || !order_preserving_variants_desirable) + && child.data + { + child = replace_order_preserving_variants(child)?; + // If ordering requirements were satisfied before repartitioning, + // make sure ordering requirements are still satisfied after. + if ordering_satisfied { + // Make sure to satisfy ordering requirement: + child = add_sort_above_with_check( + child, + required_input_ordering.clone(), + None, + ); + } + } + // Stop tracking distribution changing operators + child.data = false; + } else { + // no ordering requirement + match requirement { + // Operator requires specific distribution. + Distribution::SinglePartition | Distribution::HashPartitioned(_) => { + // Since there is no ordering requirement, preserving ordering is pointless + child = replace_order_preserving_variants(child)?; + } + Distribution::UnspecifiedDistribution => { + // Since ordering is lost, trying to preserve ordering is pointless + if !maintains || plan.as_any().is::() { + child = replace_order_preserving_variants(child)?; + } + } + } + } + Ok(child) + }, + ) + .collect::>>()?; + + let children_plans = children + .iter() + .map(|c| Arc::clone(&c.plan)) + .collect::>(); + + plan = if plan.as_any().is::() + && !config.optimizer.prefer_existing_union + && can_interleave(children_plans.iter()) + { + // Add a special case for [`UnionExec`] since we want to "bubble up" + // hash-partitioned data. So instead of + // + // Agg: + // Repartition (hash): + // Union: + // - Agg: + // Repartition (hash): + // Data + // - Agg: + // Repartition (hash): + // Data + // + // we can use: + // + // Agg: + // Interleave: + // - Agg: + // Repartition (hash): + // Data + // - Agg: + // Repartition (hash): + // Data + Arc::new(InterleaveExec::try_new(children_plans)?) + } else { + plan.with_new_children(children_plans)? + }; + + Ok(Transformed::yes(DistributionContext::new( + plan, data, children, + ))) +} + +/// Keeps track of distribution changing operators (like `RepartitionExec`, +/// `SortPreservingMergeExec`, `CoalescePartitionsExec`) and their ancestors. +/// Using this information, we can optimize distribution of the plan if/when +/// necessary. +pub type DistributionContext = PlanContext; + +fn update_children(mut dist_context: DistributionContext) -> Result { + for child_context in dist_context.children.iter_mut() { + let child_plan_any = child_context.plan.as_any(); + child_context.data = + if let Some(repartition) = child_plan_any.downcast_ref::() { + !matches!( + repartition.partitioning(), + Partitioning::UnknownPartitioning(_) + ) + } else { + child_plan_any.is::() + || child_plan_any.is::() + || child_context.plan.children().is_empty() + || child_context.children[0].data + || child_context + .plan + .required_input_distribution() + .iter() + .zip(child_context.children.iter()) + .any(|(required_dist, child_context)| { + child_context.data + && matches!( + required_dist, + Distribution::UnspecifiedDistribution + ) + }) + } + } + + dist_context.data = false; + Ok(dist_context) +} diff --git a/datafusion/physical-optimizer/src/enforce_sorting/mod.rs b/datafusion/physical-optimizer/src/enforce_sorting/mod.rs index 6853c3467b6c..f098f16d9b84 100644 --- a/datafusion/physical-optimizer/src/enforce_sorting/mod.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/mod.rs @@ -40,16 +40,6 @@ pub mod sort_pushdown; use std::sync::Arc; -use datafusion_common::plan_err; -use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; -use datafusion_common::Result; -use datafusion_physical_expr::{Distribution, Partitioning}; -use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; -use datafusion_physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; -use datafusion_physical_plan::repartition::RepartitionExec; -use datafusion_physical_plan::sorts::partial_sort::PartialSortExec; -use datafusion_physical_plan::{ExecutionPlan, ExecutionPlanProperties, InputOrderMode}; - use crate::enforce_sorting::replace_with_order_preserving_variants::{ replace_with_order_preserving_variants, OrderPreservationContext, }; @@ -61,14 +51,25 @@ use crate::utils::{ is_repartition, is_sort, is_sort_preserving_merge, is_union, is_window, }; use crate::PhysicalOptimizerRule; + use datafusion_common::config::ConfigOptions; +use datafusion_common::plan_err; +use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; +use datafusion_common::Result; +use datafusion_physical_expr::{Distribution, Partitioning}; +use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; +use datafusion_physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; +use datafusion_physical_plan::repartition::RepartitionExec; +use datafusion_physical_plan::sorts::partial_sort::PartialSortExec; use datafusion_physical_plan::sorts::sort::SortExec; use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; use datafusion_physical_plan::tree_node::PlanContext; use datafusion_physical_plan::windows::{ get_best_fitting_window, BoundedWindowAggExec, WindowAggExec, }; +use datafusion_physical_plan::{ExecutionPlan, ExecutionPlanProperties, InputOrderMode}; + use itertools::izip; /// This rule inspects [`SortExec`]'s in the given physical plan and removes the diff --git a/datafusion/physical-optimizer/src/enforce_sorting/replace_with_order_preserving_variants.rs b/datafusion/physical-optimizer/src/enforce_sorting/replace_with_order_preserving_variants.rs index 1f1a35efb251..c542f9261a24 100644 --- a/datafusion/physical-optimizer/src/enforce_sorting/replace_with_order_preserving_variants.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/replace_with_order_preserving_variants.rs @@ -31,11 +31,11 @@ use datafusion_common::Result; use datafusion_physical_expr_common::sort_expr::LexOrdering; use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; use datafusion_physical_plan::execution_plan::EmissionType; +use datafusion_physical_plan::repartition::RepartitionExec; +use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; use datafusion_physical_plan::tree_node::PlanContext; use datafusion_physical_plan::ExecutionPlanProperties; -use datafusion_physical_plan::repartition::RepartitionExec; -use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; use itertools::izip; /// For a given `plan`, this object carries the information one needs from its diff --git a/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs b/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs index d2f7bcacd089..166113807e37 100644 --- a/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs @@ -18,11 +18,11 @@ use std::fmt::Debug; use std::sync::Arc; -use arrow_schema::SchemaRef; - use crate::utils::{ add_sort_above, is_sort, is_sort_preserving_merge, is_union, is_window, }; + +use arrow_schema::SchemaRef; use datafusion_common::tree_node::{ ConcreteTreeNode, Transformed, TreeNode, TreeNodeRecursion, }; diff --git a/datafusion/physical-optimizer/src/lib.rs b/datafusion/physical-optimizer/src/lib.rs index 62a1f68c1366..bcb87944f5fd 100644 --- a/datafusion/physical-optimizer/src/lib.rs +++ b/datafusion/physical-optimizer/src/lib.rs @@ -21,6 +21,7 @@ pub mod aggregate_statistics; pub mod coalesce_batches; pub mod combine_partial_final_agg; +pub mod enforce_distribution; pub mod enforce_sorting; pub mod join_selection; pub mod limit_pushdown; @@ -29,9 +30,8 @@ mod optimizer; pub mod output_requirements; pub mod pruning; pub mod sanity_checker; -pub mod test_utils; pub mod topk_aggregation; pub mod update_aggr_exprs; -pub mod utils; - pub use optimizer::PhysicalOptimizerRule; +pub mod test_utils; +pub mod utils; diff --git a/datafusion/physical-optimizer/src/test_utils.rs b/datafusion/physical-optimizer/src/test_utils.rs index 9f0b5ddf6f40..56830d3b3f89 100644 --- a/datafusion/physical-optimizer/src/test_utils.rs +++ b/datafusion/physical-optimizer/src/test_utils.rs @@ -15,21 +15,25 @@ // specific language governing permissions and limitations // under the License. -//! Collection of testing utility functions that are leveraged by the query optimizer rules - -use std::sync::Arc; - -use std::any::Any; -use std::fmt::Formatter; - -use arrow_schema::{Schema, SchemaRef, SortOptions}; +//! Test utilities for physical optimizer tests + +use crate::limited_distinct_aggregation::LimitedDistinctAggregation; +use crate::PhysicalOptimizerRule; +use arrow::array::Int32Array; +use arrow::record_batch::RecordBatch; +use arrow_schema::{DataType, Field, Schema, SchemaRef, SortOptions}; +use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; +use datafusion_common::utils::expr::COUNT_STAR_EXPANSION; use datafusion_common::{JoinType, Result}; -use datafusion_expr::test::function_stub::count_udaf; +use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use datafusion_expr::{WindowFrame, WindowFunctionDefinition}; +use datafusion_functions_aggregate::count::count_udaf; +use datafusion_physical_expr::aggregate::{AggregateExprBuilder, AggregateFunctionExpr}; use datafusion_physical_expr::expressions::col; -use datafusion_physical_expr::PhysicalExpr; -use datafusion_physical_expr::PhysicalSortExpr; +use datafusion_physical_expr::{expressions, PhysicalExpr}; +use datafusion_physical_expr_common::sort_expr::LexRequirement; +use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; use datafusion_physical_plan::aggregates::{ AggregateExec, AggregateMode, PhysicalGroupBy, }; @@ -43,18 +47,53 @@ use datafusion_physical_plan::memory::MemoryExec; use datafusion_physical_plan::repartition::RepartitionExec; use datafusion_physical_plan::sorts::sort::SortExec; use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; +use datafusion_physical_plan::streaming::{PartitionStream, StreamingTableExec}; +use datafusion_physical_plan::tree_node::PlanContext; use datafusion_physical_plan::union::UnionExec; use datafusion_physical_plan::windows::{create_window_expr, BoundedWindowAggExec}; -use datafusion_physical_plan::{InputOrderMode, Partitioning}; - -use datafusion_execution::{SendableRecordBatchStream, TaskContext}; -use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; use datafusion_physical_plan::ExecutionPlan; - -use datafusion_physical_plan::tree_node::PlanContext; use datafusion_physical_plan::{ displayable, DisplayAs, DisplayFormatType, PlanProperties, }; +use datafusion_physical_plan::{InputOrderMode, Partitioning}; +use std::any::Any; +use std::fmt::Formatter; +use std::sync::Arc; + +pub fn schema() -> SchemaRef { + Arc::new(Schema::new(vec![ + Field::new("a", DataType::Int64, true), + Field::new("b", DataType::Int64, true), + Field::new("c", DataType::Int64, true), + Field::new("d", DataType::Int32, true), + Field::new("e", DataType::Boolean, true), + ])) +} + +pub fn create_test_schema() -> Result { + let nullable_column = Field::new("nullable_col", DataType::Int32, true); + let non_nullable_column = Field::new("non_nullable_col", DataType::Int32, false); + let schema = Arc::new(Schema::new(vec![nullable_column, non_nullable_column])); + Ok(schema) +} + +pub fn create_test_schema2() -> Result { + let col_a = Field::new("col_a", DataType::Int32, true); + let col_b = Field::new("col_b", DataType::Int32, true); + let schema = Arc::new(Schema::new(vec![col_a, col_b])); + Ok(schema) +} + +// Generate a schema which consists of 5 columns (a, b, c, d, e) +pub fn create_test_schema3() -> Result { + let a = Field::new("a", DataType::Int32, true); + let b = Field::new("b", DataType::Int32, false); + let c = Field::new("c", DataType::Int32, true); + let d = Field::new("d", DataType::Int32, false); + let e = Field::new("e", DataType::Int32, false); + let schema = Arc::new(Schema::new(vec![a, b, c, d, e])); + Ok(schema) +} pub fn sort_merge_join_exec( left: Arc, @@ -334,3 +373,189 @@ pub fn check_integrity(context: PlanContext) -> Result Vec<&str> { + plan.split('\n') + .map(|s| s.trim()) + .filter(|s| !s.is_empty()) + .collect() +} + +// construct a stream partition for test purposes +#[derive(Debug)] +pub(crate) struct TestStreamPartition { + pub schema: SchemaRef, +} + +impl PartitionStream for TestStreamPartition { + fn schema(&self) -> &SchemaRef { + &self.schema + } + fn execute(&self, _ctx: Arc) -> SendableRecordBatchStream { + unreachable!() + } +} + +/// Create an unbounded stream exec +pub fn stream_exec_ordered( + schema: &SchemaRef, + sort_exprs: impl IntoIterator, +) -> Arc { + let sort_exprs = sort_exprs.into_iter().collect(); + + Arc::new( + StreamingTableExec::try_new( + Arc::clone(schema), + vec![Arc::new(TestStreamPartition { + schema: Arc::clone(schema), + }) as _], + None, + vec![sort_exprs], + true, + None, + ) + .unwrap(), + ) +} + +// Creates a stream exec source for the test purposes +pub fn stream_exec_ordered_with_projection( + schema: &SchemaRef, + sort_exprs: impl IntoIterator, +) -> Arc { + let sort_exprs = sort_exprs.into_iter().collect(); + let projection: Vec = vec![0, 2, 3]; + + Arc::new( + StreamingTableExec::try_new( + Arc::clone(schema), + vec![Arc::new(TestStreamPartition { + schema: Arc::clone(schema), + }) as _], + Some(&projection), + vec![sort_exprs], + true, + None, + ) + .unwrap(), + ) +} + +pub fn mock_data() -> Result> { + let schema = Arc::new(Schema::new(vec![ + Field::new("a", DataType::Int32, true), + Field::new("b", DataType::Int32, true), + ])); + + let batch = RecordBatch::try_new( + Arc::clone(&schema), + vec![ + Arc::new(Int32Array::from(vec![ + Some(1), + Some(2), + None, + Some(1), + Some(4), + Some(5), + ])), + Arc::new(Int32Array::from(vec![ + Some(1), + None, + Some(6), + Some(2), + Some(8), + Some(9), + ])), + ], + )?; + + Ok(Arc::new(MemoryExec::try_new( + &[vec![batch]], + Arc::clone(&schema), + None, + )?)) +} + +pub fn build_group_by(input_schema: &SchemaRef, columns: Vec) -> PhysicalGroupBy { + let mut group_by_expr: Vec<(Arc, String)> = vec![]; + for column in columns.iter() { + group_by_expr.push((col(column, input_schema).unwrap(), column.to_string())); + } + PhysicalGroupBy::new_single(group_by_expr.clone()) +} + +pub fn assert_plan_matches_expected( + plan: &Arc, + expected: &[&str], +) -> Result<()> { + let expected_lines: Vec<&str> = expected.to_vec(); + let config = ConfigOptions::new(); + + let optimized = + LimitedDistinctAggregation::new().optimize(Arc::clone(plan), &config)?; + + let optimized_result = displayable(optimized.as_ref()).indent(true).to_string(); + let actual_lines = trim_plan_display(&optimized_result); + + assert_eq!( + &expected_lines, &actual_lines, + "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", + expected_lines, actual_lines + ); + + Ok(()) +} + +/// Describe the type of aggregate being tested +pub enum TestAggregate { + /// Testing COUNT(*) type aggregates + CountStar, + + /// Testing for COUNT(column) aggregate + ColumnA(Arc), +} + +impl TestAggregate { + /// Create a new COUNT(*) aggregate + pub fn new_count_star() -> Self { + Self::CountStar + } + + /// Create a new COUNT(column) aggregate + pub fn new_count_column(schema: &Arc) -> Self { + Self::ColumnA(Arc::clone(schema)) + } + + /// Return appropriate expr depending if COUNT is for col or table (*) + pub fn count_expr(&self, schema: &Schema) -> AggregateFunctionExpr { + AggregateExprBuilder::new(count_udaf(), vec![self.column()]) + .schema(Arc::new(schema.clone())) + .alias(self.column_name()) + .build() + .unwrap() + } + + /// what argument would this aggregate need in the plan? + fn column(&self) -> Arc { + match self { + Self::CountStar => expressions::lit(COUNT_STAR_EXPANSION), + Self::ColumnA(s) => col("a", s).unwrap(), + } + } + + /// What name would this aggregate produce in a plan? + pub fn column_name(&self) -> &'static str { + match self { + Self::CountStar => "COUNT(*)", + Self::ColumnA(_) => "COUNT(a)", + } + } + + /// What is the expected count? + pub fn expected_count(&self) -> i64 { + match self { + TestAggregate::CountStar => 3, + TestAggregate::ColumnA(_) => 2, + } + } +} diff --git a/datafusion/physical-optimizer/src/utils.rs b/datafusion/physical-optimizer/src/utils.rs index ed280452b53b..636e78a06ce7 100644 --- a/datafusion/physical-optimizer/src/utils.rs +++ b/datafusion/physical-optimizer/src/utils.rs @@ -17,7 +17,8 @@ use std::sync::Arc; -use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; +use datafusion_physical_expr::LexRequirement; +use datafusion_physical_expr_common::sort_expr::LexOrdering; use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; use datafusion_physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; use datafusion_physical_plan::repartition::RepartitionExec; diff --git a/datafusion/physical-optimizer/tests/enforce_sorting.rs b/datafusion/physical-optimizer/tests/enforce_sorting.rs new file mode 100644 index 000000000000..df1a484e1585 --- /dev/null +++ b/datafusion/physical-optimizer/tests/enforce_sorting.rs @@ -0,0 +1,861 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use std::sync::Arc; + +use arrow::compute::SortOptions; +use datafusion_common::Result; +use datafusion_physical_expr::expressions::{col, NotExpr}; +use datafusion_physical_optimizer::PhysicalOptimizerRule; +use datafusion_physical_plan::displayable; +use datafusion_physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; +use datafusion_physical_expr_common::sort_expr::LexOrdering; +use datafusion_physical_optimizer::enforce_sorting::{EnforceSorting,PlanWithCorrespondingCoalescePartitions,PlanWithCorrespondingSort,parallelize_sorts,ensure_sorting}; +use datafusion_physical_optimizer::enforce_sorting::replace_with_order_preserving_variants::{replace_with_order_preserving_variants,OrderPreservationContext}; +use datafusion_physical_optimizer::enforce_sorting::sort_pushdown::{SortPushDown, assign_initial_requirements, pushdown_sorts}; +use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; +use datafusion_physical_plan::sorts::sort::SortExec; +use datafusion_physical_plan::{get_plan_string, ExecutionPlan}; +use datafusion_common::config::ConfigOptions; +use datafusion_common::tree_node::{TreeNode, TransformedResult}; +use datafusion_physical_optimizer::enforce_distribution::EnforceDistribution; +use datafusion_physical_optimizer::test_utils::{check_integrity,aggregate_exec, bounded_window_exec, coalesce_batches_exec, create_test_schema, create_test_schema3, filter_exec, memory_exec, repartition_exec, sort_exec, sort_expr, sort_expr_options, sort_preserving_merge_exec, stream_exec_ordered, union_exec, RequirementsTestExec}; + +/// Runs the sort enforcement optimizer and asserts the plan +/// against the original and expected plans +/// +/// `$EXPECTED_PLAN_LINES`: input plan +/// `$EXPECTED_OPTIMIZED_PLAN_LINES`: optimized plan +/// `$PLAN`: the plan to optimized +/// `REPARTITION_SORTS`: Flag to set `config.options.optimizer.repartition_sorts` option. +macro_rules! assert_optimized { + ($EXPECTED_PLAN_LINES: expr, $EXPECTED_OPTIMIZED_PLAN_LINES: expr, $PLAN: expr, $REPARTITION_SORTS: expr) => { + let mut config = ConfigOptions::new(); + config.optimizer.repartition_sorts = $REPARTITION_SORTS; + + // This file has 4 rules that use tree node, apply these rules as in the + // EnforceSorting::optimize implementation + // After these operations tree nodes should be in a consistent state. + // This code block makes sure that these rules doesn't violate tree node integrity. + { + let plan_requirements = PlanWithCorrespondingSort::new_default($PLAN.clone()); + let adjusted = plan_requirements + .transform_up(ensure_sorting) + .data() + .and_then(check_integrity)?; + // TODO: End state payloads will be checked here. + + let new_plan = if config.optimizer.repartition_sorts { + let plan_with_coalesce_partitions = + PlanWithCorrespondingCoalescePartitions::new_default(adjusted.plan); + let parallel = plan_with_coalesce_partitions + .transform_up(parallelize_sorts) + .data() + .and_then(check_integrity)?; + // TODO: End state payloads will be checked here. + parallel.plan + } else { + adjusted.plan + }; + + let plan_with_pipeline_fixer = OrderPreservationContext::new_default(new_plan); + let updated_plan = plan_with_pipeline_fixer + .transform_up(|plan_with_pipeline_fixer| { + replace_with_order_preserving_variants( + plan_with_pipeline_fixer, + false, + true, + &config, + ) + }) + .data() + .and_then(check_integrity)?; + // TODO: End state payloads will be checked here. + + let mut sort_pushdown = SortPushDown::new_default(updated_plan.plan); + assign_initial_requirements(&mut sort_pushdown); + check_integrity(pushdown_sorts(sort_pushdown)?)?; + // TODO: End state payloads will be checked here. + } + + let physical_plan = $PLAN; + let formatted = displayable(physical_plan.as_ref()).indent(true).to_string(); + let actual: Vec<&str> = formatted.trim().lines().collect(); + + let expected_plan_lines: Vec<&str> = $EXPECTED_PLAN_LINES + .iter().map(|s| *s).collect(); + + assert_eq!( + expected_plan_lines, actual, + "\n**Original Plan Mismatch\n\nexpected:\n\n{expected_plan_lines:#?}\nactual:\n\n{actual:#?}\n\n" + ); + + let expected_optimized_lines: Vec<&str> = $EXPECTED_OPTIMIZED_PLAN_LINES + .iter().map(|s| *s).collect(); + + // Run the actual optimizer + let optimized_physical_plan = + EnforceSorting::new().optimize(physical_plan, &config)?; + + // Get string representation of the plan + let actual = get_plan_string(&optimized_physical_plan); + assert_eq!( + expected_optimized_lines, actual, + "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected_optimized_lines:#?}\nactual:\n\n{actual:#?}\n\n" + ); + + }; +} + +#[tokio::test] +async fn test_remove_unnecessary_sort() -> Result<()> { + let schema = create_test_schema()?; + let source = memory_exec(&schema); + let input = sort_exec(vec![sort_expr("non_nullable_col", &schema)], source); + let physical_plan = sort_exec(vec![sort_expr("nullable_col", &schema)], input); + + let expected_input = [ + "SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", + " SortExec: expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + let expected_optimized = [ + "SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + + Ok(()) +} + +#[tokio::test] +async fn test_remove_unnecessary_sort_window_multilayer() -> Result<()> { + let schema = create_test_schema()?; + let source = memory_exec(&schema); + + let sort_exprs = vec![sort_expr_options( + "non_nullable_col", + &source.schema(), + SortOptions { + descending: true, + nulls_first: true, + }, + )]; + let sort = sort_exec(sort_exprs.clone(), source); + // Add dummy layer propagating Sort above, to test whether sort can be removed from multi layer before + let coalesce_batches = coalesce_batches_exec(sort); + + let window_agg = + bounded_window_exec("non_nullable_col", sort_exprs, coalesce_batches); + + let sort_exprs = vec![sort_expr_options( + "non_nullable_col", + &window_agg.schema(), + SortOptions { + descending: false, + nulls_first: false, + }, + )]; + + let sort = sort_exec(sort_exprs.clone(), window_agg); + + // Add dummy layer propagating Sort above, to test whether sort can be removed from multi layer before + let filter = filter_exec( + Arc::new(NotExpr::new( + col("non_nullable_col", schema.as_ref()).unwrap(), + )), + sort, + ); + + let physical_plan = bounded_window_exec("non_nullable_col", sort_exprs, filter); + + 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]", + " FilterExec: NOT non_nullable_col@1", + " SortExec: expr=[non_nullable_col@1 ASC NULLS LAST], preserve_partitioning=[false]", + " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", + " CoalesceBatchesExec: target_batch_size=128", + " SortExec: expr=[non_nullable_col@1 DESC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]"]; + + let expected_optimized = ["WindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(NULL), is_causal: false }]", + " FilterExec: NOT non_nullable_col@1", + " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", + " CoalesceBatchesExec: target_batch_size=128", + " SortExec: expr=[non_nullable_col@1 DESC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]"]; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + + Ok(()) +} + +#[tokio::test] +async fn test_add_required_sort() -> Result<()> { + let schema = create_test_schema()?; + let source = memory_exec(&schema); + + let sort_exprs = vec![sort_expr("nullable_col", &schema)]; + + let physical_plan = sort_preserving_merge_exec(sort_exprs, source); + + let expected_input = [ + "SortPreservingMergeExec: [nullable_col@0 ASC]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + let expected_optimized = [ + "SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + + Ok(()) +} + +#[tokio::test] +async fn test_remove_unnecessary_sort1() -> Result<()> { + let schema = create_test_schema()?; + let source = memory_exec(&schema); + let sort_exprs = vec![sort_expr("nullable_col", &schema)]; + let sort = sort_exec(sort_exprs.clone(), source); + let spm = sort_preserving_merge_exec(sort_exprs, sort); + + let sort_exprs = vec![sort_expr("nullable_col", &schema)]; + let sort = sort_exec(sort_exprs.clone(), spm); + let physical_plan = sort_preserving_merge_exec(sort_exprs, sort); + let expected_input = [ + "SortPreservingMergeExec: [nullable_col@0 ASC]", + " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", + " SortPreservingMergeExec: [nullable_col@0 ASC]", + " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + let expected_optimized = [ + "SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + + Ok(()) +} + +#[tokio::test] +async fn test_remove_unnecessary_sort2() -> Result<()> { + let schema = create_test_schema()?; + let source = memory_exec(&schema); + let sort_exprs = vec![sort_expr("non_nullable_col", &schema)]; + let sort = sort_exec(sort_exprs.clone(), source); + let spm = sort_preserving_merge_exec(sort_exprs, sort); + + let sort_exprs = vec![ + sort_expr("nullable_col", &schema), + sort_expr("non_nullable_col", &schema), + ]; + let sort2 = sort_exec(sort_exprs.clone(), spm); + let spm2 = sort_preserving_merge_exec(sort_exprs, sort2); + + let sort_exprs = vec![sort_expr("nullable_col", &schema)]; + let sort3 = sort_exec(sort_exprs, spm2); + 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", + " 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]", + " SortPreservingMergeExec: [non_nullable_col@1 ASC]", + " SortExec: expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + + let expected_optimized = [ + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + + Ok(()) +} + +#[tokio::test] +async fn test_remove_unnecessary_sort3() -> Result<()> { + let schema = create_test_schema()?; + let source = memory_exec(&schema); + let sort_exprs = vec![sort_expr("non_nullable_col", &schema)]; + let sort = sort_exec(sort_exprs.clone(), source); + let spm = sort_preserving_merge_exec(sort_exprs, sort); + + let sort_exprs = LexOrdering::new(vec![ + sort_expr("nullable_col", &schema), + sort_expr("non_nullable_col", &schema), + ]); + let repartition_exec = repartition_exec(spm); + let sort2 = Arc::new( + SortExec::new(sort_exprs.clone(), repartition_exec) + .with_preserve_partitioning(true), + ) as _; + let spm2 = sort_preserving_merge_exec(sort_exprs, sort2); + + let physical_plan = aggregate_exec(spm2); + + // When removing a `SortPreservingMergeExec`, make sure that partitioning + // requirements are not violated. In some cases, we may need to replace + // it with a `CoalescePartitionsExec` instead of directly removing it. + let expected_input = [ + "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", + " SortPreservingMergeExec: [non_nullable_col@1 ASC]", + " SortExec: expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + + let expected_optimized = [ + "AggregateExec: mode=Final, gby=[], aggr=[]", + " CoalescePartitionsExec", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + + Ok(()) +} + +#[tokio::test] +async fn test_remove_unnecessary_sort4() -> Result<()> { + let schema = create_test_schema()?; + let source1 = repartition_exec(memory_exec(&schema)); + + let source2 = repartition_exec(memory_exec(&schema)); + let union = union_exec(vec![source1, source2]); + + let sort_exprs = LexOrdering::new(vec![sort_expr("non_nullable_col", &schema)]); + // let sort = sort_exec(sort_exprs.clone(), union); + let sort = Arc::new( + SortExec::new(sort_exprs.clone(), union).with_preserve_partitioning(true), + ) as _; + let spm = sort_preserving_merge_exec(sort_exprs, sort); + + let filter = filter_exec( + Arc::new(NotExpr::new( + col("non_nullable_col", schema.as_ref()).unwrap(), + )), + spm, + ); + + let sort_exprs = vec![ + sort_expr("nullable_col", &schema), + sort_expr("non_nullable_col", &schema), + ]; + let physical_plan = sort_exec(sort_exprs, filter); + + // When removing a `SortPreservingMergeExec`, make sure that partitioning + // requirements are not violated. In some cases, we may need to replace + // it with a `CoalescePartitionsExec` instead of directly removing it. + let expected_input = ["SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", + " FilterExec: NOT non_nullable_col@1", + " SortPreservingMergeExec: [non_nullable_col@1 ASC]", + " SortExec: expr=[non_nullable_col@1 ASC], preserve_partitioning=[true]", + " UnionExec", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " MemoryExec: partitions=1, partition_sizes=[0]", + " RepartitionExec: partitioning=RoundRobinBatch(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", + " MemoryExec: partitions=1, partition_sizes=[0]", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " MemoryExec: partitions=1, partition_sizes=[0]"]; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + + Ok(()) +} + +#[tokio::test] +async fn test_remove_unnecessary_sort6() -> Result<()> { + let schema = create_test_schema()?; + let source = memory_exec(&schema); + let input = Arc::new( + SortExec::new( + LexOrdering::new(vec![sort_expr("non_nullable_col", &schema)]), + source, + ) + .with_fetch(Some(2)), + ); + let physical_plan = sort_exec( + vec![ + sort_expr("non_nullable_col", &schema), + sort_expr("nullable_col", &schema), + ], + input, + ); + + let expected_input = [ + "SortExec: expr=[non_nullable_col@1 ASC, nullable_col@0 ASC], preserve_partitioning=[false]", + " SortExec: TopK(fetch=2), expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + let expected_optimized = [ + "SortExec: TopK(fetch=2), expr=[non_nullable_col@1 ASC, nullable_col@0 ASC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + + Ok(()) +} + +#[tokio::test] +async fn test_remove_unnecessary_sort7() -> Result<()> { + let schema = create_test_schema()?; + let source = memory_exec(&schema); + let input = Arc::new(SortExec::new( + LexOrdering::new(vec![ + sort_expr("non_nullable_col", &schema), + sort_expr("nullable_col", &schema), + ]), + source, + )); + + let physical_plan = Arc::new( + SortExec::new( + LexOrdering::new(vec![sort_expr("non_nullable_col", &schema)]), + input, + ) + .with_fetch(Some(2)), + ) as Arc; + + let expected_input = [ + "SortExec: TopK(fetch=2), expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", + " SortExec: expr=[non_nullable_col@1 ASC, nullable_col@0 ASC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + let expected_optimized = [ + "GlobalLimitExec: skip=0, fetch=2", + " SortExec: expr=[non_nullable_col@1 ASC, nullable_col@0 ASC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + + Ok(()) +} + +#[tokio::test] +async fn test_remove_unnecessary_sort8() -> Result<()> { + let schema = create_test_schema()?; + let source = memory_exec(&schema); + let input = Arc::new(SortExec::new( + LexOrdering::new(vec![sort_expr("non_nullable_col", &schema)]), + source, + )); + let limit = Arc::new(LocalLimitExec::new(input, 2)); + let physical_plan = sort_exec( + vec![ + sort_expr("non_nullable_col", &schema), + sort_expr("nullable_col", &schema), + ], + limit, + ); + + let expected_input = [ + "SortExec: expr=[non_nullable_col@1 ASC, nullable_col@0 ASC], preserve_partitioning=[false]", + " LocalLimitExec: fetch=2", + " SortExec: expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + let expected_optimized = [ + "LocalLimitExec: fetch=2", + " SortExec: TopK(fetch=2), expr=[non_nullable_col@1 ASC, nullable_col@0 ASC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + + Ok(()) +} + +#[tokio::test] +async fn test_do_not_pushdown_through_limit() -> Result<()> { + let schema = create_test_schema()?; + let source = memory_exec(&schema); + // let input = sort_exec(vec![sort_expr("non_nullable_col", &schema)], source); + let input = Arc::new(SortExec::new( + LexOrdering::new(vec![sort_expr("non_nullable_col", &schema)]), + source, + )); + let limit = Arc::new(GlobalLimitExec::new(input, 0, Some(5))) as _; + let physical_plan = sort_exec(vec![sort_expr("nullable_col", &schema)], limit); + + let expected_input = [ + "SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", + " GlobalLimitExec: skip=0, fetch=5", + " SortExec: expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + let expected_optimized = [ + "SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", + " GlobalLimitExec: skip=0, fetch=5", + " SortExec: expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + + Ok(()) +} + +#[tokio::test] +async fn test_remove_unnecessary_spm1() -> Result<()> { + let schema = create_test_schema()?; + let source = memory_exec(&schema); + let input = + sort_preserving_merge_exec(vec![sort_expr("non_nullable_col", &schema)], source); + let input2 = + sort_preserving_merge_exec(vec![sort_expr("non_nullable_col", &schema)], input); + let physical_plan = + sort_preserving_merge_exec(vec![sort_expr("nullable_col", &schema)], input2); + + let expected_input = [ + "SortPreservingMergeExec: [nullable_col@0 ASC]", + " SortPreservingMergeExec: [non_nullable_col@1 ASC]", + " SortPreservingMergeExec: [non_nullable_col@1 ASC]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + let expected_optimized = [ + "SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + + Ok(()) +} + +#[tokio::test] +async fn test_change_wrong_sorting() -> Result<()> { + let schema = create_test_schema()?; + let source = memory_exec(&schema); + let sort_exprs = vec![ + sort_expr("nullable_col", &schema), + sort_expr("non_nullable_col", &schema), + ]; + let sort = sort_exec(vec![sort_exprs[0].clone()], source); + let physical_plan = sort_preserving_merge_exec(sort_exprs, sort); + let expected_input = [ + "SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", + " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + let expected_optimized = [ + "SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + + Ok(()) +} + +#[tokio::test] +async fn test_change_wrong_sorting2() -> Result<()> { + let schema = create_test_schema()?; + let source = memory_exec(&schema); + let sort_exprs = vec![ + sort_expr("nullable_col", &schema), + sort_expr("non_nullable_col", &schema), + ]; + let spm1 = sort_preserving_merge_exec(sort_exprs.clone(), source); + let sort2 = sort_exec(vec![sort_exprs[0].clone()], spm1); + let physical_plan = sort_preserving_merge_exec(vec![sort_exprs[1].clone()], sort2); + + let expected_input = [ + "SortPreservingMergeExec: [non_nullable_col@1 ASC]", + " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", + " SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + let expected_optimized = [ + "SortExec: expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + + Ok(()) +} + +#[tokio::test] +async fn test_multiple_sort_window_exec() -> Result<()> { + let schema = create_test_schema()?; + let source = memory_exec(&schema); + + let sort_exprs1 = vec![sort_expr("nullable_col", &schema)]; + let sort_exprs2 = vec![ + sort_expr("nullable_col", &schema), + sort_expr("non_nullable_col", &schema), + ]; + + let sort1 = sort_exec(sort_exprs1.clone(), source); + let window_agg1 = bounded_window_exec("non_nullable_col", sort_exprs1.clone(), sort1); + let window_agg2 = bounded_window_exec("non_nullable_col", sort_exprs2, window_agg1); + // let filter_exec = sort_exec; + let physical_plan = bounded_window_exec("non_nullable_col", sort_exprs1, window_agg2); + + 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]", + " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", + " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", + " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]"]; + + let expected_optimized = ["BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", + " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", + " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]"]; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + + Ok(()) +} + +#[tokio::test] +// With new change in SortEnforcement EnforceSorting->EnforceDistribution->EnforceSorting +// should produce same result with EnforceDistribution+EnforceSorting +// This enables us to use EnforceSorting possibly before EnforceDistribution +// Given that it will be called at least once after last EnforceDistribution. The reason is that +// EnforceDistribution may invalidate ordering invariant. +async fn test_commutativity() -> Result<()> { + let schema = create_test_schema()?; + let config = ConfigOptions::new(); + + let memory_exec = memory_exec(&schema); + let sort_exprs = LexOrdering::new(vec![sort_expr("nullable_col", &schema)]); + let window = bounded_window_exec("nullable_col", sort_exprs.clone(), memory_exec); + let repartition = repartition_exec(window); + + let orig_plan = + Arc::new(SortExec::new(sort_exprs, repartition)) as Arc; + 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", + " 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]", + ]; + assert_eq!( + expected_input, actual, + "\n**Original Plan Mismatch\n\nexpected:\n\n{expected_input:#?}\nactual:\n\n{actual:#?}\n\n" + ); + + let mut plan = orig_plan.clone(); + let rules = vec![ + Arc::new(EnforceDistribution::new()) as Arc, + Arc::new(EnforceSorting::new()) as Arc, + ]; + for rule in rules { + plan = rule.optimize(plan, &config)?; + } + let first_plan = plan.clone(); + + let mut plan = orig_plan.clone(); + let rules = vec![ + Arc::new(EnforceSorting::new()) as Arc, + Arc::new(EnforceDistribution::new()) as Arc, + Arc::new(EnforceSorting::new()) as Arc, + ]; + for rule in rules { + plan = rule.optimize(plan, &config)?; + } + let second_plan = plan.clone(); + + assert_eq!(get_plan_string(&first_plan), get_plan_string(&second_plan)); + Ok(()) +} + +#[tokio::test] +async fn test_coalesce_propagate() -> Result<()> { + let schema = create_test_schema()?; + let source = memory_exec(&schema); + let repartition = repartition_exec(source); + let coalesce_partitions = Arc::new(CoalescePartitionsExec::new(repartition)); + let repartition = repartition_exec(coalesce_partitions); + let sort_exprs = LexOrdering::new(vec![sort_expr("nullable_col", &schema)]); + // Add local sort + let sort = Arc::new( + SortExec::new(sort_exprs.clone(), repartition).with_preserve_partitioning(true), + ) as _; + let spm = sort_preserving_merge_exec(sort_exprs.clone(), sort); + let sort = sort_exec(sort_exprs, spm); + + let physical_plan = sort.clone(); + // Sort Parallelize rule should end Coalesce + Sort linkage when Sort is Global Sort + // Also input plan is not valid as it is. We need to add SortExec before SortPreservingMergeExec. + 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", + " CoalescePartitionsExec", + " RepartitionExec: partitioning=RoundRobinBatch(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", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + + Ok(()) +} + +#[tokio::test] +async fn test_replace_with_partial_sort2() -> Result<()> { + let schema = create_test_schema3()?; + let input_sort_exprs = vec![sort_expr("a", &schema), sort_expr("c", &schema)]; + let unbounded_input = stream_exec_ordered(&schema, input_sort_exprs); + + let physical_plan = sort_exec( + vec![ + sort_expr("a", &schema), + sort_expr("c", &schema), + sort_expr("d", &schema), + ], + unbounded_input, + ); + + let expected_input = [ + "SortExec: expr=[a@0 ASC, c@2 ASC, d@3 ASC], preserve_partitioning=[false]", + " StreamingTableExec: partition_sizes=1, projection=[a, b, c, d, e], infinite_source=true, output_ordering=[a@0 ASC, c@2 ASC]" + ]; + // let optimized + let expected_optimized = [ + "PartialSortExec: expr=[a@0 ASC, c@2 ASC, d@3 ASC], common_prefix_length=[2]", + " StreamingTableExec: partition_sizes=1, projection=[a, b, c, d, e], infinite_source=true, output_ordering=[a@0 ASC, c@2 ASC]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + Ok(()) +} + +#[tokio::test] +async fn test_push_with_required_input_ordering_prohibited() -> Result<()> { + // SortExec: expr=[b] <-- can't push this down + // RequiredInputOrder expr=[a] <-- this requires input sorted by a, and preserves the input order + // SortExec: expr=[a] + // MemoryExec + let schema = create_test_schema3()?; + let sort_exprs_a = LexOrdering::new(vec![sort_expr("a", &schema)]); + let sort_exprs_b = LexOrdering::new(vec![sort_expr("b", &schema)]); + let plan = memory_exec(&schema); + let plan = sort_exec(sort_exprs_a.clone(), plan); + let plan = RequirementsTestExec::new(plan) + .with_required_input_ordering(sort_exprs_a) + .with_maintains_input_order(true) + .into_arc(); + let plan = sort_exec(sort_exprs_b, plan); + + let expected_input = [ + "SortExec: expr=[b@1 ASC], preserve_partitioning=[false]", + " RequiredInputOrderingExec", + " SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + // should not be able to push shorts + let expected_no_change = expected_input; + assert_optimized!(expected_input, expected_no_change, plan, true); + Ok(()) +} + +// test when the required input ordering is satisfied so could push through +#[tokio::test] +async fn test_push_with_required_input_ordering_allowed() -> Result<()> { + // SortExec: expr=[a,b] <-- can push this down (as it is compatible with the required input ordering) + // RequiredInputOrder expr=[a] <-- this requires input sorted by a, and preserves the input order + // SortExec: expr=[a] + // MemoryExec + let schema = create_test_schema3()?; + let sort_exprs_a = LexOrdering::new(vec![sort_expr("a", &schema)]); + let sort_exprs_ab = + LexOrdering::new(vec![sort_expr("a", &schema), sort_expr("b", &schema)]); + let plan = memory_exec(&schema); + let plan = sort_exec(sort_exprs_a.clone(), plan); + let plan = RequirementsTestExec::new(plan) + .with_required_input_ordering(sort_exprs_a) + .with_maintains_input_order(true) + .into_arc(); + let plan = sort_exec(sort_exprs_ab, plan); + + let expected_input = [ + "SortExec: expr=[a@0 ASC, b@1 ASC], preserve_partitioning=[false]", + " RequiredInputOrderingExec", + " SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + // should able to push shorts + let expected = [ + "RequiredInputOrderingExec", + " SortExec: expr=[a@0 ASC, b@1 ASC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + assert_optimized!(expected_input, expected, plan, true); + Ok(()) +} + +#[tokio::test] +async fn test_replace_with_partial_sort() -> Result<()> { + let schema = create_test_schema3()?; + let input_sort_exprs = vec![sort_expr("a", &schema)]; + let unbounded_input = stream_exec_ordered(&schema, input_sort_exprs); + + let physical_plan = sort_exec( + vec![sort_expr("a", &schema), sort_expr("c", &schema)], + unbounded_input, + ); + + let expected_input = [ + "SortExec: expr=[a@0 ASC, c@2 ASC], preserve_partitioning=[false]", + " StreamingTableExec: partition_sizes=1, projection=[a, b, c, d, e], infinite_source=true, output_ordering=[a@0 ASC]" + ]; + let expected_optimized = [ + "PartialSortExec: expr=[a@0 ASC, c@2 ASC], common_prefix_length=[1]", + " StreamingTableExec: partition_sizes=1, projection=[a, b, c, d, e], infinite_source=true, output_ordering=[a@0 ASC]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + Ok(()) +} + +#[tokio::test] +async fn test_not_replaced_with_partial_sort_for_unbounded_input() -> Result<()> { + let schema = create_test_schema3()?; + let input_sort_exprs = vec![sort_expr("b", &schema), sort_expr("c", &schema)]; + let unbounded_input = stream_exec_ordered(&schema, input_sort_exprs); + + let physical_plan = sort_exec( + vec![ + sort_expr("a", &schema), + sort_expr("b", &schema), + sort_expr("c", &schema), + ], + unbounded_input, + ); + let expected_input = [ + "SortExec: expr=[a@0 ASC, b@1 ASC, c@2 ASC], preserve_partitioning=[false]", + " StreamingTableExec: partition_sizes=1, projection=[a, b, c, d, e], infinite_source=true, output_ordering=[b@1 ASC, c@2 ASC]" + ]; + let expected_no_change = expected_input; + assert_optimized!(expected_input, expected_no_change, physical_plan, true); + Ok(()) +} diff --git a/datafusion/physical-optimizer/tests/limited_distinct_aggregation.rs b/datafusion/physical-optimizer/tests/limited_distinct_aggregation.rs new file mode 100644 index 000000000000..6427d4def1a8 --- /dev/null +++ b/datafusion/physical-optimizer/tests/limited_distinct_aggregation.rs @@ -0,0 +1,131 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +//! Tests for [`LimitedDistinctAggregation`] physical optimizer rule + +use std::sync::Arc; + +use arrow_schema::DataType; +use datafusion_common::Result; +use datafusion_expr::Operator; +use datafusion_physical_expr::expressions::{cast, col}; +use datafusion_physical_optimizer::test_utils::{ + assert_plan_matches_expected, build_group_by, mock_data, TestAggregate, +}; +use datafusion_physical_plan::{ + aggregates::{AggregateExec, AggregateMode}, + expressions, + limit::LocalLimitExec, + ExecutionPlan, +}; + +#[test] +fn test_no_group_by() -> Result<()> { + let source = mock_data()?; + let schema = source.schema(); + + // `SELECT FROM MemoryExec LIMIT 10;`, Single AggregateExec + let single_agg = AggregateExec::try_new( + AggregateMode::Single, + build_group_by(&schema, vec![]), + vec![], /* aggr_expr */ + vec![], /* filter_expr */ + source, /* input */ + schema, /* input_schema */ + )?; + let limit_exec = LocalLimitExec::new( + Arc::new(single_agg), + 10, // fetch + ); + // expected not to push the limit to the AggregateExec + let expected = [ + "LocalLimitExec: fetch=10", + "AggregateExec: mode=Single, gby=[], aggr=[]", + "MemoryExec: partitions=1, partition_sizes=[1]", + ]; + let plan: Arc = Arc::new(limit_exec); + assert_plan_matches_expected(&plan, &expected)?; + Ok(()) +} + +#[test] +fn test_has_aggregate_expression() -> Result<()> { + let source = mock_data()?; + let schema = source.schema(); + let agg = TestAggregate::new_count_star(); + + // `SELECT FROM MemoryExec LIMIT 10;`, Single AggregateExec + let single_agg = AggregateExec::try_new( + AggregateMode::Single, + build_group_by(&schema, vec!["a".to_string()]), + vec![Arc::new(agg.count_expr(&schema))], /* aggr_expr */ + vec![None], /* filter_expr */ + source, /* input */ + schema.clone(), /* input_schema */ + )?; + let limit_exec = LocalLimitExec::new( + Arc::new(single_agg), + 10, // fetch + ); + // expected not to push the limit to the AggregateExec + let expected = [ + "LocalLimitExec: fetch=10", + "AggregateExec: mode=Single, gby=[a@0 as a], aggr=[COUNT(*)]", + "MemoryExec: partitions=1, partition_sizes=[1]", + ]; + let plan: Arc = Arc::new(limit_exec); + assert_plan_matches_expected(&plan, &expected)?; + Ok(()) +} + +#[test] +fn test_has_filter() -> Result<()> { + let source = mock_data()?; + let schema = source.schema(); + + // `SELECT a FROM MemoryExec WHERE a > 1 GROUP BY a LIMIT 10;`, Single AggregateExec + // the `a > 1` filter is applied in the AggregateExec + let filter_expr = Some(expressions::binary( + col("a", &schema)?, + Operator::Gt, + cast(expressions::lit(1u32), &schema, DataType::Int32)?, + &schema, + )?); + let agg = TestAggregate::new_count_star(); + let single_agg = AggregateExec::try_new( + AggregateMode::Single, + build_group_by(&schema.clone(), vec!["a".to_string()]), + vec![Arc::new(agg.count_expr(&schema))], /* aggr_expr */ + vec![filter_expr], /* filter_expr */ + source, /* input */ + schema.clone(), /* input_schema */ + )?; + let limit_exec = LocalLimitExec::new( + Arc::new(single_agg), + 10, // fetch + ); + // expected not to push the limit to the AggregateExec + // TODO(msirek): open an issue for `filter_expr` of `AggregateExec` not printing out + let expected = [ + "LocalLimitExec: fetch=10", + "AggregateExec: mode=Single, gby=[a@0 as a], aggr=[COUNT(*)]", + "MemoryExec: partitions=1, partition_sizes=[1]", + ]; + let plan: Arc = Arc::new(limit_exec); + assert_plan_matches_expected(&plan, &expected)?; + Ok(()) +} diff --git a/datafusion/physical-optimizer/tests/mod.rs b/datafusion/physical-optimizer/tests/mod.rs new file mode 100644 index 000000000000..0e5eab515590 --- /dev/null +++ b/datafusion/physical-optimizer/tests/mod.rs @@ -0,0 +1,20 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +mod enforce_sorting; +mod limited_distinct_aggregation; +mod sanity_checker; diff --git a/datafusion/physical-optimizer/tests/sanity_checker.rs b/datafusion/physical-optimizer/tests/sanity_checker.rs new file mode 100644 index 000000000000..e21385cd9fd0 --- /dev/null +++ b/datafusion/physical-optimizer/tests/sanity_checker.rs @@ -0,0 +1,308 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +//! Tests for [`SanityCheckPlan`] physical optimizer rule + +use std::sync::Arc; + +use arrow::compute::SortOptions; +use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; +use datafusion_common::config::ConfigOptions; +use datafusion_common::Result; +use datafusion_expr::JoinType; +use datafusion_physical_expr::expressions::col; +use datafusion_physical_expr::Partitioning; +use datafusion_physical_optimizer::test_utils::{ + bounded_window_exec, global_limit_exec, local_limit_exec, memory_exec, + repartition_exec, sort_exec, sort_expr_options, sort_merge_join_exec, +}; +use datafusion_physical_optimizer::{sanity_checker::*, PhysicalOptimizerRule}; +use datafusion_physical_plan::displayable; +use datafusion_physical_plan::repartition::RepartitionExec; +use datafusion_physical_plan::ExecutionPlan; + +fn create_test_schema() -> SchemaRef { + Arc::new(Schema::new(vec![Field::new("c9", DataType::Int32, true)])) +} + +fn create_test_schema2() -> SchemaRef { + Arc::new(Schema::new(vec![ + Field::new("a", DataType::Int32, true), + Field::new("b", DataType::Int32, true), + ])) +} + +/// Check if sanity checker should accept or reject plans. +fn assert_sanity_check(plan: &Arc, is_sane: bool) { + let sanity_checker = SanityCheckPlan::new(); + let opts = ConfigOptions::default(); + assert_eq!( + sanity_checker.optimize(plan.clone(), &opts).is_ok(), + is_sane + ); +} + +/// Check if the plan we created is as expected by comparing the plan +/// formatted as a string. +fn assert_plan(plan: &dyn ExecutionPlan, expected_lines: Vec<&str>) { + let plan_str = displayable(plan).indent(true).to_string(); + let actual_lines: Vec<&str> = plan_str.trim().lines().collect(); + assert_eq!(actual_lines, expected_lines); +} + +#[tokio::test] +/// Tests that plan is valid when the sort requirements are satisfied. +async fn test_bounded_window_agg_sort_requirement() -> Result<()> { + let schema = create_test_schema(); + let source = memory_exec(&schema); + let sort_exprs = vec![sort_expr_options( + "c9", + &source.schema(), + SortOptions { + descending: false, + nulls_first: false, + }, + )]; + let sort = sort_exec(sort_exprs.clone(), source); + let bw = bounded_window_exec("c9", sort_exprs, sort); + assert_plan(bw.as_ref(), vec![ + "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=[c9@0 ASC NULLS LAST], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]" + ]); + assert_sanity_check(&bw, true); + Ok(()) +} + +#[tokio::test] +/// Tests that plan is invalid when the sort requirements are not satisfied. +async fn test_bounded_window_agg_no_sort_requirement() -> Result<()> { + let schema = create_test_schema(); + let source = memory_exec(&schema); + let sort_exprs = vec![sort_expr_options( + "c9", + &source.schema(), + SortOptions { + descending: false, + nulls_first: false, + }, + )]; + let bw = bounded_window_exec("c9", sort_exprs, source); + assert_plan(bw.as_ref(), vec![ + "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]" + ]); + // Order requirement of the `BoundedWindowAggExec` is not satisfied. We expect to receive error during sanity check. + assert_sanity_check(&bw, false); + Ok(()) +} + +#[tokio::test] +/// A valid when a single partition requirement +/// is satisfied. +async fn test_global_limit_single_partition() -> Result<()> { + let schema = create_test_schema(); + let source = memory_exec(&schema); + let limit = global_limit_exec(source); + + assert_plan( + limit.as_ref(), + vec![ + "GlobalLimitExec: skip=0, fetch=100", + " MemoryExec: partitions=1, partition_sizes=[0]", + ], + ); + assert_sanity_check(&limit, true); + Ok(()) +} + +#[tokio::test] +/// An invalid plan when a single partition requirement +/// is not satisfied. +async fn test_global_limit_multi_partition() -> Result<()> { + let schema = create_test_schema(); + let source = memory_exec(&schema); + let limit = global_limit_exec(repartition_exec(source)); + + assert_plan( + limit.as_ref(), + vec![ + "GlobalLimitExec: skip=0, fetch=100", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " MemoryExec: partitions=1, partition_sizes=[0]", + ], + ); + // Distribution requirement of the `GlobalLimitExec` is not satisfied. We expect to receive error during sanity check. + assert_sanity_check(&limit, false); + Ok(()) +} + +#[tokio::test] +/// A plan with no requirements should satisfy. +async fn test_local_limit() -> Result<()> { + let schema = create_test_schema(); + let source = memory_exec(&schema); + let limit = local_limit_exec(source); + + assert_plan( + limit.as_ref(), + vec![ + "LocalLimitExec: fetch=100", + " MemoryExec: partitions=1, partition_sizes=[0]", + ], + ); + assert_sanity_check(&limit, true); + Ok(()) +} + +#[tokio::test] +/// Valid plan with multiple children satisfy both order and distribution. +async fn test_sort_merge_join_satisfied() -> Result<()> { + let schema1 = create_test_schema(); + let schema2 = create_test_schema2(); + let source1 = memory_exec(&schema1); + let source2 = memory_exec(&schema2); + let sort_opts = SortOptions::default(); + let sort_exprs1 = vec![sort_expr_options("c9", &source1.schema(), sort_opts)]; + 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 left_jcol = col("c9", &left.schema()).unwrap(); + let right_jcol = col("a", &right.schema()).unwrap(); + let left = Arc::new(RepartitionExec::try_new( + left, + Partitioning::Hash(vec![left_jcol.clone()], 10), + )?); + + let right = Arc::new(RepartitionExec::try_new( + right, + Partitioning::Hash(vec![right_jcol.clone()], 10), + )?); + + let join_on = vec![(left_jcol as _, right_jcol as _)]; + let join_ty = JoinType::Inner; + let smj = sort_merge_join_exec(left, right, &join_on, &join_ty); + + assert_plan( + smj.as_ref(), + vec![ + "SortMergeJoin: join_type=Inner, on=[(c9@0, a@0)]", + " 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=Hash([a@0], 10), input_partitions=1", + " SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ], + ); + assert_sanity_check(&smj, true); + Ok(()) +} + +#[tokio::test] +/// Invalid case when the order is not satisfied by the 2nd +/// child. +async fn test_sort_merge_join_order_missing() -> Result<()> { + let schema1 = create_test_schema(); + let schema2 = create_test_schema2(); + let source1 = memory_exec(&schema1); + let right = memory_exec(&schema2); + let sort_exprs1 = vec![sort_expr_options( + "c9", + &source1.schema(), + SortOptions::default(), + )]; + let left = sort_exec(sort_exprs1, source1); + // Missing sort of the right child here.. + let left_jcol = col("c9", &left.schema()).unwrap(); + let right_jcol = col("a", &right.schema()).unwrap(); + let left = Arc::new(RepartitionExec::try_new( + left, + Partitioning::Hash(vec![left_jcol.clone()], 10), + )?); + + let right = Arc::new(RepartitionExec::try_new( + right, + Partitioning::Hash(vec![right_jcol.clone()], 10), + )?); + + let join_on = vec![(left_jcol as _, right_jcol as _)]; + let join_ty = JoinType::Inner; + let smj = sort_merge_join_exec(left, right, &join_on, &join_ty); + + assert_plan( + smj.as_ref(), + vec![ + "SortMergeJoin: join_type=Inner, on=[(c9@0, a@0)]", + " 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=Hash([a@0], 10), input_partitions=1", + " MemoryExec: partitions=1, partition_sizes=[0]", + ], + ); + // Order requirement for the `SortMergeJoin` is not satisfied for right child. We expect to receive error during sanity check. + assert_sanity_check(&smj, false); + Ok(()) +} + +#[tokio::test] +/// Invalid case when the distribution is not satisfied by the 2nd +/// child. +async fn test_sort_merge_join_dist_missing() -> Result<()> { + let schema1 = create_test_schema(); + let schema2 = create_test_schema2(); + let source1 = memory_exec(&schema1); + let source2 = memory_exec(&schema2); + let sort_opts = SortOptions::default(); + let sort_exprs1 = vec![sort_expr_options("c9", &source1.schema(), sort_opts)]; + 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( + right, + Partitioning::RoundRobinBatch(10), + )?); + let left_jcol = col("c9", &left.schema()).unwrap(); + let right_jcol = col("a", &right.schema()).unwrap(); + let left = Arc::new(RepartitionExec::try_new( + left, + Partitioning::Hash(vec![left_jcol.clone()], 10), + )?); + + // Missing hash partitioning on right child. + + let join_on = vec![(left_jcol as _, right_jcol as _)]; + let join_ty = JoinType::Inner; + let smj = sort_merge_join_exec(left, right, &join_on, &join_ty); + + assert_plan( + smj.as_ref(), + vec![ + "SortMergeJoin: join_type=Inner, on=[(c9@0, a@0)]", + " 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", + " SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ], + ); + // Distribution requirement for the `SortMergeJoin` is not satisfied for right child (has round-robin partitioning). We expect to receive error during sanity check. + assert_sanity_check(&smj, false); + Ok(()) +}