diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index b267b3fea485..86567f487c1b 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -1599,6 +1599,7 @@ dependencies = [ "itertools 0.14.0", "log", "recursive", + "url", ] [[package]] diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index fbadceba0948..4681efaf16b2 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -26,10 +26,6 @@ use std::sync::Arc; use crate::config::ConfigOptions; use crate::error::Result; -use crate::physical_optimizer::utils::{ - add_sort_above_with_check, is_coalesce_partitions, is_repartition, - is_sort_preserving_merge, -}; use crate::physical_plan::aggregates::{AggregateExec, AggregateMode, PhysicalGroupBy}; use crate::physical_plan::coalesce_partitions::CoalescePartitionsExec; use crate::physical_plan::joins::{ @@ -54,6 +50,10 @@ use datafusion_physical_expr::{ }; 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}; diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs deleted file mode 100644 index 76bb49efb64e..000000000000 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ /dev/null @@ -1,2587 +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. - -//! EnforceSorting optimizer rule inspects the physical plan with respect -//! to local sorting requirements and does the following: -//! - Adds a [`SortExec`] when a requirement is not met, -//! - Removes an already-existing [`SortExec`] if it is possible to prove -//! that this sort is unnecessary -//! -//! The rule can work on valid *and* invalid physical plans with respect to -//! sorting requirements, but always produces a valid physical plan in this sense. -//! -//! A non-realistic but easy to follow example for sort removals: Assume that we -//! somehow get the fragment -//! -//! ```text -//! SortExec: expr=[nullable_col@0 ASC] -//! SortExec: expr=[non_nullable_col@1 ASC] -//! ``` -//! -//! in the physical plan. The first sort is unnecessary since its result is overwritten -//! by another [`SortExec`]. Therefore, this rule removes it from the physical plan. - -use std::sync::Arc; - -use super::utils::{add_sort_above, add_sort_above_with_check}; -use crate::config::ConfigOptions; -use crate::error::Result; -use crate::physical_optimizer::replace_with_order_preserving_variants::{ - replace_with_order_preserving_variants, OrderPreservationContext, -}; -use crate::physical_optimizer::sort_pushdown::{ - assign_initial_requirements, pushdown_sorts, SortPushDown, -}; -use crate::physical_optimizer::utils::{ - is_coalesce_partitions, is_limit, is_repartition, is_sort, is_sort_preserving_merge, - is_union, is_window, -}; -use crate::physical_plan::coalesce_partitions::CoalescePartitionsExec; -use crate::physical_plan::sorts::sort::SortExec; -use crate::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; -use crate::physical_plan::tree_node::PlanContext; -use crate::physical_plan::windows::{ - get_best_fitting_window, BoundedWindowAggExec, WindowAggExec, -}; -use crate::physical_plan::{Distribution, ExecutionPlan, InputOrderMode}; - -use datafusion_common::plan_err; -use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; -use datafusion_physical_expr::Partitioning; -use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; -use datafusion_physical_optimizer::PhysicalOptimizerRule; -use datafusion_physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; -use datafusion_physical_plan::repartition::RepartitionExec; -use datafusion_physical_plan::sorts::partial_sort::PartialSortExec; -use datafusion_physical_plan::ExecutionPlanProperties; - -use itertools::izip; - -/// This rule inspects [`SortExec`]'s in the given physical plan and removes the -/// ones it can prove unnecessary. -#[derive(Default, Debug)] -pub struct EnforceSorting {} - -impl EnforceSorting { - #[allow(missing_docs)] - pub fn new() -> Self { - Self {} - } -} - -/// This object is used within the [`EnforceSorting`] rule to track the closest -/// [`SortExec`] descendant(s) for every child of a plan. The data attribute -/// stores whether the plan is a `SortExec` or is connected to a `SortExec` -/// via its children. -type PlanWithCorrespondingSort = PlanContext; - -fn update_sort_ctx_children( - mut node: PlanWithCorrespondingSort, - data: bool, -) -> Result { - for child_node in node.children.iter_mut() { - let plan = &child_node.plan; - child_node.data = if is_sort(plan) { - // Initiate connection: - true - } else if is_limit(plan) { - // There is no sort linkage for this path, it starts at a limit. - false - } else { - let is_spm = is_sort_preserving_merge(plan); - let required_orderings = plan.required_input_ordering(); - let flags = plan.maintains_input_order(); - // Add parent node to the tree if there is at least one child with - // a sort connection: - izip!(flags, required_orderings).any(|(maintains, required_ordering)| { - let propagates_ordering = - (maintains && required_ordering.is_none()) || is_spm; - let connected_to_sort = - child_node.children.iter().any(|child| child.data); - propagates_ordering && connected_to_sort - }) - } - } - - node.data = data; - node.update_plan_from_children() -} - -/// This object is used within the [`EnforceSorting`] rule to track the closest -/// [`CoalescePartitionsExec`] descendant(s) for every child of a plan. The data -/// attribute stores whether the plan is a `CoalescePartitionsExec` or is -/// connected to a `CoalescePartitionsExec` via its children. -type PlanWithCorrespondingCoalescePartitions = PlanContext; - -fn update_coalesce_ctx_children( - coalesce_context: &mut PlanWithCorrespondingCoalescePartitions, -) { - let children = &coalesce_context.children; - coalesce_context.data = if children.is_empty() { - // Plan has no children, it cannot be a `CoalescePartitionsExec`. - false - } else if is_coalesce_partitions(&coalesce_context.plan) { - // Initiate a connection: - true - } else { - children.iter().enumerate().any(|(idx, node)| { - // Only consider operators that don't require a single partition, - // and connected to some `CoalescePartitionsExec`: - node.data - && !matches!( - coalesce_context.plan.required_input_distribution()[idx], - Distribution::SinglePartition - ) - }) - }; -} - -/// The boolean flag `repartition_sorts` defined in the config indicates -/// whether we elect to transform [`CoalescePartitionsExec`] + [`SortExec`] cascades -/// into [`SortExec`] + [`SortPreservingMergeExec`] cascades, which enables us to -/// perform sorting in parallel. -impl PhysicalOptimizerRule for EnforceSorting { - fn optimize( - &self, - plan: Arc, - config: &ConfigOptions, - ) -> Result> { - let plan_requirements = PlanWithCorrespondingSort::new_default(plan); - // Execute a bottom-up traversal to enforce sorting requirements, - // remove unnecessary sorts, and optimize sort-sensitive operators: - let adjusted = plan_requirements.transform_up(ensure_sorting)?.data; - 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()?; - 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()?; - // Execute a top-down traversal to exploit sort push-down opportunities - // missed by the bottom-up traversal: - let mut sort_pushdown = SortPushDown::new_default(updated_plan.plan); - assign_initial_requirements(&mut sort_pushdown); - let adjusted = pushdown_sorts(sort_pushdown)?; - adjusted - .plan - .transform_up(|plan| Ok(Transformed::yes(replace_with_partial_sort(plan)?))) - .data() - } - - fn name(&self) -> &str { - "EnforceSorting" - } - - fn schema_check(&self) -> bool { - true - } -} - -fn replace_with_partial_sort( - plan: Arc, -) -> Result> { - let plan_any = plan.as_any(); - if let Some(sort_plan) = plan_any.downcast_ref::() { - let child = Arc::clone(sort_plan.children()[0]); - if !child.boundedness().is_unbounded() { - return Ok(plan); - } - - // here we're trying to find the common prefix for sorted columns that is required for the - // sort and already satisfied by the given ordering - let child_eq_properties = child.equivalence_properties(); - let sort_req = LexRequirement::from(sort_plan.expr().clone()); - - let mut common_prefix_length = 0; - while child_eq_properties.ordering_satisfy_requirement(&LexRequirement { - inner: sort_req[0..common_prefix_length + 1].to_vec(), - }) { - common_prefix_length += 1; - } - if common_prefix_length > 0 { - return Ok(Arc::new( - PartialSortExec::new( - LexOrdering::new(sort_plan.expr().to_vec()), - Arc::clone(sort_plan.input()), - common_prefix_length, - ) - .with_preserve_partitioning(sort_plan.preserve_partitioning()) - .with_fetch(sort_plan.fetch()), - )); - } - } - Ok(plan) -} - -/// This function turns plans of the form -/// ```text -/// "SortExec: expr=\[a@0 ASC\]", -/// " CoalescePartitionsExec", -/// " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", -/// ``` -/// to -/// ```text -/// "SortPreservingMergeExec: \[a@0 ASC\]", -/// " SortExec: expr=\[a@0 ASC\]", -/// " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", -/// ``` -/// by following connections from [`CoalescePartitionsExec`]s to [`SortExec`]s. -/// By performing sorting in parallel, we can increase performance in some scenarios. -fn parallelize_sorts( - mut requirements: PlanWithCorrespondingCoalescePartitions, -) -> Result> { - update_coalesce_ctx_children(&mut requirements); - - if requirements.children.is_empty() || !requirements.children[0].data { - // We only take an action when the plan is either a `SortExec`, a - // `SortPreservingMergeExec` or a `CoalescePartitionsExec`, and they - // all have a single child. Therefore, if the first child has no - // connection, we can return immediately. - Ok(Transformed::no(requirements)) - } else if (is_sort(&requirements.plan) - || is_sort_preserving_merge(&requirements.plan)) - && requirements.plan.output_partitioning().partition_count() <= 1 - { - // Take the initial sort expressions and requirements - let (sort_exprs, fetch) = get_sort_exprs(&requirements.plan)?; - let sort_reqs = LexRequirement::from(sort_exprs.clone()); - let sort_exprs = sort_exprs.clone(); - - // If there is a connection between a `CoalescePartitionsExec` and a - // global sort that satisfy the requirements (i.e. intermediate - // executors don't require single partition), then we can replace - // the `CoalescePartitionsExec` + `SortExec` cascade with a `SortExec` - // + `SortPreservingMergeExec` cascade to parallelize sorting. - requirements = remove_bottleneck_in_subplan(requirements)?; - // We also need to remove the self node since `remove_corresponding_coalesce_in_sub_plan` - // deals with the children and their children and so on. - requirements = requirements.children.swap_remove(0); - - requirements = add_sort_above_with_check(requirements, sort_reqs, fetch); - - let spm = - SortPreservingMergeExec::new(sort_exprs, Arc::clone(&requirements.plan)); - Ok(Transformed::yes( - PlanWithCorrespondingCoalescePartitions::new( - Arc::new(spm.with_fetch(fetch)), - false, - vec![requirements], - ), - )) - } else if is_coalesce_partitions(&requirements.plan) { - // There is an unnecessary `CoalescePartitionsExec` in the plan. - // This will handle the recursive `CoalescePartitionsExec` plans. - requirements = remove_bottleneck_in_subplan(requirements)?; - // For the removal of self node which is also a `CoalescePartitionsExec`. - requirements = requirements.children.swap_remove(0); - - Ok(Transformed::yes( - PlanWithCorrespondingCoalescePartitions::new( - Arc::new(CoalescePartitionsExec::new(Arc::clone(&requirements.plan))), - false, - vec![requirements], - ), - )) - } else { - Ok(Transformed::yes(requirements)) - } -} - -/// This function enforces sorting requirements and makes optimizations without -/// violating these requirements whenever possible. -fn ensure_sorting( - mut requirements: PlanWithCorrespondingSort, -) -> Result> { - requirements = update_sort_ctx_children(requirements, false)?; - - // Perform naive analysis at the beginning -- remove already-satisfied sorts: - if requirements.children.is_empty() { - return Ok(Transformed::no(requirements)); - } - let maybe_requirements = analyze_immediate_sort_removal(requirements); - requirements = if !maybe_requirements.transformed { - maybe_requirements.data - } else { - return Ok(maybe_requirements); - }; - - let plan = &requirements.plan; - let mut updated_children = vec![]; - for (idx, (required_ordering, mut child)) in plan - .required_input_ordering() - .into_iter() - .zip(requirements.children.into_iter()) - .enumerate() - { - let physical_ordering = child.plan.output_ordering(); - - if let Some(required) = required_ordering { - let eq_properties = child.plan.equivalence_properties(); - if !eq_properties.ordering_satisfy_requirement(&required) { - // Make sure we preserve the ordering requirements: - if physical_ordering.is_some() { - child = update_child_to_remove_unnecessary_sort(idx, child, plan)?; - } - child = add_sort_above(child, required, None); - child = update_sort_ctx_children(child, true)?; - } - } else if physical_ordering.is_none() - || !plan.maintains_input_order()[idx] - || is_union(plan) - { - // We have a `SortExec` whose effect may be neutralized by another - // order-imposing operator, remove this sort: - child = update_child_to_remove_unnecessary_sort(idx, child, plan)?; - } - updated_children.push(child); - } - requirements.children = updated_children; - // For window expressions, we can remove some sorts when we can - // calculate the result in reverse: - let child_node = &requirements.children[0]; - if is_window(plan) && child_node.data { - return adjust_window_sort_removal(requirements).map(Transformed::yes); - } else if is_sort_preserving_merge(plan) - && child_node.plan.output_partitioning().partition_count() <= 1 - { - // This `SortPreservingMergeExec` is unnecessary, input already has a - // single partition. - let child_node = requirements.children.swap_remove(0); - return Ok(Transformed::yes(child_node)); - } - - update_sort_ctx_children(requirements, false).map(Transformed::yes) -} - -/// Analyzes a given [`SortExec`] (`plan`) to determine whether its input -/// already has a finer ordering than it enforces. -fn analyze_immediate_sort_removal( - mut node: PlanWithCorrespondingSort, -) -> Transformed { - if let Some(sort_exec) = node.plan.as_any().downcast_ref::() { - let sort_input = sort_exec.input(); - // If this sort is unnecessary, we should remove it: - if sort_input.equivalence_properties().ordering_satisfy( - sort_exec - .properties() - .output_ordering() - .unwrap_or(LexOrdering::empty()), - ) { - node.plan = if !sort_exec.preserve_partitioning() - && sort_input.output_partitioning().partition_count() > 1 - { - // Replace the sort with a sort-preserving merge: - let expr = LexOrdering::new(sort_exec.expr().to_vec()); - Arc::new( - SortPreservingMergeExec::new(expr, Arc::clone(sort_input)) - .with_fetch(sort_exec.fetch()), - ) as _ - } else { - // Remove the sort: - node.children = node.children.swap_remove(0).children; - if let Some(fetch) = sort_exec.fetch() { - // If the sort has a fetch, we need to add a limit: - if sort_exec - .properties() - .output_partitioning() - .partition_count() - == 1 - { - Arc::new(GlobalLimitExec::new( - Arc::clone(sort_input), - 0, - Some(fetch), - )) - } else { - Arc::new(LocalLimitExec::new(Arc::clone(sort_input), fetch)) - } - } else { - Arc::clone(sort_input) - } - }; - for child in node.children.iter_mut() { - child.data = false; - } - node.data = false; - return Transformed::yes(node); - } - } - Transformed::no(node) -} - -/// Adjusts a [`WindowAggExec`] or a [`BoundedWindowAggExec`] to determine -/// whether it may allow removing a sort. -fn adjust_window_sort_removal( - mut window_tree: PlanWithCorrespondingSort, -) -> Result { - // Window operators have a single child we need to adjust: - let child_node = remove_corresponding_sort_from_sub_plan( - window_tree.children.swap_remove(0), - matches!( - window_tree.plan.required_input_distribution()[0], - Distribution::SinglePartition - ), - )?; - window_tree.children.push(child_node); - - let plan = window_tree.plan.as_any(); - let child_plan = &window_tree.children[0].plan; - let (window_expr, new_window) = - if let Some(exec) = plan.downcast_ref::() { - let window_expr = exec.window_expr(); - let new_window = - get_best_fitting_window(window_expr, child_plan, &exec.partition_keys)?; - (window_expr, new_window) - } else if let Some(exec) = plan.downcast_ref::() { - let window_expr = exec.window_expr(); - let new_window = - get_best_fitting_window(window_expr, child_plan, &exec.partition_keys)?; - (window_expr, new_window) - } else { - return plan_err!("Expected WindowAggExec or BoundedWindowAggExec"); - }; - - window_tree.plan = if let Some(new_window) = new_window { - // We were able to change the window to accommodate the input, use it: - new_window - } else { - // We were unable to change the window to accommodate the input, so we - // will insert a sort. - let reqs = window_tree - .plan - .required_input_ordering() - .swap_remove(0) - .unwrap_or_default(); - - // Satisfy the ordering requirement so that the window can run: - let mut child_node = window_tree.children.swap_remove(0); - child_node = add_sort_above(child_node, reqs, None); - let child_plan = Arc::clone(&child_node.plan); - window_tree.children.push(child_node); - - if window_expr.iter().all(|e| e.uses_bounded_memory()) { - Arc::new(BoundedWindowAggExec::try_new( - window_expr.to_vec(), - child_plan, - window_expr[0].partition_by().to_vec(), - InputOrderMode::Sorted, - )?) as _ - } else { - Arc::new(WindowAggExec::try_new( - window_expr.to_vec(), - child_plan, - window_expr[0].partition_by().to_vec(), - )?) as _ - } - }; - - window_tree.data = false; - Ok(window_tree) -} - -/// Removes parallelization-reducing, avoidable [`CoalescePartitionsExec`]s from -/// the plan in `node`. After the removal of such `CoalescePartitionsExec`s from -/// the plan, some of the remaining `RepartitionExec`s might become unnecessary. -/// Removes such `RepartitionExec`s from the plan as well. -fn remove_bottleneck_in_subplan( - mut requirements: PlanWithCorrespondingCoalescePartitions, -) -> Result { - let plan = &requirements.plan; - let children = &mut requirements.children; - if is_coalesce_partitions(&children[0].plan) { - // We can safely use the 0th index since we have a `CoalescePartitionsExec`. - let mut new_child_node = children[0].children.swap_remove(0); - while new_child_node.plan.output_partitioning() == plan.output_partitioning() - && is_repartition(&new_child_node.plan) - && is_repartition(plan) - { - new_child_node = new_child_node.children.swap_remove(0) - } - children[0] = new_child_node; - } else { - requirements.children = requirements - .children - .into_iter() - .map(|node| { - if node.data { - remove_bottleneck_in_subplan(node) - } else { - Ok(node) - } - }) - .collect::>()?; - } - let mut new_reqs = requirements.update_plan_from_children()?; - if let Some(repartition) = new_reqs.plan.as_any().downcast_ref::() { - let input_partitioning = repartition.input().output_partitioning(); - // We can remove this repartitioning operator if it is now a no-op: - let mut can_remove = input_partitioning.eq(repartition.partitioning()); - // We can also remove it if we ended up with an ineffective RR: - if let Partitioning::RoundRobinBatch(n_out) = repartition.partitioning() { - can_remove |= *n_out == input_partitioning.partition_count(); - } - if can_remove { - new_reqs = new_reqs.children.swap_remove(0) - } - } - Ok(new_reqs) -} - -/// Updates child to remove the unnecessary sort below it. -fn update_child_to_remove_unnecessary_sort( - child_idx: usize, - mut node: PlanWithCorrespondingSort, - parent: &Arc, -) -> Result { - if node.data { - let requires_single_partition = matches!( - parent.required_input_distribution()[child_idx], - Distribution::SinglePartition - ); - node = remove_corresponding_sort_from_sub_plan(node, requires_single_partition)?; - } - node.data = false; - Ok(node) -} - -/// Removes the sort from the plan in `node`. -fn remove_corresponding_sort_from_sub_plan( - mut node: PlanWithCorrespondingSort, - requires_single_partition: bool, -) -> Result { - // A `SortExec` is always at the bottom of the tree. - if let Some(sort_exec) = node.plan.as_any().downcast_ref::() { - // Do not remove sorts with fetch: - if sort_exec.fetch().is_none() { - node = node.children.swap_remove(0); - } - } else { - let mut any_connection = false; - let required_dist = node.plan.required_input_distribution(); - node.children = node - .children - .into_iter() - .enumerate() - .map(|(idx, child)| { - if child.data { - any_connection = true; - remove_corresponding_sort_from_sub_plan( - child, - matches!(required_dist[idx], Distribution::SinglePartition), - ) - } else { - Ok(child) - } - }) - .collect::>()?; - if any_connection || node.children.is_empty() { - node = update_sort_ctx_children(node, false)?; - } - - // Replace with variants that do not preserve order. - if is_sort_preserving_merge(&node.plan) { - node.children = node.children.swap_remove(0).children; - node.plan = Arc::clone(node.plan.children().swap_remove(0)); - } else if let Some(repartition) = - node.plan.as_any().downcast_ref::() - { - node.plan = Arc::new(RepartitionExec::try_new( - Arc::clone(&node.children[0].plan), - repartition.properties().output_partitioning().clone(), - )?) as _; - } - }; - // Deleting a merging sort may invalidate distribution requirements. - // Ensure that we stay compliant with such requirements: - if requires_single_partition && node.plan.output_partitioning().partition_count() > 1 - { - // If there is existing ordering, to preserve ordering use - // `SortPreservingMergeExec` instead of a `CoalescePartitionsExec`. - let plan = Arc::clone(&node.plan); - let fetch = plan.fetch(); - let plan = if let Some(ordering) = plan.output_ordering() { - Arc::new( - SortPreservingMergeExec::new(LexOrdering::new(ordering.to_vec()), plan) - .with_fetch(fetch), - ) as _ - } else { - Arc::new(CoalescePartitionsExec::new(plan)) as _ - }; - node = PlanWithCorrespondingSort::new(plan, false, vec![node]); - node = update_sort_ctx_children(node, false)?; - } - Ok(node) -} - -/// Converts an [ExecutionPlan] trait object to a [LexOrdering] reference when possible. -fn get_sort_exprs( - sort_any: &Arc, -) -> Result<(&LexOrdering, Option)> { - if let Some(sort_exec) = sort_any.as_any().downcast_ref::() { - Ok((sort_exec.expr(), sort_exec.fetch())) - } else if let Some(spm) = sort_any.as_any().downcast_ref::() - { - Ok((spm.expr(), spm.fetch())) - } else { - plan_err!("Given ExecutionPlan is not a SortExec or a SortPreservingMergeExec") - } -} - -#[cfg(test)] -mod tests { - use super::*; - use crate::physical_optimizer::enforce_distribution::EnforceDistribution; - use crate::physical_optimizer::test_utils::{parquet_exec, parquet_exec_sorted}; - use crate::physical_plan::{displayable, get_plan_string, Partitioning}; - use crate::prelude::{SessionConfig, SessionContext}; - use crate::test::{csv_exec_ordered, csv_exec_sorted, stream_exec_ordered}; - 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 arrow::compute::SortOptions; - use arrow::datatypes::{DataType, Field, 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 rstest::rstest; - - 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) - } - - 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) - 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) - } - - /// 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 config = SessionConfig::new().with_repartition_sorts($REPARTITION_SORTS); - let session_ctx = SessionContext::new_with_config(config); - let state = session_ctx.state(); - - // 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 state.config_options().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, - state.config_options(), - ) - }) - .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, state.config_options())?; - - // 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_sort5() -> Result<()> { - let left_schema = create_test_schema2()?; - let right_schema = create_test_schema3()?; - let left_input = memory_exec(&left_schema); - let parquet_sort_exprs = vec![sort_expr("a", &right_schema)]; - let right_input = parquet_exec_sorted(&right_schema, parquet_sort_exprs); - - let on = vec![( - Arc::new(Column::new_with_schema("col_a", &left_schema)?) as _, - Arc::new(Column::new_with_schema("c", &right_schema)?) as _, - )]; - let join = hash_join_exec(left_input, right_input, on, None, &JoinType::Inner)?; - 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]"]; - - 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]", - ]; - assert_optimized!(expected_input, expected_optimized, physical_plan, true); - - Ok(()) - } - - #[tokio::test] - async fn test_do_not_remove_sort_with_limit() -> Result<()> { - let schema = create_test_schema()?; - - let source1 = parquet_exec(&schema); - let sort_exprs = vec![ - sort_expr("nullable_col", &schema), - sort_expr("non_nullable_col", &schema), - ]; - let sort = sort_exec(sort_exprs.clone(), source1); - let limit = limit_exec(sort); - - let parquet_sort_exprs = vec![sort_expr("nullable_col", &schema)]; - let source2 = parquet_exec_sorted(&schema, parquet_sort_exprs); - - let union = union_exec(vec![source2, limit]); - let repartition = repartition_exec(union); - 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]"]; - - // 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]", - ]; - assert_optimized!(expected_input, expected_optimized, physical_plan, true); - - Ok(()) - } - - #[tokio::test] - async fn test_union_inputs_sorted() -> Result<()> { - let schema = create_test_schema()?; - - let source1 = parquet_exec(&schema); - let sort_exprs = vec![sort_expr("nullable_col", &schema)]; - let sort = sort_exec(sort_exprs.clone(), source1); - - let source2 = parquet_exec_sorted(&schema, sort_exprs.clone()); - - let union = union_exec(vec![source2, sort]); - let physical_plan = sort_preserving_merge_exec(sort_exprs, union); - - // 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]", - ]; - // 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); - - Ok(()) - } - - #[tokio::test] - async fn test_union_inputs_different_sorted() -> Result<()> { - let schema = create_test_schema()?; - - let source1 = parquet_exec(&schema); - let sort_exprs = vec![sort_expr("nullable_col", &schema)]; - let sort = sort_exec(sort_exprs.clone(), source1); - - let parquet_sort_exprs = vec![ - sort_expr("nullable_col", &schema), - sort_expr("non_nullable_col", &schema), - ]; - let source2 = parquet_exec_sorted(&schema, parquet_sort_exprs); - - let union = union_exec(vec![source2, sort]); - let physical_plan = sort_preserving_merge_exec(sort_exprs, union); - - // 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]", - ]; - // 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); - - Ok(()) - } - - #[tokio::test] - async fn test_union_inputs_different_sorted2() -> Result<()> { - let schema = create_test_schema()?; - - let source1 = parquet_exec(&schema); - let sort_exprs = vec![ - sort_expr("nullable_col", &schema), - sort_expr("non_nullable_col", &schema), - ]; - let sort = sort_exec(sort_exprs.clone(), source1); - - let parquet_sort_exprs = vec![sort_expr("nullable_col", &schema)]; - let source2 = parquet_exec_sorted(&schema, parquet_sort_exprs); - - let union = union_exec(vec![source2, sort]); - let physical_plan = sort_preserving_merge_exec(sort_exprs, union); - - // Input is an invalid plan. In this case rule should add required sorting in appropriate places. - // First ParquetExec has output ordering(nullable_col@0 ASC). However, it doesn't satisfy the - // 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]"]; - - 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]"]; - assert_optimized!(expected_input, expected_optimized, physical_plan, true); - - Ok(()) - } - - #[tokio::test] - async fn test_union_inputs_different_sorted3() -> Result<()> { - let schema = create_test_schema()?; - - let source1 = parquet_exec(&schema); - let sort_exprs1 = vec![ - sort_expr("nullable_col", &schema), - sort_expr("non_nullable_col", &schema), - ]; - let sort1 = sort_exec(sort_exprs1, source1.clone()); - let sort_exprs2 = vec![sort_expr("nullable_col", &schema)]; - let sort2 = sort_exec(sort_exprs2, source1); - - let parquet_sort_exprs = vec![sort_expr("nullable_col", &schema)]; - let source2 = parquet_exec_sorted(&schema, parquet_sort_exprs.clone()); - - let union = union_exec(vec![sort1, source2, sort2]); - let physical_plan = sort_preserving_merge_exec(parquet_sort_exprs, union); - - // First input to the union is not Sorted (SortExec is finer than required ordering by the SortPreservingMergeExec above). - // 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]"]; - // 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]"]; - assert_optimized!(expected_input, expected_optimized, physical_plan, true); - - Ok(()) - } - - #[tokio::test] - async fn test_union_inputs_different_sorted4() -> Result<()> { - let schema = create_test_schema()?; - - let source1 = parquet_exec(&schema); - let sort_exprs1 = vec![ - sort_expr("nullable_col", &schema), - sort_expr("non_nullable_col", &schema), - ]; - let sort_exprs2 = vec![sort_expr("nullable_col", &schema)]; - let sort1 = sort_exec(sort_exprs2.clone(), source1.clone()); - let sort2 = sort_exec(sort_exprs2.clone(), source1); - - let source2 = parquet_exec_sorted(&schema, sort_exprs2); - - let union = union_exec(vec![sort1, source2, sort2]); - let physical_plan = sort_preserving_merge_exec(sort_exprs1, union); - - // Ordering requirement of the `SortPreservingMergeExec` is not met. - // Should modify the plan to ensure that all three inputs to the - // `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]"]; - 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]"]; - assert_optimized!(expected_input, expected_optimized, physical_plan, true); - - Ok(()) - } - - #[tokio::test] - async fn test_union_inputs_different_sorted5() -> Result<()> { - let schema = create_test_schema()?; - - let source1 = parquet_exec(&schema); - let sort_exprs1 = vec![ - sort_expr("nullable_col", &schema), - sort_expr("non_nullable_col", &schema), - ]; - let sort_exprs2 = vec![ - sort_expr("nullable_col", &schema), - sort_expr_options( - "non_nullable_col", - &schema, - SortOptions { - descending: true, - nulls_first: false, - }, - ), - ]; - let sort_exprs3 = vec![sort_expr("nullable_col", &schema)]; - let sort1 = sort_exec(sort_exprs1, source1.clone()); - let sort2 = sort_exec(sort_exprs2, source1); - - let union = union_exec(vec![sort1, sort2]); - let physical_plan = sort_preserving_merge_exec(sort_exprs3, union); - - // The `UnionExec` doesn't preserve any of the inputs ordering in the - // 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]"]; - 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]"]; - assert_optimized!(expected_input, expected_optimized, physical_plan, true); - - Ok(()) - } - - #[tokio::test] - async fn test_union_inputs_different_sorted6() -> Result<()> { - let schema = create_test_schema()?; - - let source1 = parquet_exec(&schema); - let sort_exprs1 = vec![sort_expr("nullable_col", &schema)]; - let sort1 = sort_exec(sort_exprs1, source1.clone()); - let sort_exprs2 = vec![ - sort_expr("nullable_col", &schema), - sort_expr("non_nullable_col", &schema), - ]; - let repartition = repartition_exec(source1); - let spm = sort_preserving_merge_exec(sort_exprs2, repartition); - - let parquet_sort_exprs = vec![sort_expr("nullable_col", &schema)]; - let source2 = parquet_exec_sorted(&schema, parquet_sort_exprs.clone()); - - let union = union_exec(vec![sort1, source2, spm]); - let physical_plan = sort_preserving_merge_exec(parquet_sort_exprs, union); - - // The plan is not valid as it is -- the input ordering requirement - // of the `SortPreservingMergeExec` under the third child of the - // `UnionExec` is not met. We should add a `SortExec` below it. - // At the same time, this ordering requirement is unnecessarily fine. - // 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]"]; - // 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]"]; - assert_optimized!(expected_input, expected_optimized, physical_plan, true); - - Ok(()) - } - - #[tokio::test] - async fn test_union_inputs_different_sorted7() -> Result<()> { - let schema = create_test_schema()?; - - let source1 = parquet_exec(&schema); - let sort_exprs1 = vec![ - sort_expr("nullable_col", &schema), - sort_expr("non_nullable_col", &schema), - ]; - let sort_exprs3 = vec![sort_expr("nullable_col", &schema)]; - let sort1 = sort_exec(sort_exprs1.clone(), source1.clone()); - let sort2 = sort_exec(sort_exprs1, source1); - - let union = union_exec(vec![sort1, sort2]); - let physical_plan = sort_preserving_merge_exec(sort_exprs3, union); - - // 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]"]; - // 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]"]; - assert_optimized!(expected_input, expected_output, physical_plan, true); - - Ok(()) - } - - #[tokio::test] - async fn test_union_inputs_different_sorted8() -> Result<()> { - let schema = create_test_schema()?; - - let source1 = parquet_exec(&schema); - let sort_exprs1 = vec![ - sort_expr("nullable_col", &schema), - sort_expr("non_nullable_col", &schema), - ]; - let sort_exprs2 = vec![ - sort_expr_options( - "nullable_col", - &schema, - SortOptions { - descending: true, - nulls_first: false, - }, - ), - sort_expr_options( - "non_nullable_col", - &schema, - SortOptions { - descending: true, - nulls_first: false, - }, - ), - ]; - let sort1 = sort_exec(sort_exprs1, source1.clone()); - let sort2 = sort_exec(sort_exprs2, source1); - - let physical_plan = union_exec(vec![sort1, sort2]); - - // 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]"]; - // 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]"]; - assert_optimized!(expected_input, expected_optimized, physical_plan, true); - - Ok(()) - } - - #[tokio::test] - async fn test_window_multi_path_sort() -> Result<()> { - let schema = create_test_schema()?; - - let sort_exprs1 = vec![ - sort_expr("nullable_col", &schema), - sort_expr("non_nullable_col", &schema), - ]; - let sort_exprs2 = vec![sort_expr("nullable_col", &schema)]; - // reverse sorting of sort_exprs2 - let sort_exprs3 = vec![sort_expr_options( - "nullable_col", - &schema, - SortOptions { - descending: true, - nulls_first: false, - }, - )]; - let source1 = parquet_exec_sorted(&schema, sort_exprs1); - let source2 = parquet_exec_sorted(&schema, sort_exprs2); - let sort1 = sort_exec(sort_exprs3.clone(), source1); - let sort2 = sort_exec(sort_exprs3.clone(), source2); - - let union = union_exec(vec![sort1, sort2]); - let spm = sort_preserving_merge_exec(sort_exprs3.clone(), union); - let physical_plan = bounded_window_exec("nullable_col", sort_exprs3, spm); - - // The `WindowAggExec` gets its sorting from multiple children jointly. - // During the removal of `SortExec`s, it should be able to remove the - // 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]"]; - 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]"]; - assert_optimized!(expected_input, expected_optimized, physical_plan, true); - - Ok(()) - } - - #[tokio::test] - async fn test_window_multi_path_sort2() -> Result<()> { - let schema = create_test_schema()?; - - let sort_exprs1 = LexOrdering::new(vec![ - sort_expr("nullable_col", &schema), - sort_expr("non_nullable_col", &schema), - ]); - let sort_exprs2 = vec![sort_expr("nullable_col", &schema)]; - let source1 = parquet_exec_sorted(&schema, sort_exprs2.clone()); - let source2 = parquet_exec_sorted(&schema, sort_exprs2.clone()); - let sort1 = sort_exec(sort_exprs1.clone(), source1); - let sort2 = sort_exec(sort_exprs1.clone(), source2); - - let union = union_exec(vec![sort1, sort2]); - let spm = Arc::new(SortPreservingMergeExec::new(sort_exprs1, union)) as _; - let physical_plan = bounded_window_exec("nullable_col", sort_exprs2, spm); - - // 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]"]; - 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]"]; - assert_optimized!(expected_input, expected_optimized, physical_plan, true); - - Ok(()) - } - - #[tokio::test] - async fn test_union_inputs_different_sorted_with_limit() -> Result<()> { - let schema = create_test_schema()?; - - let source1 = parquet_exec(&schema); - let sort_exprs1 = vec![ - sort_expr("nullable_col", &schema), - sort_expr("non_nullable_col", &schema), - ]; - let sort_exprs2 = vec![ - sort_expr("nullable_col", &schema), - sort_expr_options( - "non_nullable_col", - &schema, - SortOptions { - descending: true, - nulls_first: false, - }, - ), - ]; - let sort_exprs3 = vec![sort_expr("nullable_col", &schema)]; - let sort1 = sort_exec(sort_exprs1, source1.clone()); - - let sort2 = sort_exec(sort_exprs2, source1); - let limit = local_limit_exec(sort2); - let limit = global_limit_exec(limit); - - let union = union_exec(vec![sort1, limit]); - let physical_plan = sort_preserving_merge_exec(sort_exprs3, union); - - // 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]"]; - 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]"]; - assert_optimized!(expected_input, expected_optimized, physical_plan, true); - - Ok(()) - } - - #[tokio::test] - async fn test_sort_merge_join_order_by_left() -> Result<()> { - let left_schema = create_test_schema()?; - let right_schema = create_test_schema2()?; - - let left = parquet_exec(&left_schema); - let right = parquet_exec(&right_schema); - - // Join on (nullable_col == col_a) - let join_on = vec![( - Arc::new(Column::new_with_schema("nullable_col", &left.schema()).unwrap()) - as _, - Arc::new(Column::new_with_schema("col_a", &right.schema()).unwrap()) as _, - )]; - - let join_types = vec![ - JoinType::Inner, - JoinType::Left, - JoinType::Right, - JoinType::Full, - JoinType::LeftSemi, - JoinType::LeftAnti, - ]; - for join_type in join_types { - let join = - sort_merge_join_exec(left.clone(), right.clone(), &join_on, &join_type); - let sort_exprs = vec![ - sort_expr("nullable_col", &join.schema()), - sort_expr("non_nullable_col", &join.schema()), - ]; - let physical_plan = sort_preserving_merge_exec(sort_exprs.clone(), join); - - let join_plan = format!( - "SortMergeJoin: join_type={join_type}, on=[(nullable_col@0, col_a@0)]" - ); - let join_plan2 = format!( - " 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]"]; - let expected_optimized = match join_type { - JoinType::Inner - | JoinType::Left - | JoinType::LeftSemi - | 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]", - ] - } - _ => { - // 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]", - ] - } - }; - assert_optimized!(expected_input, expected_optimized, physical_plan, true); - } - Ok(()) - } - - #[tokio::test] - async fn test_sort_merge_join_order_by_right() -> Result<()> { - let left_schema = create_test_schema()?; - let right_schema = create_test_schema2()?; - - let left = parquet_exec(&left_schema); - let right = parquet_exec(&right_schema); - - // Join on (nullable_col == col_a) - let join_on = vec![( - Arc::new(Column::new_with_schema("nullable_col", &left.schema()).unwrap()) - as _, - Arc::new(Column::new_with_schema("col_a", &right.schema()).unwrap()) as _, - )]; - - let join_types = vec![ - JoinType::Inner, - JoinType::Left, - JoinType::Right, - JoinType::Full, - JoinType::RightAnti, - ]; - for join_type in join_types { - let join = - sort_merge_join_exec(left.clone(), right.clone(), &join_on, &join_type); - let sort_exprs = vec![ - sort_expr("col_a", &join.schema()), - sort_expr("col_b", &join.schema()), - ]; - let physical_plan = sort_preserving_merge_exec(sort_exprs, join); - - let join_plan = format!( - "SortMergeJoin: join_type={join_type}, on=[(nullable_col@0, col_a@0)]" - ); - let spm_plan = match join_type { - JoinType::RightAnti => { - "SortPreservingMergeExec: [col_a@0 ASC, col_b@1 ASC]" - } - _ => "SortPreservingMergeExec: [col_a@2 ASC, col_b@3 ASC]", - }; - let join_plan2 = format!( - " 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]"]; - 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]", - ] - } - _ => { - // 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]", - ] - } - }; - assert_optimized!(expected_input, expected_optimized, physical_plan, true); - } - Ok(()) - } - - #[tokio::test] - async fn test_sort_merge_join_complex_order_by() -> Result<()> { - let left_schema = create_test_schema()?; - let right_schema = create_test_schema2()?; - - let left = parquet_exec(&left_schema); - let right = parquet_exec(&right_schema); - - // Join on (nullable_col == col_a) - let join_on = vec![( - Arc::new(Column::new_with_schema("nullable_col", &left.schema()).unwrap()) - as _, - Arc::new(Column::new_with_schema("col_a", &right.schema()).unwrap()) as _, - )]; - - let join = sort_merge_join_exec(left, right, &join_on, &JoinType::Inner); - - // order by (col_b, col_a) - let sort_exprs1 = vec![ - sort_expr("col_b", &join.schema()), - sort_expr("col_a", &join.schema()), - ]; - 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]"]; - - // 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]"]; - assert_optimized!(expected_input, expected_optimized, physical_plan, true); - - // order by (nullable_col, col_b, col_a) - let sort_exprs2 = vec![ - sort_expr("nullable_col", &join.schema()), - sort_expr("col_b", &join.schema()), - sort_expr("col_a", &join.schema()), - ]; - 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]"]; - - // 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]"]; - assert_optimized!(expected_input, expected_optimized, physical_plan, true); - - Ok(()) - } - - #[tokio::test] - async fn test_multilayer_coalesce_partitions() -> Result<()> { - let schema = create_test_schema()?; - - let source1 = parquet_exec(&schema); - let repartition = repartition_exec(source1); - let coalesce = Arc::new(CoalescePartitionsExec::new(repartition)) as _; - // 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(), - )), - coalesce, - ); - let sort_exprs = vec![sort_expr("nullable_col", &schema)]; - let physical_plan = sort_exec(sort_exprs, filter); - - // CoalescePartitionsExec and SortExec are not directly consecutive. In this case - // 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]", - " 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_with_lost_ordering_bounded() -> Result<()> { - let schema = create_test_schema3()?; - let sort_exprs = vec![sort_expr("a", &schema)]; - let source = csv_exec_sorted(&schema, sort_exprs); - let repartition_rr = repartition_exec(source); - let repartition_hash = Arc::new(RepartitionExec::try_new( - repartition_rr, - Partitioning::Hash(vec![col("c", &schema).unwrap()], 10), - )?) as _; - let coalesce_partitions = coalesce_partitions_exec(repartition_hash); - 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"]; - 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"]; - assert_optimized!(expected_input, expected_optimized, physical_plan, true); - - Ok(()) - } - - #[rstest] - #[tokio::test] - async fn test_with_lost_ordering_unbounded_bounded( - #[values(false, true)] source_unbounded: bool, - ) -> Result<()> { - let schema = create_test_schema3()?; - let sort_exprs = vec![sort_expr("a", &schema)]; - // create either bounded or unbounded source - let source = if source_unbounded { - stream_exec_ordered(&schema, sort_exprs) - } else { - csv_exec_ordered(&schema, sort_exprs) - }; - let repartition_rr = repartition_exec(source); - let repartition_hash = Arc::new(RepartitionExec::try_new( - repartition_rr, - Partitioning::Hash(vec![col("c", &schema).unwrap()], 10), - )?) as _; - let coalesce_partitions = coalesce_partitions_exec(repartition_hash); - let physical_plan = sort_exec(vec![sort_expr("a", &schema)], coalesce_partitions); - - // 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]", - ]; - 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", - ]; - - // 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]", - ]; - - // 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", - ]; - 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", - ]; - let (expected_input, expected_optimized, expected_optimized_sort_parallelize) = - if source_unbounded { - ( - expected_input_unbounded, - expected_optimized_unbounded.clone(), - expected_optimized_unbounded, - ) - } else { - ( - expected_input_bounded, - expected_optimized_bounded, - expected_optimized_bounded_parallelize_sort, - ) - }; - assert_optimized!( - expected_input, - expected_optimized, - physical_plan.clone(), - false - ); - assert_optimized!( - expected_input, - expected_optimized_sort_parallelize, - physical_plan, - true - ); - - Ok(()) - } - - #[tokio::test] - async fn test_do_not_pushdown_through_spm() -> Result<()> { - let schema = create_test_schema3()?; - let sort_exprs = vec![sort_expr("a", &schema), sort_expr("b", &schema)]; - let source = csv_exec_sorted(&schema, sort_exprs.clone()); - let repartition_rr = repartition_exec(source); - let spm = sort_preserving_merge_exec(sort_exprs, repartition_rr); - 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",]; - 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",]; - assert_optimized!(expected_input, expected_optimized, physical_plan, false); - - Ok(()) - } - - #[tokio::test] - async fn test_pushdown_through_spm() -> Result<()> { - let schema = create_test_schema3()?; - let sort_exprs = vec![sort_expr("a", &schema), sort_expr("b", &schema)]; - let source = csv_exec_sorted(&schema, sort_exprs.clone()); - let repartition_rr = repartition_exec(source); - let spm = sort_preserving_merge_exec(sort_exprs, repartition_rr); - let physical_plan = sort_exec( - vec![ - sort_expr("a", &schema), - sort_expr("b", &schema), - sort_expr("c", &schema), - ], - spm, - ); - - 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",]; - 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",]; - assert_optimized!(expected_input, expected_optimized, physical_plan, false); - - Ok(()) - } - - #[tokio::test] - async fn test_window_multi_layer_requirement() -> Result<()> { - let schema = create_test_schema3()?; - let sort_exprs = vec![sort_expr("a", &schema), sort_expr("b", &schema)]; - let source = csv_exec_sorted(&schema, vec![]); - let sort = sort_exec(sort_exprs.clone(), source); - let repartition = repartition_exec(sort); - let repartition = spr_repartition_exec(repartition); - let spm = sort_preserving_merge_exec(sort_exprs.clone(), repartition); - - 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", - ]; - 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", - ]; - 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()?; - let input_sort_exprs = vec![sort_expr("b", &schema), sort_expr("c", &schema)]; - let parquet_input = parquet_exec_sorted(&schema, input_sort_exprs); - - let physical_plan = sort_exec( - vec![ - sort_expr("a", &schema), - sort_expr("b", &schema), - sort_expr("c", &schema), - ], - 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]", - ]; - // 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(()) - } -} diff --git a/datafusion/core/src/physical_optimizer/mod.rs b/datafusion/core/src/physical_optimizer/mod.rs index 63fe115e602c..4ceefe60676e 100644 --- a/datafusion/core/src/physical_optimizer/mod.rs +++ b/datafusion/core/src/physical_optimizer/mod.rs @@ -23,14 +23,7 @@ //! [`ExecutionPlan`]: crate::physical_plan::ExecutionPlan pub mod enforce_distribution; -pub mod enforce_sorting; pub mod optimizer; pub mod projection_pushdown; -pub mod replace_with_order_preserving_variants; -#[cfg(test)] -pub mod test_utils; - -mod sort_pushdown; -mod utils; pub use datafusion_physical_optimizer::*; diff --git a/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs b/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs deleted file mode 100644 index fa6d7f62166a..000000000000 --- a/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs +++ /dev/null @@ -1,1580 +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. - -//! Optimizer rule that replaces executors that lose ordering with their -//! order-preserving variants when it is helpful; either in terms of -//! performance or to accommodate unbounded streams by fixing the pipeline. - -use std::sync::Arc; - -use super::utils::{is_repartition, is_sort_preserving_merge}; -use crate::error::Result; -use crate::physical_optimizer::utils::{is_coalesce_partitions, is_sort}; -use crate::physical_plan::repartition::RepartitionExec; -use crate::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; - -use datafusion_common::config::ConfigOptions; -use datafusion_common::tree_node::Transformed; -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::tree_node::PlanContext; -use datafusion_physical_plan::ExecutionPlanProperties; - -use itertools::izip; - -/// For a given `plan`, this object carries the information one needs from its -/// descendants to decide whether it is beneficial to replace order-losing (but -/// somewhat faster) variants of certain operators with their order-preserving -/// (but somewhat slower) cousins. -pub type OrderPreservationContext = PlanContext; - -/// Updates order-preservation data for all children of the given node. -pub fn update_children(opc: &mut OrderPreservationContext) { - for PlanContext { - plan, - children, - data, - } in opc.children.iter_mut() - { - let maintains_input_order = plan.maintains_input_order(); - let inspect_child = |idx| { - maintains_input_order[idx] - || is_coalesce_partitions(plan) - || is_repartition(plan) - }; - - // We cut the path towards nodes that do not maintain ordering. - for (idx, c) in children.iter_mut().enumerate() { - c.data &= inspect_child(idx); - } - - let plan_children = plan.children(); - *data = if plan_children.is_empty() { - false - } else if !children[0].data - && ((is_repartition(plan) && !maintains_input_order[0]) - || (is_coalesce_partitions(plan) - && plan_children[0].output_ordering().is_some())) - { - // We either have a RepartitionExec or a CoalescePartitionsExec - // and they lose their input ordering, so initiate connection: - true - } else { - // Maintain connection if there is a child with a connection, - // and operator can possibly maintain that connection (either - // in its current form or when we replace it with the corresponding - // order preserving operator). - children - .iter() - .enumerate() - .any(|(idx, c)| c.data && inspect_child(idx)) - } - } - opc.data = false; -} - -/// Calculates the updated plan by replacing operators that lose ordering -/// inside `sort_input` with their order-preserving variants. This will -/// generate an alternative plan, which will be accepted or rejected later on -/// depending on whether it helps us remove a `SortExec`. -fn plan_with_order_preserving_variants( - mut sort_input: OrderPreservationContext, - // Flag indicating that it is desirable to replace `RepartitionExec`s with - // `SortPreservingRepartitionExec`s: - is_spr_better: bool, - // Flag indicating that it is desirable to replace `CoalescePartitionsExec`s - // with `SortPreservingMergeExec`s: - is_spm_better: bool, - fetch: Option, -) -> Result { - sort_input.children = sort_input - .children - .into_iter() - .map(|node| { - // Update descendants in the given tree if there is a connection: - if node.data { - plan_with_order_preserving_variants( - node, - is_spr_better, - is_spm_better, - fetch, - ) - } else { - Ok(node) - } - }) - .collect::>()?; - sort_input.data = false; - - if is_repartition(&sort_input.plan) - && !sort_input.plan.maintains_input_order()[0] - && is_spr_better - { - // When a `RepartitionExec` doesn't preserve ordering, replace it with - // a sort-preserving variant if appropriate: - let child = Arc::clone(&sort_input.children[0].plan); - let partitioning = sort_input.plan.output_partitioning().clone(); - sort_input.plan = Arc::new( - RepartitionExec::try_new(child, partitioning)?.with_preserve_order(), - ) as _; - sort_input.children[0].data = true; - return Ok(sort_input); - } else if is_coalesce_partitions(&sort_input.plan) && is_spm_better { - let child = &sort_input.children[0].plan; - if let Some(ordering) = child.output_ordering() { - // When the input of a `CoalescePartitionsExec` has an ordering, - // replace it with a `SortPreservingMergeExec` if appropriate: - let spm = SortPreservingMergeExec::new(ordering.clone(), Arc::clone(child)) - .with_fetch(fetch); - sort_input.plan = Arc::new(spm) as _; - sort_input.children[0].data = true; - return Ok(sort_input); - } - } - - sort_input.update_plan_from_children() -} - -/// Calculates the updated plan by replacing operators that preserve ordering -/// inside `sort_input` with their order-breaking variants. This will restore -/// the original plan modified by [`plan_with_order_preserving_variants`]. -fn plan_with_order_breaking_variants( - mut sort_input: OrderPreservationContext, -) -> Result { - let plan = &sort_input.plan; - sort_input.children = izip!( - sort_input.children, - plan.maintains_input_order(), - plan.required_input_ordering() - ) - .map(|(node, maintains, required_ordering)| { - // Replace with non-order preserving variants as long as ordering is - // not required by intermediate operators: - if maintains - && (is_sort_preserving_merge(plan) - || !required_ordering.is_some_and(|required_ordering| { - node.plan - .equivalence_properties() - .ordering_satisfy_requirement(&required_ordering) - })) - { - plan_with_order_breaking_variants(node) - } else { - Ok(node) - } - }) - .collect::>()?; - sort_input.data = false; - - if is_repartition(plan) && plan.maintains_input_order()[0] { - // When a `RepartitionExec` preserves ordering, replace it with a - // non-sort-preserving variant: - let child = Arc::clone(&sort_input.children[0].plan); - let partitioning = plan.output_partitioning().clone(); - sort_input.plan = Arc::new(RepartitionExec::try_new(child, partitioning)?) as _; - } else if is_sort_preserving_merge(plan) { - // Replace `SortPreservingMergeExec` with a `CoalescePartitionsExec`: - let child = Arc::clone(&sort_input.children[0].plan); - let coalesce = CoalescePartitionsExec::new(child); - sort_input.plan = Arc::new(coalesce) as _; - } else { - return sort_input.update_plan_from_children(); - } - - sort_input.children[0].data = false; - Ok(sort_input) -} - -/// The `replace_with_order_preserving_variants` optimizer sub-rule tries to -/// remove `SortExec`s from the physical plan by replacing operators that do -/// not preserve ordering with their order-preserving variants; i.e. by replacing -/// ordinary `RepartitionExec`s with their sort-preserving variants or by replacing -/// `CoalescePartitionsExec`s with `SortPreservingMergeExec`s. -/// -/// If this replacement is helpful for removing a `SortExec`, it updates the plan. -/// Otherwise, it leaves the plan unchanged. -/// -/// NOTE: This optimizer sub-rule will only produce sort-preserving `RepartitionExec`s -/// if the query is bounded or if the config option `prefer_existing_sort` is -/// set to `true`. -/// -/// The algorithm flow is simply like this: -/// 1. Visit nodes of the physical plan bottom-up and look for `SortExec` nodes. -/// During the traversal, keep track of operators that maintain ordering (or -/// can maintain ordering when replaced by an order-preserving variant) until -/// a `SortExec` is found. -/// 2. When a `SortExec` is found, update the child of the `SortExec` by replacing -/// operators that do not preserve ordering in the tree with their order -/// preserving variants. -/// 3. Check if the `SortExec` is still necessary in the updated plan by comparing -/// its input ordering with the output ordering it imposes. We do this because -/// replacing operators that lose ordering with their order-preserving variants -/// enables us to preserve the previously lost ordering at the input of `SortExec`. -/// 4. If the `SortExec` in question turns out to be unnecessary, remove it and -/// use updated plan. Otherwise, use the original plan. -/// 5. Continue the bottom-up traversal until another `SortExec` is seen, or the -/// traversal is complete. -pub(crate) fn replace_with_order_preserving_variants( - mut requirements: OrderPreservationContext, - // A flag indicating that replacing `RepartitionExec`s with sort-preserving - // variants is desirable when it helps to remove a `SortExec` from the plan. - // If this flag is `false`, this replacement should only be made to fix the - // pipeline (streaming). - is_spr_better: bool, - // A flag indicating that replacing `CoalescePartitionsExec`s with - // `SortPreservingMergeExec`s is desirable when it helps to remove a - // `SortExec` from the plan. If this flag is `false`, this replacement - // should only be made to fix the pipeline (streaming). - is_spm_better: bool, - config: &ConfigOptions, -) -> Result> { - update_children(&mut requirements); - if !(is_sort(&requirements.plan) && requirements.children[0].data) { - return Ok(Transformed::no(requirements)); - } - - // For unbounded cases, we replace with the order-preserving variant in any - // case, as doing so helps fix the pipeline. Also replace if config allows. - let use_order_preserving_variant = config.optimizer.prefer_existing_sort - || (requirements.plan.boundedness().is_unbounded() - && requirements.plan.pipeline_behavior() == EmissionType::Final); - - // Create an alternate plan with order-preserving variants: - let mut alternate_plan = plan_with_order_preserving_variants( - requirements.children.swap_remove(0), - is_spr_better || use_order_preserving_variant, - is_spm_better || use_order_preserving_variant, - requirements.plan.fetch(), - )?; - - // If the alternate plan makes this sort unnecessary, accept the alternate: - if alternate_plan - .plan - .equivalence_properties() - .ordering_satisfy( - requirements - .plan - .output_ordering() - .unwrap_or(LexOrdering::empty()), - ) - { - for child in alternate_plan.children.iter_mut() { - child.data = false; - } - Ok(Transformed::yes(alternate_plan)) - } else { - // The alternate plan does not help, use faster order-breaking variants: - alternate_plan = plan_with_order_breaking_variants(alternate_plan)?; - alternate_plan.data = false; - requirements.children = vec![alternate_plan]; - Ok(Transformed::yes(requirements)) - } -} - -#[cfg(test)] -mod tests { - use super::*; - - use crate::execution::TaskContext; - use crate::physical_plan::coalesce_batches::CoalesceBatchesExec; - use crate::physical_plan::filter::FilterExec; - use crate::physical_plan::joins::{HashJoinExec, PartitionMode}; - use crate::physical_plan::sorts::sort::SortExec; - use crate::physical_plan::{ - displayable, get_plan_string, ExecutionPlan, Partitioning, - }; - use crate::prelude::{SessionConfig, SessionContext}; - use crate::test::TestStreamPartition; - use datafusion_physical_optimizer::test_utils::check_integrity; - - 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_plan::collect; - use datafusion_physical_plan::memory::MemoryExec; - use datafusion_physical_plan::streaming::StreamingTableExec; - use object_store::memory::InMemory; - use object_store::ObjectStore; - use url::Url; - - use rstest::rstest; - - /// 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. - /// - /// # Parameters - /// - /// * `EXPECTED_UNBOUNDED_PLAN_LINES`: Expected input unbounded plan. - /// * `EXPECTED_BOUNDED_PLAN_LINES`: Expected input bounded plan. - /// * `EXPECTED_UNBOUNDED_OPTIMIZED_PLAN_LINES`: Optimized plan, which is - /// the same regardless of the value of the `prefer_existing_sort` flag. - /// * `EXPECTED_BOUNDED_OPTIMIZED_PLAN_LINES`: Optimized plan when the flag - /// `prefer_existing_sort` is `false` for bounded cases. - /// * `EXPECTED_BOUNDED_PREFER_SORT_ON_OPTIMIZED_PLAN_LINES`: Optimized plan - /// when the flag `prefer_existing_sort` is `true` for bounded cases. - /// * `$PLAN`: The plan to optimize. - /// * `$SOURCE_UNBOUNDED`: Whether the given plan contains an unbounded source. - macro_rules! assert_optimized_in_all_boundedness_situations { - ($EXPECTED_UNBOUNDED_PLAN_LINES: expr, $EXPECTED_BOUNDED_PLAN_LINES: expr, $EXPECTED_UNBOUNDED_OPTIMIZED_PLAN_LINES: expr, $EXPECTED_BOUNDED_OPTIMIZED_PLAN_LINES: expr, $EXPECTED_BOUNDED_PREFER_SORT_ON_OPTIMIZED_PLAN_LINES: expr, $PLAN: expr, $SOURCE_UNBOUNDED: expr, $PREFER_EXISTING_SORT: expr) => { - if $SOURCE_UNBOUNDED { - assert_optimized_prefer_sort_on_off!( - $EXPECTED_UNBOUNDED_PLAN_LINES, - $EXPECTED_UNBOUNDED_OPTIMIZED_PLAN_LINES, - $EXPECTED_UNBOUNDED_OPTIMIZED_PLAN_LINES, - $PLAN, - $PREFER_EXISTING_SORT, - $SOURCE_UNBOUNDED - ); - } else { - assert_optimized_prefer_sort_on_off!( - $EXPECTED_BOUNDED_PLAN_LINES, - $EXPECTED_BOUNDED_OPTIMIZED_PLAN_LINES, - $EXPECTED_BOUNDED_PREFER_SORT_ON_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. - /// - /// # 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. - /// - /// # Parameters - /// - /// * `$EXPECTED_PLAN_LINES`: Expected input plan. - /// * `$EXPECTED_OPTIMIZED_PLAN_LINES`: Expected optimized plan. - /// * `$PLAN`: The plan to optimize. - /// * `$PREFER_EXISTING_SORT`: Value of the `prefer_existing_sort` flag. - macro_rules! assert_optimized { - ($EXPECTED_PLAN_LINES: expr, $EXPECTED_OPTIMIZED_PLAN_LINES: expr, $PLAN: expr, $PREFER_EXISTING_SORT: expr, $SOURCE_UNBOUNDED: 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_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 rule top-down - let config = SessionConfig::new().with_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 optimized_physical_plan = parallel.plan; - - // 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" - ); - - if !$SOURCE_UNBOUNDED { - let ctx = SessionContext::new(); - let object_store = InMemory::new(); - object_store.put(&object_store::path::Path::from("file_path"), bytes::Bytes::from("").into()).await?; - ctx.register_object_store(&Url::parse("test://").unwrap(), Arc::new(object_store)); - let task_ctx = Arc::new(TaskContext::from(&ctx)); - let res = collect(optimized_physical_plan, task_ctx).await; - assert!( - res.is_ok(), - "Some errors occurred while executing the optimized physical plan: {:?}", res.unwrap_err() - ); - } - }; - } - - #[rstest] - #[tokio::test] - // Searches for a simple sort and a repartition just after it, the second repartition with 1 input partition should not be affected - async fn test_replace_multiple_input_repartition_1( - #[values(false, true)] source_unbounded: bool, - #[values(false, true)] prefer_existing_sort: bool, - ) -> Result<()> { - let schema = create_test_schema()?; - let sort_exprs = vec![sort_expr("a", &schema)]; - let source = if source_unbounded { - stream_exec_ordered(&schema, sort_exprs) - } else { - memory_exec_sorted(&schema, sort_exprs) - }; - let repartition = repartition_exec_hash(repartition_exec_round_robin(source)); - let sort = sort_exec(vec![sort_expr("a", &schema)], repartition, true); - - let physical_plan = - sort_preserving_merge_exec(vec![sort_expr("a", &schema)], sort); - - // Expected inputs unbounded and bounded - let expected_input_unbounded = [ - "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", - " SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true]", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", - ]; - let expected_input_bounded = [ - "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", - " SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true]", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", - ]; - - // Expected unbounded result (same for with and without flag) - let expected_optimized_unbounded = [ - "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", - ]; - - // Expected bounded results with and without flag - let expected_optimized_bounded = [ - "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", - " SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true]", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", - ]; - let expected_optimized_bounded_sort_preserve = [ - "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", - ]; - assert_optimized_in_all_boundedness_situations!( - expected_input_unbounded, - expected_input_bounded, - expected_optimized_unbounded, - expected_optimized_bounded, - expected_optimized_bounded_sort_preserve, - physical_plan, - source_unbounded, - prefer_existing_sort - ); - Ok(()) - } - - #[rstest] - #[tokio::test] - async fn test_with_inter_children_change_only( - #[values(false, true)] source_unbounded: bool, - #[values(false, true)] prefer_existing_sort: bool, - ) -> Result<()> { - let schema = create_test_schema()?; - let sort_exprs = vec![sort_expr_default("a", &schema)]; - let source = if source_unbounded { - stream_exec_ordered(&schema, sort_exprs) - } else { - memory_exec_sorted(&schema, sort_exprs) - }; - let repartition_rr = repartition_exec_round_robin(source); - let repartition_hash = repartition_exec_hash(repartition_rr); - let coalesce_partitions = coalesce_partitions_exec(repartition_hash); - let sort = sort_exec( - vec![sort_expr_default("a", &coalesce_partitions.schema())], - coalesce_partitions, - false, - ); - let repartition_rr2 = repartition_exec_round_robin(sort); - let repartition_hash2 = repartition_exec_hash(repartition_rr2); - let filter = filter_exec(repartition_hash2); - let sort2 = - sort_exec(vec![sort_expr_default("a", &filter.schema())], filter, true); - - let physical_plan = sort_preserving_merge_exec( - vec![sort_expr_default("a", &sort2.schema())], - sort2, - ); - - // Expected inputs unbounded and bounded - let expected_input_unbounded = [ - "SortPreservingMergeExec: [a@0 ASC]", - " SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", - " FilterExec: c@1 > 3", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - " CoalescePartitionsExec", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC]", - ]; - let expected_input_bounded = [ - "SortPreservingMergeExec: [a@0 ASC]", - " SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", - " FilterExec: c@1 > 3", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - " CoalescePartitionsExec", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC", - ]; - - // Expected unbounded result (same for with and without flag) - let expected_optimized_unbounded = [ - "SortPreservingMergeExec: [a@0 ASC]", - " FilterExec: c@1 > 3", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " SortPreservingMergeExec: [a@0 ASC]", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC]", - ]; - - // Expected bounded results with and without flag - let expected_optimized_bounded = [ - "SortPreservingMergeExec: [a@0 ASC]", - " SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", - " FilterExec: c@1 > 3", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - " CoalescePartitionsExec", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC", - ]; - let expected_optimized_bounded_sort_preserve = [ - "SortPreservingMergeExec: [a@0 ASC]", - " FilterExec: c@1 > 3", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " SortPreservingMergeExec: [a@0 ASC]", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC", - ]; - assert_optimized_in_all_boundedness_situations!( - expected_input_unbounded, - expected_input_bounded, - expected_optimized_unbounded, - expected_optimized_bounded, - expected_optimized_bounded_sort_preserve, - physical_plan, - source_unbounded, - prefer_existing_sort - ); - Ok(()) - } - - #[rstest] - #[tokio::test] - async fn test_replace_multiple_input_repartition_2( - #[values(false, true)] source_unbounded: bool, - #[values(false, true)] prefer_existing_sort: bool, - ) -> Result<()> { - let schema = create_test_schema()?; - let sort_exprs = vec![sort_expr("a", &schema)]; - let source = if source_unbounded { - stream_exec_ordered(&schema, sort_exprs) - } else { - memory_exec_sorted(&schema, sort_exprs) - }; - let repartition_rr = repartition_exec_round_robin(source); - let filter = filter_exec(repartition_rr); - let repartition_hash = repartition_exec_hash(filter); - let sort = sort_exec(vec![sort_expr("a", &schema)], repartition_hash, true); - - let physical_plan = - sort_preserving_merge_exec(vec![sort_expr("a", &schema)], sort); - - // Expected inputs unbounded and bounded - let expected_input_unbounded = [ - "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", - " SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true]", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " FilterExec: c@1 > 3", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", - ]; - let expected_input_bounded = [ - "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", - " SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true]", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " FilterExec: c@1 > 3", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", - ]; - - // Expected unbounded result (same for with and without flag) - let expected_optimized_unbounded = [ - "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", - " FilterExec: c@1 > 3", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", - ]; - - // Expected bounded results with and without flag - let expected_optimized_bounded = [ - "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", - " SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true]", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " FilterExec: c@1 > 3", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", - ]; - let expected_optimized_bounded_sort_preserve = [ - "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", - " FilterExec: c@1 > 3", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", - ]; - assert_optimized_in_all_boundedness_situations!( - expected_input_unbounded, - expected_input_bounded, - expected_optimized_unbounded, - expected_optimized_bounded, - expected_optimized_bounded_sort_preserve, - physical_plan, - source_unbounded, - prefer_existing_sort - ); - Ok(()) - } - - #[rstest] - #[tokio::test] - async fn test_replace_multiple_input_repartition_with_extra_steps( - #[values(false, true)] source_unbounded: bool, - #[values(false, true)] prefer_existing_sort: bool, - ) -> Result<()> { - let schema = create_test_schema()?; - let sort_exprs = vec![sort_expr("a", &schema)]; - let source = if source_unbounded { - stream_exec_ordered(&schema, sort_exprs) - } else { - memory_exec_sorted(&schema, sort_exprs) - }; - let repartition_rr = repartition_exec_round_robin(source); - let repartition_hash = repartition_exec_hash(repartition_rr); - let filter = filter_exec(repartition_hash); - let coalesce_batches_exec: Arc = coalesce_batches_exec(filter); - let sort = sort_exec(vec![sort_expr("a", &schema)], coalesce_batches_exec, true); - - let physical_plan = - sort_preserving_merge_exec(vec![sort_expr("a", &schema)], sort); - - // Expected inputs unbounded and bounded - let expected_input_unbounded = [ - "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", - " SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true]", - " CoalesceBatchesExec: target_batch_size=8192", - " FilterExec: c@1 > 3", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", - ]; - let expected_input_bounded = [ - "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", - " SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true]", - " CoalesceBatchesExec: target_batch_size=8192", - " FilterExec: c@1 > 3", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", - ]; - - // Expected unbounded result (same for with and without flag) - let expected_optimized_unbounded = [ - "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", - " CoalesceBatchesExec: target_batch_size=8192", - " FilterExec: c@1 > 3", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", - ]; - - // Expected bounded results with and without flag - let expected_optimized_bounded = [ - "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", - " SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true]", - " CoalesceBatchesExec: target_batch_size=8192", - " FilterExec: c@1 > 3", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", - ]; - let expected_optimized_bounded_sort_preserve = [ - "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", - " CoalesceBatchesExec: target_batch_size=8192", - " FilterExec: c@1 > 3", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", - ]; - assert_optimized_in_all_boundedness_situations!( - expected_input_unbounded, - expected_input_bounded, - expected_optimized_unbounded, - expected_optimized_bounded, - expected_optimized_bounded_sort_preserve, - physical_plan, - source_unbounded, - prefer_existing_sort - ); - Ok(()) - } - - #[rstest] - #[tokio::test] - async fn test_replace_multiple_input_repartition_with_extra_steps_2( - #[values(false, true)] source_unbounded: bool, - #[values(false, true)] prefer_existing_sort: bool, - ) -> Result<()> { - let schema = create_test_schema()?; - let sort_exprs = vec![sort_expr("a", &schema)]; - let source = if source_unbounded { - stream_exec_ordered(&schema, sort_exprs) - } else { - memory_exec_sorted(&schema, sort_exprs) - }; - let repartition_rr = repartition_exec_round_robin(source); - let coalesce_batches_exec_1 = coalesce_batches_exec(repartition_rr); - let repartition_hash = repartition_exec_hash(coalesce_batches_exec_1); - let filter = filter_exec(repartition_hash); - let coalesce_batches_exec_2 = coalesce_batches_exec(filter); - let sort = - sort_exec(vec![sort_expr("a", &schema)], coalesce_batches_exec_2, true); - - let physical_plan = - sort_preserving_merge_exec(vec![sort_expr("a", &schema)], sort); - - // Expected inputs unbounded and bounded - let expected_input_unbounded = [ - "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", - " SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true]", - " CoalesceBatchesExec: target_batch_size=8192", - " FilterExec: c@1 > 3", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " CoalesceBatchesExec: target_batch_size=8192", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", - ]; - let expected_input_bounded = [ - "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", - " SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true]", - " CoalesceBatchesExec: target_batch_size=8192", - " FilterExec: c@1 > 3", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " CoalesceBatchesExec: target_batch_size=8192", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", - ]; - - // Expected unbounded result (same for with and without flag) - let expected_optimized_unbounded = [ - "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", - " CoalesceBatchesExec: target_batch_size=8192", - " FilterExec: c@1 > 3", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", - " CoalesceBatchesExec: target_batch_size=8192", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", - ]; - - // Expected bounded results with and without flag - let expected_optimized_bounded = [ - "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", - " SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true]", - " CoalesceBatchesExec: target_batch_size=8192", - " FilterExec: c@1 > 3", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " CoalesceBatchesExec: target_batch_size=8192", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", - ]; - let expected_optimized_bounded_sort_preserve = [ - "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", - " CoalesceBatchesExec: target_batch_size=8192", - " FilterExec: c@1 > 3", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", - " CoalesceBatchesExec: target_batch_size=8192", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", - ]; - assert_optimized_in_all_boundedness_situations!( - expected_input_unbounded, - expected_input_bounded, - expected_optimized_unbounded, - expected_optimized_bounded, - expected_optimized_bounded_sort_preserve, - physical_plan, - source_unbounded, - prefer_existing_sort - ); - Ok(()) - } - - #[rstest] - #[tokio::test] - async fn test_not_replacing_when_no_need_to_preserve_sorting( - #[values(false, true)] source_unbounded: bool, - #[values(false, true)] prefer_existing_sort: bool, - ) -> Result<()> { - let schema = create_test_schema()?; - let sort_exprs = vec![sort_expr("a", &schema)]; - let source = if source_unbounded { - stream_exec_ordered(&schema, sort_exprs) - } else { - memory_exec_sorted(&schema, sort_exprs) - }; - let repartition_rr = repartition_exec_round_robin(source); - let repartition_hash = repartition_exec_hash(repartition_rr); - let filter = filter_exec(repartition_hash); - let coalesce_batches_exec: Arc = coalesce_batches_exec(filter); - - let physical_plan: Arc = - coalesce_partitions_exec(coalesce_batches_exec); - - // Expected inputs unbounded and bounded - let expected_input_unbounded = [ - "CoalescePartitionsExec", - " CoalesceBatchesExec: target_batch_size=8192", - " FilterExec: c@1 > 3", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", - ]; - let expected_input_bounded = [ - "CoalescePartitionsExec", - " CoalesceBatchesExec: target_batch_size=8192", - " FilterExec: c@1 > 3", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", - ]; - - // Expected unbounded result (same for with and without flag) - let expected_optimized_unbounded = [ - "CoalescePartitionsExec", - " CoalesceBatchesExec: target_batch_size=8192", - " FilterExec: c@1 > 3", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", - ]; - - // Expected bounded results same with and without flag, because there is no executor with ordering requirement - let expected_optimized_bounded = [ - "CoalescePartitionsExec", - " CoalesceBatchesExec: target_batch_size=8192", - " FilterExec: c@1 > 3", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", - ]; - let expected_optimized_bounded_sort_preserve = expected_optimized_bounded; - - assert_optimized_in_all_boundedness_situations!( - expected_input_unbounded, - expected_input_bounded, - expected_optimized_unbounded, - expected_optimized_bounded, - expected_optimized_bounded_sort_preserve, - physical_plan, - source_unbounded, - prefer_existing_sort - ); - Ok(()) - } - - #[rstest] - #[tokio::test] - async fn test_with_multiple_replacable_repartitions( - #[values(false, true)] source_unbounded: bool, - #[values(false, true)] prefer_existing_sort: bool, - ) -> Result<()> { - let schema = create_test_schema()?; - let sort_exprs = vec![sort_expr("a", &schema)]; - let source = if source_unbounded { - stream_exec_ordered(&schema, sort_exprs) - } else { - memory_exec_sorted(&schema, sort_exprs) - }; - let repartition_rr = repartition_exec_round_robin(source); - let repartition_hash = repartition_exec_hash(repartition_rr); - let filter = filter_exec(repartition_hash); - let coalesce_batches = coalesce_batches_exec(filter); - let repartition_hash_2 = repartition_exec_hash(coalesce_batches); - let sort = sort_exec(vec![sort_expr("a", &schema)], repartition_hash_2, true); - - let physical_plan = - sort_preserving_merge_exec(vec![sort_expr("a", &schema)], sort); - - // Expected inputs unbounded and bounded - let expected_input_unbounded = [ - "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", - " SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true]", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " CoalesceBatchesExec: target_batch_size=8192", - " FilterExec: c@1 > 3", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", - ]; - let expected_input_bounded = [ - "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", - " SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true]", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " CoalesceBatchesExec: target_batch_size=8192", - " FilterExec: c@1 > 3", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", - ]; - - // Expected unbounded result (same for with and without flag) - let expected_optimized_unbounded = [ - "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", - " CoalesceBatchesExec: target_batch_size=8192", - " FilterExec: c@1 > 3", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", - ]; - - // Expected bounded results with and without flag - let expected_optimized_bounded = [ - "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", - " SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true]", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " CoalesceBatchesExec: target_batch_size=8192", - " FilterExec: c@1 > 3", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", - ]; - let expected_optimized_bounded_sort_preserve = [ - "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", - " CoalesceBatchesExec: target_batch_size=8192", - " FilterExec: c@1 > 3", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", - ]; - assert_optimized_in_all_boundedness_situations!( - expected_input_unbounded, - expected_input_bounded, - expected_optimized_unbounded, - expected_optimized_bounded, - expected_optimized_bounded_sort_preserve, - physical_plan, - source_unbounded, - prefer_existing_sort - ); - Ok(()) - } - - #[rstest] - #[tokio::test] - async fn test_not_replace_with_different_orderings( - #[values(false, true)] source_unbounded: bool, - #[values(false, true)] prefer_existing_sort: bool, - ) -> Result<()> { - let schema = create_test_schema()?; - let sort_exprs = vec![sort_expr("a", &schema)]; - let source = if source_unbounded { - stream_exec_ordered(&schema, sort_exprs) - } else { - memory_exec_sorted(&schema, sort_exprs) - }; - let repartition_rr = repartition_exec_round_robin(source); - let repartition_hash = repartition_exec_hash(repartition_rr); - let sort = sort_exec( - vec![sort_expr_default("c", &repartition_hash.schema())], - repartition_hash, - true, - ); - - let physical_plan = sort_preserving_merge_exec( - vec![sort_expr_default("c", &sort.schema())], - sort, - ); - - // Expected inputs unbounded and bounded - let expected_input_unbounded = [ - "SortPreservingMergeExec: [c@1 ASC]", - " SortExec: expr=[c@1 ASC], preserve_partitioning=[true]", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", - ]; - let expected_input_bounded = [ - "SortPreservingMergeExec: [c@1 ASC]", - " SortExec: expr=[c@1 ASC], preserve_partitioning=[true]", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", - ]; - - // Expected unbounded result (same for with and without flag) - let expected_optimized_unbounded = [ - "SortPreservingMergeExec: [c@1 ASC]", - " SortExec: expr=[c@1 ASC], preserve_partitioning=[true]", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", - ]; - - // Expected bounded results same with and without flag, because ordering requirement of the executor is different than the existing ordering. - let expected_optimized_bounded = [ - "SortPreservingMergeExec: [c@1 ASC]", - " SortExec: expr=[c@1 ASC], preserve_partitioning=[true]", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", - ]; - let expected_optimized_bounded_sort_preserve = expected_optimized_bounded; - - assert_optimized_in_all_boundedness_situations!( - expected_input_unbounded, - expected_input_bounded, - expected_optimized_unbounded, - expected_optimized_bounded, - expected_optimized_bounded_sort_preserve, - physical_plan, - source_unbounded, - prefer_existing_sort - ); - Ok(()) - } - - #[rstest] - #[tokio::test] - async fn test_with_lost_ordering( - #[values(false, true)] source_unbounded: bool, - #[values(false, true)] prefer_existing_sort: bool, - ) -> Result<()> { - let schema = create_test_schema()?; - let sort_exprs = vec![sort_expr("a", &schema)]; - let source = if source_unbounded { - stream_exec_ordered(&schema, sort_exprs) - } else { - memory_exec_sorted(&schema, sort_exprs) - }; - let repartition_rr = repartition_exec_round_robin(source); - let repartition_hash = repartition_exec_hash(repartition_rr); - let coalesce_partitions = coalesce_partitions_exec(repartition_hash); - let physical_plan = - sort_exec(vec![sort_expr("a", &schema)], coalesce_partitions, false); - - // Expected inputs unbounded and bounded - let expected_input_unbounded = [ - "SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[false]", - " CoalescePartitionsExec", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", - ]; - let expected_input_bounded = [ - "SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[false]", - " CoalescePartitionsExec", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", - ]; - - // Expected unbounded result (same for with and without flag) - let expected_optimized_unbounded = [ - "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", - ]; - - // Expected bounded results with and without flag - let expected_optimized_bounded = [ - "SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[false]", - " CoalescePartitionsExec", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", - ]; - let expected_optimized_bounded_sort_preserve = [ - "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", - ]; - assert_optimized_in_all_boundedness_situations!( - expected_input_unbounded, - expected_input_bounded, - expected_optimized_unbounded, - expected_optimized_bounded, - expected_optimized_bounded_sort_preserve, - physical_plan, - source_unbounded, - prefer_existing_sort - ); - Ok(()) - } - - #[rstest] - #[tokio::test] - async fn test_with_lost_and_kept_ordering( - #[values(false, true)] source_unbounded: bool, - #[values(false, true)] prefer_existing_sort: bool, - ) -> Result<()> { - let schema = create_test_schema()?; - let sort_exprs = vec![sort_expr("a", &schema)]; - let source = if source_unbounded { - stream_exec_ordered(&schema, sort_exprs) - } else { - memory_exec_sorted(&schema, sort_exprs) - }; - let repartition_rr = repartition_exec_round_robin(source); - let repartition_hash = repartition_exec_hash(repartition_rr); - let coalesce_partitions = coalesce_partitions_exec(repartition_hash); - let sort = sort_exec( - vec![sort_expr_default("c", &coalesce_partitions.schema())], - coalesce_partitions, - false, - ); - let repartition_rr2 = repartition_exec_round_robin(sort); - let repartition_hash2 = repartition_exec_hash(repartition_rr2); - let filter = filter_exec(repartition_hash2); - let sort2 = - sort_exec(vec![sort_expr_default("c", &filter.schema())], filter, true); - - let physical_plan = sort_preserving_merge_exec( - vec![sort_expr_default("c", &sort2.schema())], - sort2, - ); - - // Expected inputs unbounded and bounded - let expected_input_unbounded = [ - "SortPreservingMergeExec: [c@1 ASC]", - " SortExec: expr=[c@1 ASC], preserve_partitioning=[true]", - " FilterExec: c@1 > 3", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " SortExec: expr=[c@1 ASC], preserve_partitioning=[false]", - " CoalescePartitionsExec", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", - ]; - let expected_input_bounded = [ - "SortPreservingMergeExec: [c@1 ASC]", - " SortExec: expr=[c@1 ASC], preserve_partitioning=[true]", - " FilterExec: c@1 > 3", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " SortExec: expr=[c@1 ASC], preserve_partitioning=[false]", - " CoalescePartitionsExec", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", - ]; - - // Expected unbounded result (same for with and without flag) - let expected_optimized_unbounded = [ - "SortPreservingMergeExec: [c@1 ASC]", - " FilterExec: c@1 > 3", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=c@1 ASC", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " SortExec: expr=[c@1 ASC], preserve_partitioning=[false]", - " CoalescePartitionsExec", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", - ]; - - // Expected bounded results with and without flag - let expected_optimized_bounded = [ - "SortPreservingMergeExec: [c@1 ASC]", - " SortExec: expr=[c@1 ASC], preserve_partitioning=[true]", - " FilterExec: c@1 > 3", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " SortExec: expr=[c@1 ASC], preserve_partitioning=[false]", - " CoalescePartitionsExec", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", - ]; - let expected_optimized_bounded_sort_preserve = [ - "SortPreservingMergeExec: [c@1 ASC]", - " FilterExec: c@1 > 3", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=c@1 ASC", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " SortExec: expr=[c@1 ASC], preserve_partitioning=[false]", - " CoalescePartitionsExec", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", - ]; - assert_optimized_in_all_boundedness_situations!( - expected_input_unbounded, - expected_input_bounded, - expected_optimized_unbounded, - expected_optimized_bounded, - expected_optimized_bounded_sort_preserve, - physical_plan, - source_unbounded, - prefer_existing_sort - ); - Ok(()) - } - - #[rstest] - #[tokio::test] - async fn test_with_multiple_child_trees( - #[values(false, true)] source_unbounded: bool, - #[values(false, true)] prefer_existing_sort: bool, - ) -> Result<()> { - let schema = create_test_schema()?; - - let left_sort_exprs = vec![sort_expr("a", &schema)]; - let left_source = if source_unbounded { - stream_exec_ordered(&schema, left_sort_exprs) - } else { - memory_exec_sorted(&schema, left_sort_exprs) - }; - let left_repartition_rr = repartition_exec_round_robin(left_source); - let left_repartition_hash = repartition_exec_hash(left_repartition_rr); - let left_coalesce_partitions = - Arc::new(CoalesceBatchesExec::new(left_repartition_hash, 4096)); - - let right_sort_exprs = vec![sort_expr("a", &schema)]; - let right_source = if source_unbounded { - stream_exec_ordered(&schema, right_sort_exprs) - } else { - memory_exec_sorted(&schema, right_sort_exprs) - }; - let right_repartition_rr = repartition_exec_round_robin(right_source); - let right_repartition_hash = repartition_exec_hash(right_repartition_rr); - let right_coalesce_partitions = - Arc::new(CoalesceBatchesExec::new(right_repartition_hash, 4096)); - - let hash_join_exec = - hash_join_exec(left_coalesce_partitions, right_coalesce_partitions); - let sort = sort_exec( - vec![sort_expr_default("a", &hash_join_exec.schema())], - hash_join_exec, - true, - ); - - let physical_plan = sort_preserving_merge_exec( - vec![sort_expr_default("a", &sort.schema())], - sort, - ); - - // Expected inputs unbounded and bounded - let expected_input_unbounded = [ - "SortPreservingMergeExec: [a@0 ASC]", - " SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", - " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@1, c@1)]", - " CoalesceBatchesExec: target_batch_size=4096", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", - " CoalesceBatchesExec: target_batch_size=4096", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", - ]; - let expected_input_bounded = [ - "SortPreservingMergeExec: [a@0 ASC]", - " SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", - " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@1, c@1)]", - " CoalesceBatchesExec: target_batch_size=4096", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", - " CoalesceBatchesExec: target_batch_size=4096", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", - ]; - - // Expected unbounded result (same for with and without flag) - let expected_optimized_unbounded = [ - "SortPreservingMergeExec: [a@0 ASC]", - " SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", - " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@1, c@1)]", - " CoalesceBatchesExec: target_batch_size=4096", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", - " CoalesceBatchesExec: target_batch_size=4096", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", - ]; - - // Expected bounded results same with and without flag, because ordering get lost during intermediate executor anyway. Hence no need to preserve - // existing ordering. - let expected_optimized_bounded = [ - "SortPreservingMergeExec: [a@0 ASC]", - " SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", - " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@1, c@1)]", - " CoalesceBatchesExec: target_batch_size=4096", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", - " CoalesceBatchesExec: target_batch_size=4096", - " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", - " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", - ]; - let expected_optimized_bounded_sort_preserve = expected_optimized_bounded; - - assert_optimized_in_all_boundedness_situations!( - expected_input_unbounded, - expected_input_bounded, - expected_optimized_unbounded, - expected_optimized_bounded, - expected_optimized_bounded_sort_preserve, - physical_plan, - source_unbounded, - prefer_existing_sort - ); - Ok(()) - } - - // End test cases - // Start test helpers - - fn sort_expr(name: &str, schema: &Schema) -> PhysicalSortExpr { - let sort_opts = SortOptions { - nulls_first: false, - descending: false, - }; - sort_expr_options(name, schema, sort_opts) - } - - fn sort_expr_default(name: &str, schema: &Schema) -> PhysicalSortExpr { - let sort_opts = SortOptions::default(); - sort_expr_options(name, schema, sort_opts) - } - - fn sort_expr_options( - name: &str, - schema: &Schema, - options: SortOptions, - ) -> PhysicalSortExpr { - PhysicalSortExpr { - expr: col(name, schema).unwrap(), - options, - } - } - - fn sort_exec( - sort_exprs: impl IntoIterator, - input: Arc, - preserve_partitioning: bool, - ) -> Arc { - let sort_exprs = sort_exprs.into_iter().collect(); - Arc::new( - SortExec::new(sort_exprs, input) - .with_preserve_partitioning(preserve_partitioning), - ) - } - - fn sort_preserving_merge_exec( - sort_exprs: impl IntoIterator, - input: Arc, - ) -> Arc { - let sort_exprs = sort_exprs.into_iter().collect(); - Arc::new(SortPreservingMergeExec::new(sort_exprs, input)) - } - - fn repartition_exec_round_robin( - input: Arc, - ) -> Arc { - Arc::new( - RepartitionExec::try_new(input, Partitioning::RoundRobinBatch(8)).unwrap(), - ) - } - - fn repartition_exec_hash(input: Arc) -> Arc { - let input_schema = input.schema(); - Arc::new( - RepartitionExec::try_new( - input, - Partitioning::Hash(vec![col("c", &input_schema).unwrap()], 8), - ) - .unwrap(), - ) - } - - fn filter_exec(input: Arc) -> Arc { - let input_schema = input.schema(); - let predicate = expressions::binary( - col("c", &input_schema).unwrap(), - Operator::Gt, - expressions::lit(3i32), - &input_schema, - ) - .unwrap(); - Arc::new(FilterExec::try_new(predicate, input).unwrap()) - } - - fn coalesce_batches_exec(input: Arc) -> Arc { - Arc::new(CoalesceBatchesExec::new(input, 8192)) - } - - fn coalesce_partitions_exec(input: Arc) -> Arc { - Arc::new(CoalescePartitionsExec::new(input)) - } - - fn hash_join_exec( - left: Arc, - right: Arc, - ) -> Arc { - let left_on = col("c", &left.schema()).unwrap(); - let right_on = col("c", &right.schema()).unwrap(); - let left_col = left_on.as_any().downcast_ref::().unwrap(); - let right_col = right_on.as_any().downcast_ref::().unwrap(); - Arc::new( - HashJoinExec::try_new( - left, - right, - vec![(Arc::new(left_col.clone()), Arc::new(right_col.clone()))], - None, - &JoinType::Inner, - None, - PartitionMode::Partitioned, - false, - ) - .unwrap(), - ) - } - - fn create_test_schema() -> Result { - let column_a = Field::new("a", DataType::Int32, false); - let column_b = Field::new("b", DataType::Int32, false); - let column_c = Field::new("c", DataType::Int32, false); - let column_d = Field::new("d", DataType::Int32, false); - let schema = Arc::new(Schema::new(vec![column_a, column_b, column_c, column_d])); - - Ok(schema) - } - - // creates a stream exec source for the test purposes - fn stream_exec_ordered( - 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(), - ) - } - - // creates a memory exec source for the test purposes - // projection parameter is given static due to testing needs - fn memory_exec_sorted( - schema: &SchemaRef, - sort_exprs: impl IntoIterator, - ) -> Arc { - pub fn make_partition(schema: &SchemaRef, sz: i32) -> RecordBatch { - let values = (0..sz).collect::>(); - let arr = Arc::new(Int32Array::from(values)); - let arr = arr as ArrayRef; - - RecordBatch::try_new( - schema.clone(), - vec![arr.clone(), arr.clone(), arr.clone(), arr], - ) - .unwrap() - } - - let rows = 5; - let partitions = 1; - let sort_exprs = sort_exprs.into_iter().collect(); - Arc::new({ - let data: Vec> = (0..partitions) - .map(|_| vec![make_partition(schema, rows)]) - .collect(); - let projection: Vec = vec![0, 2, 3]; - MemoryExec::try_new(&data, schema.clone(), Some(projection)) - .unwrap() - .try_with_sort_information(vec![sort_exprs]) - .unwrap() - }) - } -} diff --git a/datafusion/core/src/physical_optimizer/test_utils.rs b/datafusion/core/src/physical_optimizer/test_utils.rs deleted file mode 100644 index aba24309b2a0..000000000000 --- a/datafusion/core/src/physical_optimizer/test_utils.rs +++ /dev/null @@ -1,54 +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. - -//! Collection of testing utility functions that are leveraged by the query optimizer rules - -#![allow(missing_docs)] - -use std::sync::Arc; - -use crate::datasource::listing::PartitionedFile; -use crate::datasource::physical_plan::{FileScanConfig, ParquetExec}; -use crate::physical_plan::ExecutionPlan; - -use arrow_schema::SchemaRef; -use datafusion_execution::object_store::ObjectStoreUrl; -use datafusion_physical_expr::PhysicalSortExpr; - -/// 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() -} diff --git a/datafusion/core/src/test/mod.rs b/datafusion/core/src/test/mod.rs index e91785c7421a..0d659582aca3 100644 --- a/datafusion/core/src/test/mod.rs +++ b/datafusion/core/src/test/mod.rs @@ -34,7 +34,6 @@ use crate::datasource::physical_plan::{CsvExec, FileScanConfig}; use crate::datasource::{MemTable, TableProvider}; use crate::error::Result; use crate::logical_expr::LogicalPlan; -use crate::physical_plan::ExecutionPlan; use crate::test::object_store::local_unpartitioned_file; use crate::test_util::{aggr_test_schema, arrow_test_data}; @@ -42,9 +41,6 @@ use arrow::array::{self, Array, ArrayRef, Decimal128Builder, Int32Array}; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use arrow::record_batch::RecordBatch; use datafusion_common::DataFusionError; -use datafusion_execution::{SendableRecordBatchStream, TaskContext}; -use datafusion_physical_expr::PhysicalSortExpr; -use datafusion_physical_plan::streaming::{PartitionStream, StreamingTableExec}; #[cfg(feature = "compression")] use bzip2::write::BzEncoder; @@ -290,96 +286,5 @@ fn make_decimal() -> RecordBatch { RecordBatch::try_new(Arc::new(schema), vec![Arc::new(array)]).unwrap() } -/// 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(), - ) -} - -/// 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(), - ) -} - pub mod object_store; pub mod variable; diff --git a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs new file mode 100644 index 000000000000..21dbe06d7ac2 --- /dev/null +++ b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs @@ -0,0 +1,1944 @@ +// 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 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 arrow::compute::SortOptions; +use arrow::datatypes::{DataType, Field, 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_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 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) +} + +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) +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) +} + +/// 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 config = SessionConfig::new().with_repartition_sorts($REPARTITION_SORTS); + let session_ctx = SessionContext::new_with_config(config); + let state = session_ctx.state(); + + // 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 state.config_options().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, + state.config_options(), + ) + }) + .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, state.config_options())?; + + // 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_sort5() -> Result<()> { + let left_schema = create_test_schema2()?; + let right_schema = create_test_schema3()?; + let left_input = memory_exec(&left_schema); + let parquet_sort_exprs = vec![sort_expr("a", &right_schema)]; + let right_input = parquet_exec_sorted(&right_schema, parquet_sort_exprs); + + let on = vec![( + Arc::new(Column::new_with_schema("col_a", &left_schema)?) as _, + Arc::new(Column::new_with_schema("c", &right_schema)?) as _, + )]; + let join = hash_join_exec(left_input, right_input, on, None, &JoinType::Inner)?; + 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]"]; + + 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]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + + Ok(()) +} + +#[tokio::test] +async fn test_do_not_remove_sort_with_limit() -> Result<()> { + let schema = create_test_schema()?; + + let source1 = parquet_exec(&schema); + let sort_exprs = vec![ + sort_expr("nullable_col", &schema), + sort_expr("non_nullable_col", &schema), + ]; + let sort = sort_exec(sort_exprs.clone(), source1); + let limit = limit_exec(sort); + + let parquet_sort_exprs = vec![sort_expr("nullable_col", &schema)]; + let source2 = parquet_exec_sorted(&schema, parquet_sort_exprs); + + let union = union_exec(vec![source2, limit]); + let repartition = repartition_exec(union); + 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]"]; + + // 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]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + + Ok(()) +} + +#[tokio::test] +async fn test_union_inputs_sorted() -> Result<()> { + let schema = create_test_schema()?; + + let source1 = parquet_exec(&schema); + let sort_exprs = vec![sort_expr("nullable_col", &schema)]; + let sort = sort_exec(sort_exprs.clone(), source1); + + let source2 = parquet_exec_sorted(&schema, sort_exprs.clone()); + + let union = union_exec(vec![source2, sort]); + let physical_plan = sort_preserving_merge_exec(sort_exprs, union); + + // 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]", + ]; + // 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); + + Ok(()) +} + +#[tokio::test] +async fn test_union_inputs_different_sorted() -> Result<()> { + let schema = create_test_schema()?; + + let source1 = parquet_exec(&schema); + let sort_exprs = vec![sort_expr("nullable_col", &schema)]; + let sort = sort_exec(sort_exprs.clone(), source1); + + let parquet_sort_exprs = vec![ + sort_expr("nullable_col", &schema), + sort_expr("non_nullable_col", &schema), + ]; + let source2 = parquet_exec_sorted(&schema, parquet_sort_exprs); + + let union = union_exec(vec![source2, sort]); + let physical_plan = sort_preserving_merge_exec(sort_exprs, union); + + // 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]", + ]; + // 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); + + Ok(()) +} + +#[tokio::test] +async fn test_union_inputs_different_sorted2() -> Result<()> { + let schema = create_test_schema()?; + + let source1 = parquet_exec(&schema); + let sort_exprs = vec![ + sort_expr("nullable_col", &schema), + sort_expr("non_nullable_col", &schema), + ]; + let sort = sort_exec(sort_exprs.clone(), source1); + + let parquet_sort_exprs = vec![sort_expr("nullable_col", &schema)]; + let source2 = parquet_exec_sorted(&schema, parquet_sort_exprs); + + let union = union_exec(vec![source2, sort]); + let physical_plan = sort_preserving_merge_exec(sort_exprs, union); + + // Input is an invalid plan. In this case rule should add required sorting in appropriate places. + // First ParquetExec has output ordering(nullable_col@0 ASC). However, it doesn't satisfy the + // 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]"]; + + 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]"]; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + + Ok(()) +} + +#[tokio::test] +async fn test_union_inputs_different_sorted3() -> Result<()> { + let schema = create_test_schema()?; + + let source1 = parquet_exec(&schema); + let sort_exprs1 = vec![ + sort_expr("nullable_col", &schema), + sort_expr("non_nullable_col", &schema), + ]; + let sort1 = sort_exec(sort_exprs1, source1.clone()); + let sort_exprs2 = vec![sort_expr("nullable_col", &schema)]; + let sort2 = sort_exec(sort_exprs2, source1); + + let parquet_sort_exprs = vec![sort_expr("nullable_col", &schema)]; + let source2 = parquet_exec_sorted(&schema, parquet_sort_exprs.clone()); + + let union = union_exec(vec![sort1, source2, sort2]); + let physical_plan = sort_preserving_merge_exec(parquet_sort_exprs, union); + + // First input to the union is not Sorted (SortExec is finer than required ordering by the SortPreservingMergeExec above). + // 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]"]; + // 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]"]; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + + Ok(()) +} + +#[tokio::test] +async fn test_union_inputs_different_sorted4() -> Result<()> { + let schema = create_test_schema()?; + + let source1 = parquet_exec(&schema); + let sort_exprs1 = vec![ + sort_expr("nullable_col", &schema), + sort_expr("non_nullable_col", &schema), + ]; + let sort_exprs2 = vec![sort_expr("nullable_col", &schema)]; + let sort1 = sort_exec(sort_exprs2.clone(), source1.clone()); + let sort2 = sort_exec(sort_exprs2.clone(), source1); + + let source2 = parquet_exec_sorted(&schema, sort_exprs2); + + let union = union_exec(vec![sort1, source2, sort2]); + let physical_plan = sort_preserving_merge_exec(sort_exprs1, union); + + // Ordering requirement of the `SortPreservingMergeExec` is not met. + // Should modify the plan to ensure that all three inputs to the + // `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]"]; + 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]"]; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + + Ok(()) +} + +#[tokio::test] +async fn test_union_inputs_different_sorted5() -> Result<()> { + let schema = create_test_schema()?; + + let source1 = parquet_exec(&schema); + let sort_exprs1 = vec![ + sort_expr("nullable_col", &schema), + sort_expr("non_nullable_col", &schema), + ]; + let sort_exprs2 = vec![ + sort_expr("nullable_col", &schema), + sort_expr_options( + "non_nullable_col", + &schema, + SortOptions { + descending: true, + nulls_first: false, + }, + ), + ]; + let sort_exprs3 = vec![sort_expr("nullable_col", &schema)]; + let sort1 = sort_exec(sort_exprs1, source1.clone()); + let sort2 = sort_exec(sort_exprs2, source1); + + let union = union_exec(vec![sort1, sort2]); + let physical_plan = sort_preserving_merge_exec(sort_exprs3, union); + + // The `UnionExec` doesn't preserve any of the inputs ordering in the + // 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]"]; + 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]"]; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + + Ok(()) +} + +#[tokio::test] +async fn test_union_inputs_different_sorted6() -> Result<()> { + let schema = create_test_schema()?; + + let source1 = parquet_exec(&schema); + let sort_exprs1 = vec![sort_expr("nullable_col", &schema)]; + let sort1 = sort_exec(sort_exprs1, source1.clone()); + let sort_exprs2 = vec![ + sort_expr("nullable_col", &schema), + sort_expr("non_nullable_col", &schema), + ]; + let repartition = repartition_exec(source1); + let spm = sort_preserving_merge_exec(sort_exprs2, repartition); + + let parquet_sort_exprs = vec![sort_expr("nullable_col", &schema)]; + let source2 = parquet_exec_sorted(&schema, parquet_sort_exprs.clone()); + + let union = union_exec(vec![sort1, source2, spm]); + let physical_plan = sort_preserving_merge_exec(parquet_sort_exprs, union); + + // The plan is not valid as it is -- the input ordering requirement + // of the `SortPreservingMergeExec` under the third child of the + // `UnionExec` is not met. We should add a `SortExec` below it. + // At the same time, this ordering requirement is unnecessarily fine. + // 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]"]; + // 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]"]; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + + Ok(()) +} + +#[tokio::test] +async fn test_union_inputs_different_sorted7() -> Result<()> { + let schema = create_test_schema()?; + + let source1 = parquet_exec(&schema); + let sort_exprs1 = vec![ + sort_expr("nullable_col", &schema), + sort_expr("non_nullable_col", &schema), + ]; + let sort_exprs3 = vec![sort_expr("nullable_col", &schema)]; + let sort1 = sort_exec(sort_exprs1.clone(), source1.clone()); + let sort2 = sort_exec(sort_exprs1, source1); + + let union = union_exec(vec![sort1, sort2]); + let physical_plan = sort_preserving_merge_exec(sort_exprs3, union); + + // 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]"]; + // 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]"]; + assert_optimized!(expected_input, expected_output, physical_plan, true); + + Ok(()) +} + +#[tokio::test] +async fn test_union_inputs_different_sorted8() -> Result<()> { + let schema = create_test_schema()?; + + let source1 = parquet_exec(&schema); + let sort_exprs1 = vec![ + sort_expr("nullable_col", &schema), + sort_expr("non_nullable_col", &schema), + ]; + let sort_exprs2 = vec![ + sort_expr_options( + "nullable_col", + &schema, + SortOptions { + descending: true, + nulls_first: false, + }, + ), + sort_expr_options( + "non_nullable_col", + &schema, + SortOptions { + descending: true, + nulls_first: false, + }, + ), + ]; + let sort1 = sort_exec(sort_exprs1, source1.clone()); + let sort2 = sort_exec(sort_exprs2, source1); + + let physical_plan = union_exec(vec![sort1, sort2]); + + // 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]"]; + // 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]"]; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + + Ok(()) +} + +#[tokio::test] +async fn test_window_multi_path_sort() -> Result<()> { + let schema = create_test_schema()?; + + let sort_exprs1 = vec![ + sort_expr("nullable_col", &schema), + sort_expr("non_nullable_col", &schema), + ]; + let sort_exprs2 = vec![sort_expr("nullable_col", &schema)]; + // reverse sorting of sort_exprs2 + let sort_exprs3 = vec![sort_expr_options( + "nullable_col", + &schema, + SortOptions { + descending: true, + nulls_first: false, + }, + )]; + let source1 = parquet_exec_sorted(&schema, sort_exprs1); + let source2 = parquet_exec_sorted(&schema, sort_exprs2); + let sort1 = sort_exec(sort_exprs3.clone(), source1); + let sort2 = sort_exec(sort_exprs3.clone(), source2); + + let union = union_exec(vec![sort1, sort2]); + let spm = sort_preserving_merge_exec(sort_exprs3.clone(), union); + let physical_plan = bounded_window_exec("nullable_col", sort_exprs3, spm); + + // The `WindowAggExec` gets its sorting from multiple children jointly. + // During the removal of `SortExec`s, it should be able to remove the + // 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]"]; + 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]"]; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + + Ok(()) +} + +#[tokio::test] +async fn test_window_multi_path_sort2() -> Result<()> { + let schema = create_test_schema()?; + + let sort_exprs1 = LexOrdering::new(vec![ + sort_expr("nullable_col", &schema), + sort_expr("non_nullable_col", &schema), + ]); + let sort_exprs2 = vec![sort_expr("nullable_col", &schema)]; + let source1 = parquet_exec_sorted(&schema, sort_exprs2.clone()); + let source2 = parquet_exec_sorted(&schema, sort_exprs2.clone()); + let sort1 = sort_exec(sort_exprs1.clone(), source1); + let sort2 = sort_exec(sort_exprs1.clone(), source2); + + let union = union_exec(vec![sort1, sort2]); + let spm = Arc::new(SortPreservingMergeExec::new(sort_exprs1, union)) as _; + let physical_plan = bounded_window_exec("nullable_col", sort_exprs2, spm); + + // 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]"]; + 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]"]; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + + Ok(()) +} + +#[tokio::test] +async fn test_union_inputs_different_sorted_with_limit() -> Result<()> { + let schema = create_test_schema()?; + + let source1 = parquet_exec(&schema); + let sort_exprs1 = vec![ + sort_expr("nullable_col", &schema), + sort_expr("non_nullable_col", &schema), + ]; + let sort_exprs2 = vec![ + sort_expr("nullable_col", &schema), + sort_expr_options( + "non_nullable_col", + &schema, + SortOptions { + descending: true, + nulls_first: false, + }, + ), + ]; + let sort_exprs3 = vec![sort_expr("nullable_col", &schema)]; + let sort1 = sort_exec(sort_exprs1, source1.clone()); + + let sort2 = sort_exec(sort_exprs2, source1); + let limit = local_limit_exec(sort2); + let limit = global_limit_exec(limit); + + let union = union_exec(vec![sort1, limit]); + let physical_plan = sort_preserving_merge_exec(sort_exprs3, union); + + // 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]"]; + 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]"]; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + + Ok(()) +} + +#[tokio::test] +async fn test_sort_merge_join_order_by_left() -> Result<()> { + let left_schema = create_test_schema()?; + let right_schema = create_test_schema2()?; + + let left = parquet_exec(&left_schema); + let right = parquet_exec(&right_schema); + + // Join on (nullable_col == col_a) + let join_on = vec![( + Arc::new(Column::new_with_schema("nullable_col", &left.schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("col_a", &right.schema()).unwrap()) as _, + )]; + + let join_types = vec![ + JoinType::Inner, + JoinType::Left, + JoinType::Right, + JoinType::Full, + JoinType::LeftSemi, + JoinType::LeftAnti, + ]; + for join_type in join_types { + let join = + sort_merge_join_exec(left.clone(), right.clone(), &join_on, &join_type); + let sort_exprs = vec![ + sort_expr("nullable_col", &join.schema()), + sort_expr("non_nullable_col", &join.schema()), + ]; + let physical_plan = sort_preserving_merge_exec(sort_exprs.clone(), join); + + let join_plan = format!( + "SortMergeJoin: join_type={join_type}, on=[(nullable_col@0, col_a@0)]" + ); + let join_plan2 = format!( + " 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]"]; + let expected_optimized = match join_type { + JoinType::Inner + | JoinType::Left + | JoinType::LeftSemi + | 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]", + ] + } + _ => { + // 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]", + ] + } + }; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + } + Ok(()) +} + +#[tokio::test] +async fn test_sort_merge_join_order_by_right() -> Result<()> { + let left_schema = create_test_schema()?; + let right_schema = create_test_schema2()?; + + let left = parquet_exec(&left_schema); + let right = parquet_exec(&right_schema); + + // Join on (nullable_col == col_a) + let join_on = vec![( + Arc::new(Column::new_with_schema("nullable_col", &left.schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("col_a", &right.schema()).unwrap()) as _, + )]; + + let join_types = vec![ + JoinType::Inner, + JoinType::Left, + JoinType::Right, + JoinType::Full, + JoinType::RightAnti, + ]; + for join_type in join_types { + let join = + sort_merge_join_exec(left.clone(), right.clone(), &join_on, &join_type); + let sort_exprs = vec![ + sort_expr("col_a", &join.schema()), + sort_expr("col_b", &join.schema()), + ]; + let physical_plan = sort_preserving_merge_exec(sort_exprs, join); + + let join_plan = format!( + "SortMergeJoin: join_type={join_type}, on=[(nullable_col@0, col_a@0)]" + ); + let spm_plan = match join_type { + JoinType::RightAnti => "SortPreservingMergeExec: [col_a@0 ASC, col_b@1 ASC]", + _ => "SortPreservingMergeExec: [col_a@2 ASC, col_b@3 ASC]", + }; + let join_plan2 = format!( + " 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]"]; + 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]", + ] + } + _ => { + // 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]", + ] + } + }; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + } + Ok(()) +} + +#[tokio::test] +async fn test_sort_merge_join_complex_order_by() -> Result<()> { + let left_schema = create_test_schema()?; + let right_schema = create_test_schema2()?; + + let left = parquet_exec(&left_schema); + let right = parquet_exec(&right_schema); + + // Join on (nullable_col == col_a) + let join_on = vec![( + Arc::new(Column::new_with_schema("nullable_col", &left.schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("col_a", &right.schema()).unwrap()) as _, + )]; + + let join = sort_merge_join_exec(left, right, &join_on, &JoinType::Inner); + + // order by (col_b, col_a) + let sort_exprs1 = vec![ + sort_expr("col_b", &join.schema()), + sort_expr("col_a", &join.schema()), + ]; + 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]"]; + + // 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]"]; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + + // order by (nullable_col, col_b, col_a) + let sort_exprs2 = vec![ + sort_expr("nullable_col", &join.schema()), + sort_expr("col_b", &join.schema()), + sort_expr("col_a", &join.schema()), + ]; + 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]"]; + + // 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]"]; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + + Ok(()) +} + +#[tokio::test] +async fn test_multilayer_coalesce_partitions() -> Result<()> { + let schema = create_test_schema()?; + + let source1 = parquet_exec(&schema); + let repartition = repartition_exec(source1); + let coalesce = Arc::new(CoalescePartitionsExec::new(repartition)) as _; + // 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(), + )), + coalesce, + ); + let sort_exprs = vec![sort_expr("nullable_col", &schema)]; + let physical_plan = sort_exec(sort_exprs, filter); + + // CoalescePartitionsExec and SortExec are not directly consecutive. In this case + // 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]", + " 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_with_lost_ordering_bounded() -> Result<()> { + let schema = create_test_schema3()?; + let sort_exprs = vec![sort_expr("a", &schema)]; + let source = csv_exec_sorted(&schema, sort_exprs); + let repartition_rr = repartition_exec(source); + let repartition_hash = Arc::new(RepartitionExec::try_new( + repartition_rr, + Partitioning::Hash(vec![col("c", &schema).unwrap()], 10), + )?) as _; + let coalesce_partitions = coalesce_partitions_exec(repartition_hash); + 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"]; + 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"]; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + + Ok(()) +} + +#[rstest] +#[tokio::test] +async fn test_with_lost_ordering_unbounded_bounded( + #[values(false, true)] source_unbounded: bool, +) -> Result<()> { + let schema = create_test_schema3()?; + let sort_exprs = vec![sort_expr("a", &schema)]; + // create either bounded or unbounded source + let source = if source_unbounded { + stream_exec_ordered(&schema, sort_exprs) + } else { + csv_exec_ordered(&schema, sort_exprs) + }; + let repartition_rr = repartition_exec(source); + let repartition_hash = Arc::new(RepartitionExec::try_new( + repartition_rr, + Partitioning::Hash(vec![col("c", &schema).unwrap()], 10), + )?) as _; + let coalesce_partitions = coalesce_partitions_exec(repartition_hash); + let physical_plan = sort_exec(vec![sort_expr("a", &schema)], coalesce_partitions); + + // 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]", + ]; + 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", + ]; + + // 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]", + ]; + + // 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", + ]; + 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", + ]; + let (expected_input, expected_optimized, expected_optimized_sort_parallelize) = + if source_unbounded { + ( + expected_input_unbounded, + expected_optimized_unbounded.clone(), + expected_optimized_unbounded, + ) + } else { + ( + expected_input_bounded, + expected_optimized_bounded, + expected_optimized_bounded_parallelize_sort, + ) + }; + assert_optimized!( + expected_input, + expected_optimized, + physical_plan.clone(), + false + ); + assert_optimized!( + expected_input, + expected_optimized_sort_parallelize, + physical_plan, + true + ); + + Ok(()) +} + +#[tokio::test] +async fn test_do_not_pushdown_through_spm() -> Result<()> { + let schema = create_test_schema3()?; + let sort_exprs = vec![sort_expr("a", &schema), sort_expr("b", &schema)]; + let source = csv_exec_sorted(&schema, sort_exprs.clone()); + let repartition_rr = repartition_exec(source); + let spm = sort_preserving_merge_exec(sort_exprs, repartition_rr); + 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",]; + 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",]; + assert_optimized!(expected_input, expected_optimized, physical_plan, false); + + Ok(()) +} + +#[tokio::test] +async fn test_pushdown_through_spm() -> Result<()> { + let schema = create_test_schema3()?; + let sort_exprs = vec![sort_expr("a", &schema), sort_expr("b", &schema)]; + let source = csv_exec_sorted(&schema, sort_exprs.clone()); + let repartition_rr = repartition_exec(source); + let spm = sort_preserving_merge_exec(sort_exprs, repartition_rr); + let physical_plan = sort_exec( + vec![ + sort_expr("a", &schema), + sort_expr("b", &schema), + sort_expr("c", &schema), + ], + spm, + ); + + 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",]; + 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",]; + assert_optimized!(expected_input, expected_optimized, physical_plan, false); + + Ok(()) +} + +#[tokio::test] +async fn test_window_multi_layer_requirement() -> Result<()> { + let schema = create_test_schema3()?; + let sort_exprs = vec![sort_expr("a", &schema), sort_expr("b", &schema)]; + let source = csv_exec_sorted(&schema, vec![]); + let sort = sort_exec(sort_exprs.clone(), source); + let repartition = repartition_exec(sort); + let repartition = spr_repartition_exec(repartition); + let spm = sort_preserving_merge_exec(sort_exprs.clone(), repartition); + + 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", + ]; + 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", + ]; + 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()?; + let input_sort_exprs = vec![sort_expr("b", &schema), sort_expr("c", &schema)]; + let parquet_input = parquet_exec_sorted(&schema, input_sort_exprs); + + let physical_plan = sort_exec( + vec![ + sort_expr("a", &schema), + sort_expr("b", &schema), + sort_expr("c", &schema), + ], + 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]", + ]; + // 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(()) +} diff --git a/datafusion/core/tests/physical_optimizer/mod.rs b/datafusion/core/tests/physical_optimizer/mod.rs index 1fac68e2505c..55f50a682b4a 100644 --- a/datafusion/core/tests/physical_optimizer/mod.rs +++ b/datafusion/core/tests/physical_optimizer/mod.rs @@ -16,6 +16,8 @@ // under the License. mod combine_partial_final_agg; +mod enforce_sorting; mod limited_distinct_aggregation; +mod replace_with_order_preserving_variants; mod sanity_checker; pub(crate) mod test_util; 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 new file mode 100644 index 000000000000..e102ab5041b7 --- /dev/null +++ b/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs @@ -0,0 +1,1270 @@ +// 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 datafusion::prelude::{SessionConfig, SessionContext}; +use datafusion_execution::TaskContext; + +use datafusion_physical_plan::coalesce_batches::CoalesceBatchesExec; +use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; +use datafusion_physical_plan::collect; +use datafusion_physical_plan::filter::FilterExec; +use datafusion_physical_plan::joins::{HashJoinExec, PartitionMode}; +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::{ + 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 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 for both bounded and +/// unbounded cases. +/// +/// # Parameters +/// +/// * `EXPECTED_UNBOUNDED_PLAN_LINES`: Expected input unbounded plan. +/// * `EXPECTED_BOUNDED_PLAN_LINES`: Expected input bounded plan. +/// * `EXPECTED_UNBOUNDED_OPTIMIZED_PLAN_LINES`: Optimized plan, which is +/// the same regardless of the value of the `prefer_existing_sort` flag. +/// * `EXPECTED_BOUNDED_OPTIMIZED_PLAN_LINES`: Optimized plan when the flag +/// `prefer_existing_sort` is `false` for bounded cases. +/// * `EXPECTED_BOUNDED_PREFER_SORT_ON_OPTIMIZED_PLAN_LINES`: Optimized plan +/// when the flag `prefer_existing_sort` is `true` for bounded cases. +/// * `$PLAN`: The plan to optimize. +/// * `$SOURCE_UNBOUNDED`: Whether the given plan contains an unbounded source. +macro_rules! assert_optimized_in_all_boundedness_situations { + ($EXPECTED_UNBOUNDED_PLAN_LINES: expr, $EXPECTED_BOUNDED_PLAN_LINES: expr, $EXPECTED_UNBOUNDED_OPTIMIZED_PLAN_LINES: expr, $EXPECTED_BOUNDED_OPTIMIZED_PLAN_LINES: expr, $EXPECTED_BOUNDED_PREFER_SORT_ON_OPTIMIZED_PLAN_LINES: expr, $PLAN: expr, $SOURCE_UNBOUNDED: expr, $PREFER_EXISTING_SORT: expr) => { + if $SOURCE_UNBOUNDED { + assert_optimized_prefer_sort_on_off!( + $EXPECTED_UNBOUNDED_PLAN_LINES, + $EXPECTED_UNBOUNDED_OPTIMIZED_PLAN_LINES, + $EXPECTED_UNBOUNDED_OPTIMIZED_PLAN_LINES, + $PLAN, + $PREFER_EXISTING_SORT, + $SOURCE_UNBOUNDED + ); + } else { + assert_optimized_prefer_sort_on_off!( + $EXPECTED_BOUNDED_PLAN_LINES, + $EXPECTED_BOUNDED_OPTIMIZED_PLAN_LINES, + $EXPECTED_BOUNDED_PREFER_SORT_ON_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. +/// +/// # 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. +/// +/// # Parameters +/// +/// * `$EXPECTED_PLAN_LINES`: Expected input plan. +/// * `$EXPECTED_OPTIMIZED_PLAN_LINES`: Expected optimized plan. +/// * `$PLAN`: The plan to optimize. +/// * `$PREFER_EXISTING_SORT`: Value of the `prefer_existing_sort` flag. +#[macro_export] +macro_rules! assert_optimized { + ($EXPECTED_PLAN_LINES: expr, $EXPECTED_OPTIMIZED_PLAN_LINES: expr, $PLAN: expr, $PREFER_EXISTING_SORT: expr, $SOURCE_UNBOUNDED: 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_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 rule top-down + let config = SessionConfig::new().with_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 optimized_physical_plan = parallel.plan; + + // 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" + ); + + if !$SOURCE_UNBOUNDED { + let ctx = SessionContext::new(); + let object_store = InMemory::new(); + object_store.put(&object_store::path::Path::from("file_path"), bytes::Bytes::from("").into()).await?; + ctx.register_object_store(&Url::parse("test://").unwrap(), Arc::new(object_store)); + let task_ctx = Arc::new(TaskContext::from(&ctx)); + let res = collect(optimized_physical_plan, task_ctx).await; + assert!( + res.is_ok(), + "Some errors occurred while executing the optimized physical plan: {:?}", res.unwrap_err() + ); + } + }; + } + +#[rstest] +#[tokio::test] +// Searches for a simple sort and a repartition just after it, the second repartition with 1 input partition should not be affected +async fn test_replace_multiple_input_repartition_1( + #[values(false, true)] source_unbounded: bool, + #[values(false, true)] prefer_existing_sort: bool, +) -> Result<()> { + let schema = create_test_schema()?; + let sort_exprs = vec![sort_expr("a", &schema)]; + let source = if source_unbounded { + stream_exec_ordered_with_projection(&schema, sort_exprs) + } else { + memory_exec_sorted(&schema, sort_exprs) + }; + let repartition = repartition_exec_hash(repartition_exec_round_robin(source)); + let sort = sort_exec(vec![sort_expr("a", &schema)], repartition, true); + + let physical_plan = sort_preserving_merge_exec(vec![sort_expr("a", &schema)], sort); + + // Expected inputs unbounded and bounded + let expected_input_unbounded = [ + "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", + " SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true]", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", + ]; + let expected_input_bounded = [ + "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", + " SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true]", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", + ]; + + // Expected unbounded result (same for with and without flag) + let expected_optimized_unbounded = [ + "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", + ]; + + // Expected bounded results with and without flag + let expected_optimized_bounded = [ + "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", + " SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true]", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", + ]; + let expected_optimized_bounded_sort_preserve = [ + "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", + ]; + assert_optimized_in_all_boundedness_situations!( + expected_input_unbounded, + expected_input_bounded, + expected_optimized_unbounded, + expected_optimized_bounded, + expected_optimized_bounded_sort_preserve, + physical_plan, + source_unbounded, + prefer_existing_sort + ); + Ok(()) +} + +#[rstest] +#[tokio::test] +async fn test_with_inter_children_change_only( + #[values(false, true)] source_unbounded: bool, + #[values(false, true)] prefer_existing_sort: bool, +) -> Result<()> { + let schema = create_test_schema()?; + let sort_exprs = vec![sort_expr_default("a", &schema)]; + let source = if source_unbounded { + stream_exec_ordered_with_projection(&schema, sort_exprs) + } else { + memory_exec_sorted(&schema, sort_exprs) + }; + let repartition_rr = repartition_exec_round_robin(source); + let repartition_hash = repartition_exec_hash(repartition_rr); + let coalesce_partitions = coalesce_partitions_exec(repartition_hash); + let sort = sort_exec( + vec![sort_expr_default("a", &coalesce_partitions.schema())], + coalesce_partitions, + false, + ); + let repartition_rr2 = repartition_exec_round_robin(sort); + let repartition_hash2 = repartition_exec_hash(repartition_rr2); + let filter = filter_exec(repartition_hash2); + let sort2 = sort_exec(vec![sort_expr_default("a", &filter.schema())], filter, true); + + let physical_plan = + sort_preserving_merge_exec(vec![sort_expr_default("a", &sort2.schema())], sort2); + + // Expected inputs unbounded and bounded + let expected_input_unbounded = [ + "SortPreservingMergeExec: [a@0 ASC]", + " SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", + " CoalescePartitionsExec", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC]", + ]; + let expected_input_bounded = [ + "SortPreservingMergeExec: [a@0 ASC]", + " SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", + " CoalescePartitionsExec", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC", + ]; + + // Expected unbounded result (same for with and without flag) + let expected_optimized_unbounded = [ + "SortPreservingMergeExec: [a@0 ASC]", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " SortPreservingMergeExec: [a@0 ASC]", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC]", + ]; + + // Expected bounded results with and without flag + let expected_optimized_bounded = [ + "SortPreservingMergeExec: [a@0 ASC]", + " SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", + " CoalescePartitionsExec", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC", + ]; + let expected_optimized_bounded_sort_preserve = [ + "SortPreservingMergeExec: [a@0 ASC]", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " SortPreservingMergeExec: [a@0 ASC]", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC", + ]; + assert_optimized_in_all_boundedness_situations!( + expected_input_unbounded, + expected_input_bounded, + expected_optimized_unbounded, + expected_optimized_bounded, + expected_optimized_bounded_sort_preserve, + physical_plan, + source_unbounded, + prefer_existing_sort + ); + Ok(()) +} + +#[rstest] +#[tokio::test] +async fn test_replace_multiple_input_repartition_2( + #[values(false, true)] source_unbounded: bool, + #[values(false, true)] prefer_existing_sort: bool, +) -> Result<()> { + let schema = create_test_schema()?; + let sort_exprs = vec![sort_expr("a", &schema)]; + let source = if source_unbounded { + stream_exec_ordered_with_projection(&schema, sort_exprs) + } else { + memory_exec_sorted(&schema, sort_exprs) + }; + let repartition_rr = repartition_exec_round_robin(source); + let filter = filter_exec(repartition_rr); + let repartition_hash = repartition_exec_hash(filter); + let sort = sort_exec(vec![sort_expr("a", &schema)], repartition_hash, true); + + let physical_plan = sort_preserving_merge_exec(vec![sort_expr("a", &schema)], sort); + + // Expected inputs unbounded and bounded + let expected_input_unbounded = [ + "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", + " SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true]", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", + ]; + let expected_input_bounded = [ + "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", + " SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true]", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", + ]; + + // Expected unbounded result (same for with and without flag) + let expected_optimized_unbounded = [ + "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", + ]; + + // Expected bounded results with and without flag + let expected_optimized_bounded = [ + "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", + " SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true]", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", + ]; + let expected_optimized_bounded_sort_preserve = [ + "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", + ]; + assert_optimized_in_all_boundedness_situations!( + expected_input_unbounded, + expected_input_bounded, + expected_optimized_unbounded, + expected_optimized_bounded, + expected_optimized_bounded_sort_preserve, + physical_plan, + source_unbounded, + prefer_existing_sort + ); + Ok(()) +} + +#[rstest] +#[tokio::test] +async fn test_replace_multiple_input_repartition_with_extra_steps( + #[values(false, true)] source_unbounded: bool, + #[values(false, true)] prefer_existing_sort: bool, +) -> Result<()> { + let schema = create_test_schema()?; + let sort_exprs = vec![sort_expr("a", &schema)]; + let source = if source_unbounded { + stream_exec_ordered_with_projection(&schema, sort_exprs) + } else { + memory_exec_sorted(&schema, sort_exprs) + }; + let repartition_rr = repartition_exec_round_robin(source); + let repartition_hash = repartition_exec_hash(repartition_rr); + let filter = filter_exec(repartition_hash); + let coalesce_batches_exec: Arc = coalesce_batches_exec(filter); + let sort = sort_exec(vec![sort_expr("a", &schema)], coalesce_batches_exec, true); + + let physical_plan = sort_preserving_merge_exec(vec![sort_expr("a", &schema)], sort); + + // Expected inputs unbounded and bounded + let expected_input_unbounded = [ + "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", + " SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true]", + " CoalesceBatchesExec: target_batch_size=8192", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", + ]; + let expected_input_bounded = [ + "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", + " SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true]", + " CoalesceBatchesExec: target_batch_size=8192", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", + ]; + + // Expected unbounded result (same for with and without flag) + let expected_optimized_unbounded = [ + "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", + " CoalesceBatchesExec: target_batch_size=8192", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", + ]; + + // Expected bounded results with and without flag + let expected_optimized_bounded = [ + "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", + " SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true]", + " CoalesceBatchesExec: target_batch_size=8192", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", + ]; + let expected_optimized_bounded_sort_preserve = [ + "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", + " CoalesceBatchesExec: target_batch_size=8192", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", + ]; + assert_optimized_in_all_boundedness_situations!( + expected_input_unbounded, + expected_input_bounded, + expected_optimized_unbounded, + expected_optimized_bounded, + expected_optimized_bounded_sort_preserve, + physical_plan, + source_unbounded, + prefer_existing_sort + ); + Ok(()) +} + +#[rstest] +#[tokio::test] +async fn test_replace_multiple_input_repartition_with_extra_steps_2( + #[values(false, true)] source_unbounded: bool, + #[values(false, true)] prefer_existing_sort: bool, +) -> Result<()> { + let schema = create_test_schema()?; + let sort_exprs = vec![sort_expr("a", &schema)]; + let source = if source_unbounded { + stream_exec_ordered_with_projection(&schema, sort_exprs) + } else { + memory_exec_sorted(&schema, sort_exprs) + }; + let repartition_rr = repartition_exec_round_robin(source); + let coalesce_batches_exec_1 = coalesce_batches_exec(repartition_rr); + let repartition_hash = repartition_exec_hash(coalesce_batches_exec_1); + let filter = filter_exec(repartition_hash); + let coalesce_batches_exec_2 = coalesce_batches_exec(filter); + let sort = sort_exec(vec![sort_expr("a", &schema)], coalesce_batches_exec_2, true); + + let physical_plan = sort_preserving_merge_exec(vec![sort_expr("a", &schema)], sort); + + // Expected inputs unbounded and bounded + let expected_input_unbounded = [ + "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", + " SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true]", + " CoalesceBatchesExec: target_batch_size=8192", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " CoalesceBatchesExec: target_batch_size=8192", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", + ]; + let expected_input_bounded = [ + "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", + " SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true]", + " CoalesceBatchesExec: target_batch_size=8192", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " CoalesceBatchesExec: target_batch_size=8192", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", + ]; + + // Expected unbounded result (same for with and without flag) + let expected_optimized_unbounded = [ + "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", + " CoalesceBatchesExec: target_batch_size=8192", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", + " CoalesceBatchesExec: target_batch_size=8192", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", + ]; + + // Expected bounded results with and without flag + let expected_optimized_bounded = [ + "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", + " SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true]", + " CoalesceBatchesExec: target_batch_size=8192", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " CoalesceBatchesExec: target_batch_size=8192", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", + ]; + let expected_optimized_bounded_sort_preserve = [ + "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", + " CoalesceBatchesExec: target_batch_size=8192", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", + " CoalesceBatchesExec: target_batch_size=8192", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", + ]; + assert_optimized_in_all_boundedness_situations!( + expected_input_unbounded, + expected_input_bounded, + expected_optimized_unbounded, + expected_optimized_bounded, + expected_optimized_bounded_sort_preserve, + physical_plan, + source_unbounded, + prefer_existing_sort + ); + Ok(()) +} + +#[rstest] +#[tokio::test] +async fn test_not_replacing_when_no_need_to_preserve_sorting( + #[values(false, true)] source_unbounded: bool, + #[values(false, true)] prefer_existing_sort: bool, +) -> Result<()> { + let schema = create_test_schema()?; + let sort_exprs = vec![sort_expr("a", &schema)]; + let source = if source_unbounded { + stream_exec_ordered_with_projection(&schema, sort_exprs) + } else { + memory_exec_sorted(&schema, sort_exprs) + }; + let repartition_rr = repartition_exec_round_robin(source); + let repartition_hash = repartition_exec_hash(repartition_rr); + let filter = filter_exec(repartition_hash); + let coalesce_batches_exec: Arc = coalesce_batches_exec(filter); + + let physical_plan: Arc = + coalesce_partitions_exec(coalesce_batches_exec); + + // Expected inputs unbounded and bounded + let expected_input_unbounded = [ + "CoalescePartitionsExec", + " CoalesceBatchesExec: target_batch_size=8192", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", + ]; + let expected_input_bounded = [ + "CoalescePartitionsExec", + " CoalesceBatchesExec: target_batch_size=8192", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", + ]; + + // Expected unbounded result (same for with and without flag) + let expected_optimized_unbounded = [ + "CoalescePartitionsExec", + " CoalesceBatchesExec: target_batch_size=8192", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", + ]; + + // Expected bounded results same with and without flag, because there is no executor with ordering requirement + let expected_optimized_bounded = [ + "CoalescePartitionsExec", + " CoalesceBatchesExec: target_batch_size=8192", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", + ]; + let expected_optimized_bounded_sort_preserve = expected_optimized_bounded; + + assert_optimized_in_all_boundedness_situations!( + expected_input_unbounded, + expected_input_bounded, + expected_optimized_unbounded, + expected_optimized_bounded, + expected_optimized_bounded_sort_preserve, + physical_plan, + source_unbounded, + prefer_existing_sort + ); + Ok(()) +} + +#[rstest] +#[tokio::test] +async fn test_with_multiple_replacable_repartitions( + #[values(false, true)] source_unbounded: bool, + #[values(false, true)] prefer_existing_sort: bool, +) -> Result<()> { + let schema = create_test_schema()?; + let sort_exprs = vec![sort_expr("a", &schema)]; + let source = if source_unbounded { + stream_exec_ordered_with_projection(&schema, sort_exprs) + } else { + memory_exec_sorted(&schema, sort_exprs) + }; + let repartition_rr = repartition_exec_round_robin(source); + let repartition_hash = repartition_exec_hash(repartition_rr); + let filter = filter_exec(repartition_hash); + let coalesce_batches = coalesce_batches_exec(filter); + let repartition_hash_2 = repartition_exec_hash(coalesce_batches); + let sort = sort_exec(vec![sort_expr("a", &schema)], repartition_hash_2, true); + + let physical_plan = sort_preserving_merge_exec(vec![sort_expr("a", &schema)], sort); + + // Expected inputs unbounded and bounded + let expected_input_unbounded = [ + "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", + " SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true]", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " CoalesceBatchesExec: target_batch_size=8192", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", + ]; + let expected_input_bounded = [ + "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", + " SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true]", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " CoalesceBatchesExec: target_batch_size=8192", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", + ]; + + // Expected unbounded result (same for with and without flag) + let expected_optimized_unbounded = [ + "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", + " CoalesceBatchesExec: target_batch_size=8192", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", + ]; + + // Expected bounded results with and without flag + let expected_optimized_bounded = [ + "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", + " SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[true]", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " CoalesceBatchesExec: target_batch_size=8192", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", + ]; + let expected_optimized_bounded_sort_preserve = [ + "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", + " CoalesceBatchesExec: target_batch_size=8192", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", + ]; + assert_optimized_in_all_boundedness_situations!( + expected_input_unbounded, + expected_input_bounded, + expected_optimized_unbounded, + expected_optimized_bounded, + expected_optimized_bounded_sort_preserve, + physical_plan, + source_unbounded, + prefer_existing_sort + ); + Ok(()) +} + +#[rstest] +#[tokio::test] +async fn test_not_replace_with_different_orderings( + #[values(false, true)] source_unbounded: bool, + #[values(false, true)] prefer_existing_sort: bool, +) -> Result<()> { + let schema = create_test_schema()?; + let sort_exprs = vec![sort_expr("a", &schema)]; + let source = if source_unbounded { + stream_exec_ordered_with_projection(&schema, sort_exprs) + } else { + memory_exec_sorted(&schema, sort_exprs) + }; + let repartition_rr = repartition_exec_round_robin(source); + let repartition_hash = repartition_exec_hash(repartition_rr); + let sort = sort_exec( + vec![sort_expr_default("c", &repartition_hash.schema())], + repartition_hash, + true, + ); + + let physical_plan = + sort_preserving_merge_exec(vec![sort_expr_default("c", &sort.schema())], sort); + + // Expected inputs unbounded and bounded + let expected_input_unbounded = [ + "SortPreservingMergeExec: [c@1 ASC]", + " SortExec: expr=[c@1 ASC], preserve_partitioning=[true]", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", + ]; + let expected_input_bounded = [ + "SortPreservingMergeExec: [c@1 ASC]", + " SortExec: expr=[c@1 ASC], preserve_partitioning=[true]", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", + ]; + + // Expected unbounded result (same for with and without flag) + let expected_optimized_unbounded = [ + "SortPreservingMergeExec: [c@1 ASC]", + " SortExec: expr=[c@1 ASC], preserve_partitioning=[true]", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", + ]; + + // Expected bounded results same with and without flag, because ordering requirement of the executor is different than the existing ordering. + let expected_optimized_bounded = [ + "SortPreservingMergeExec: [c@1 ASC]", + " SortExec: expr=[c@1 ASC], preserve_partitioning=[true]", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", + ]; + let expected_optimized_bounded_sort_preserve = expected_optimized_bounded; + + assert_optimized_in_all_boundedness_situations!( + expected_input_unbounded, + expected_input_bounded, + expected_optimized_unbounded, + expected_optimized_bounded, + expected_optimized_bounded_sort_preserve, + physical_plan, + source_unbounded, + prefer_existing_sort + ); + Ok(()) +} + +#[rstest] +#[tokio::test] +async fn test_with_lost_ordering( + #[values(false, true)] source_unbounded: bool, + #[values(false, true)] prefer_existing_sort: bool, +) -> Result<()> { + let schema = create_test_schema()?; + let sort_exprs = vec![sort_expr("a", &schema)]; + let source = if source_unbounded { + stream_exec_ordered_with_projection(&schema, sort_exprs) + } else { + memory_exec_sorted(&schema, sort_exprs) + }; + let repartition_rr = repartition_exec_round_robin(source); + let repartition_hash = repartition_exec_hash(repartition_rr); + let coalesce_partitions = coalesce_partitions_exec(repartition_hash); + let physical_plan = + sort_exec(vec![sort_expr("a", &schema)], coalesce_partitions, false); + + // Expected inputs unbounded and bounded + let expected_input_unbounded = [ + "SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[false]", + " CoalescePartitionsExec", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", + ]; + let expected_input_bounded = [ + "SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[false]", + " CoalescePartitionsExec", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", + ]; + + // Expected unbounded result (same for with and without flag) + let expected_optimized_unbounded = [ + "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", + ]; + + // Expected bounded results with and without flag + let expected_optimized_bounded = [ + "SortExec: expr=[a@0 ASC NULLS LAST], preserve_partitioning=[false]", + " CoalescePartitionsExec", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", + ]; + let expected_optimized_bounded_sort_preserve = [ + "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=a@0 ASC NULLS LAST", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", + ]; + assert_optimized_in_all_boundedness_situations!( + expected_input_unbounded, + expected_input_bounded, + expected_optimized_unbounded, + expected_optimized_bounded, + expected_optimized_bounded_sort_preserve, + physical_plan, + source_unbounded, + prefer_existing_sort + ); + Ok(()) +} + +#[rstest] +#[tokio::test] +async fn test_with_lost_and_kept_ordering( + #[values(false, true)] source_unbounded: bool, + #[values(false, true)] prefer_existing_sort: bool, +) -> Result<()> { + let schema = create_test_schema()?; + let sort_exprs = vec![sort_expr("a", &schema)]; + let source = if source_unbounded { + stream_exec_ordered_with_projection(&schema, sort_exprs) + } else { + memory_exec_sorted(&schema, sort_exprs) + }; + let repartition_rr = repartition_exec_round_robin(source); + let repartition_hash = repartition_exec_hash(repartition_rr); + let coalesce_partitions = coalesce_partitions_exec(repartition_hash); + let sort = sort_exec( + vec![sort_expr_default("c", &coalesce_partitions.schema())], + coalesce_partitions, + false, + ); + let repartition_rr2 = repartition_exec_round_robin(sort); + let repartition_hash2 = repartition_exec_hash(repartition_rr2); + let filter = filter_exec(repartition_hash2); + let sort2 = sort_exec(vec![sort_expr_default("c", &filter.schema())], filter, true); + + let physical_plan = + sort_preserving_merge_exec(vec![sort_expr_default("c", &sort2.schema())], sort2); + + // Expected inputs unbounded and bounded + let expected_input_unbounded = [ + "SortPreservingMergeExec: [c@1 ASC]", + " SortExec: expr=[c@1 ASC], preserve_partitioning=[true]", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " SortExec: expr=[c@1 ASC], preserve_partitioning=[false]", + " CoalescePartitionsExec", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", + ]; + let expected_input_bounded = [ + "SortPreservingMergeExec: [c@1 ASC]", + " SortExec: expr=[c@1 ASC], preserve_partitioning=[true]", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " SortExec: expr=[c@1 ASC], preserve_partitioning=[false]", + " CoalescePartitionsExec", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", + ]; + + // Expected unbounded result (same for with and without flag) + let expected_optimized_unbounded = [ + "SortPreservingMergeExec: [c@1 ASC]", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=c@1 ASC", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " SortExec: expr=[c@1 ASC], preserve_partitioning=[false]", + " CoalescePartitionsExec", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", + ]; + + // Expected bounded results with and without flag + let expected_optimized_bounded = [ + "SortPreservingMergeExec: [c@1 ASC]", + " SortExec: expr=[c@1 ASC], preserve_partitioning=[true]", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " SortExec: expr=[c@1 ASC], preserve_partitioning=[false]", + " CoalescePartitionsExec", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", + ]; + let expected_optimized_bounded_sort_preserve = [ + "SortPreservingMergeExec: [c@1 ASC]", + " FilterExec: c@1 > 3", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8, preserve_order=true, sort_exprs=c@1 ASC", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " SortExec: expr=[c@1 ASC], preserve_partitioning=[false]", + " CoalescePartitionsExec", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", + ]; + assert_optimized_in_all_boundedness_situations!( + expected_input_unbounded, + expected_input_bounded, + expected_optimized_unbounded, + expected_optimized_bounded, + expected_optimized_bounded_sort_preserve, + physical_plan, + source_unbounded, + prefer_existing_sort + ); + Ok(()) +} + +#[rstest] +#[tokio::test] +async fn test_with_multiple_child_trees( + #[values(false, true)] source_unbounded: bool, + #[values(false, true)] prefer_existing_sort: bool, +) -> Result<()> { + let schema = create_test_schema()?; + + let left_sort_exprs = vec![sort_expr("a", &schema)]; + let left_source = if source_unbounded { + stream_exec_ordered_with_projection(&schema, left_sort_exprs) + } else { + memory_exec_sorted(&schema, left_sort_exprs) + }; + let left_repartition_rr = repartition_exec_round_robin(left_source); + let left_repartition_hash = repartition_exec_hash(left_repartition_rr); + let left_coalesce_partitions = + Arc::new(CoalesceBatchesExec::new(left_repartition_hash, 4096)); + + let right_sort_exprs = vec![sort_expr("a", &schema)]; + let right_source = if source_unbounded { + stream_exec_ordered_with_projection(&schema, right_sort_exprs) + } else { + memory_exec_sorted(&schema, right_sort_exprs) + }; + let right_repartition_rr = repartition_exec_round_robin(right_source); + let right_repartition_hash = repartition_exec_hash(right_repartition_rr); + let right_coalesce_partitions = + Arc::new(CoalesceBatchesExec::new(right_repartition_hash, 4096)); + + let hash_join_exec = + hash_join_exec(left_coalesce_partitions, right_coalesce_partitions); + let sort = sort_exec( + vec![sort_expr_default("a", &hash_join_exec.schema())], + hash_join_exec, + true, + ); + + let physical_plan = + sort_preserving_merge_exec(vec![sort_expr_default("a", &sort.schema())], sort); + + // Expected inputs unbounded and bounded + let expected_input_unbounded = [ + "SortPreservingMergeExec: [a@0 ASC]", + " SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", + " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@1, c@1)]", + " CoalesceBatchesExec: target_batch_size=4096", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", + " CoalesceBatchesExec: target_batch_size=4096", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", + ]; + let expected_input_bounded = [ + "SortPreservingMergeExec: [a@0 ASC]", + " SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", + " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@1, c@1)]", + " CoalesceBatchesExec: target_batch_size=4096", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", + " CoalesceBatchesExec: target_batch_size=4096", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", + ]; + + // Expected unbounded result (same for with and without flag) + let expected_optimized_unbounded = [ + "SortPreservingMergeExec: [a@0 ASC]", + " SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", + " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@1, c@1)]", + " CoalesceBatchesExec: target_batch_size=4096", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", + " CoalesceBatchesExec: target_batch_size=4096", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " StreamingTableExec: partition_sizes=1, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST]", + ]; + + // Expected bounded results same with and without flag, because ordering get lost during intermediate executor anyway. Hence no need to preserve + // existing ordering. + let expected_optimized_bounded = [ + "SortPreservingMergeExec: [a@0 ASC]", + " SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", + " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@1, c@1)]", + " CoalesceBatchesExec: target_batch_size=4096", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", + " CoalesceBatchesExec: target_batch_size=4096", + " RepartitionExec: partitioning=Hash([c@1], 8), input_partitions=8", + " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", + " MemoryExec: partitions=1, partition_sizes=[1], output_ordering=a@0 ASC NULLS LAST", + ]; + let expected_optimized_bounded_sort_preserve = expected_optimized_bounded; + + assert_optimized_in_all_boundedness_situations!( + expected_input_unbounded, + expected_input_bounded, + expected_optimized_unbounded, + expected_optimized_bounded, + expected_optimized_bounded_sort_preserve, + physical_plan, + source_unbounded, + prefer_existing_sort + ); + Ok(()) +} + +// End test cases +// Start test helpers + +fn sort_expr(name: &str, schema: &Schema) -> PhysicalSortExpr { + let sort_opts = SortOptions { + nulls_first: false, + descending: false, + }; + sort_expr_options(name, schema, sort_opts) +} + +fn sort_expr_default(name: &str, schema: &Schema) -> PhysicalSortExpr { + let sort_opts = SortOptions::default(); + sort_expr_options(name, schema, sort_opts) +} + +fn sort_expr_options( + name: &str, + schema: &Schema, + options: SortOptions, +) -> PhysicalSortExpr { + PhysicalSortExpr { + expr: col(name, schema).unwrap(), + options, + } +} + +fn sort_exec( + sort_exprs: impl IntoIterator, + input: Arc, + preserve_partitioning: bool, +) -> Arc { + let sort_exprs = sort_exprs.into_iter().collect(); + Arc::new( + SortExec::new(sort_exprs, input) + .with_preserve_partitioning(preserve_partitioning), + ) +} + +fn sort_preserving_merge_exec( + sort_exprs: impl IntoIterator, + input: Arc, +) -> Arc { + let sort_exprs = sort_exprs.into_iter().collect(); + Arc::new(SortPreservingMergeExec::new(sort_exprs, input)) +} + +fn repartition_exec_round_robin(input: Arc) -> Arc { + Arc::new(RepartitionExec::try_new(input, Partitioning::RoundRobinBatch(8)).unwrap()) +} + +fn repartition_exec_hash(input: Arc) -> Arc { + let input_schema = input.schema(); + Arc::new( + RepartitionExec::try_new( + input, + Partitioning::Hash(vec![col("c", &input_schema).unwrap()], 8), + ) + .unwrap(), + ) +} + +fn filter_exec(input: Arc) -> Arc { + let input_schema = input.schema(); + let predicate = expressions::binary( + col("c", &input_schema).unwrap(), + Operator::Gt, + expressions::lit(3i32), + &input_schema, + ) + .unwrap(); + Arc::new(FilterExec::try_new(predicate, input).unwrap()) +} + +fn coalesce_batches_exec(input: Arc) -> Arc { + Arc::new(CoalesceBatchesExec::new(input, 8192)) +} + +fn coalesce_partitions_exec(input: Arc) -> Arc { + Arc::new(CoalescePartitionsExec::new(input)) +} + +fn hash_join_exec( + left: Arc, + right: Arc, +) -> Arc { + let left_on = col("c", &left.schema()).unwrap(); + let right_on = col("c", &right.schema()).unwrap(); + let left_col = left_on.as_any().downcast_ref::().unwrap(); + let right_col = right_on.as_any().downcast_ref::().unwrap(); + Arc::new( + HashJoinExec::try_new( + left, + right, + vec![(Arc::new(left_col.clone()), Arc::new(right_col.clone()))], + None, + &JoinType::Inner, + None, + PartitionMode::Partitioned, + false, + ) + .unwrap(), + ) +} + +fn create_test_schema() -> Result { + let column_a = Field::new("a", DataType::Int32, false); + let column_b = Field::new("b", DataType::Int32, false); + let column_c = Field::new("c", DataType::Int32, false); + let column_d = Field::new("d", DataType::Int32, false); + let schema = Arc::new(Schema::new(vec![column_a, column_b, column_c, column_d])); + + Ok(schema) +} + +// creates a memory exec source for the test purposes +// projection parameter is given static due to testing needs +fn memory_exec_sorted( + schema: &SchemaRef, + sort_exprs: impl IntoIterator, +) -> Arc { + pub fn make_partition(schema: &SchemaRef, sz: i32) -> RecordBatch { + let values = (0..sz).collect::>(); + let arr = Arc::new(Int32Array::from(values)); + let arr = arr as ArrayRef; + + RecordBatch::try_new( + schema.clone(), + vec![arr.clone(), arr.clone(), arr.clone(), arr], + ) + .unwrap() + } + + let rows = 5; + let partitions = 1; + let sort_exprs = sort_exprs.into_iter().collect(); + Arc::new({ + let data: Vec> = (0..partitions) + .map(|_| vec![make_partition(schema, rows)]) + .collect(); + let projection: Vec = vec![0, 2, 3]; + MemoryExec::try_new(&data, schema.clone(), Some(projection)) + .unwrap() + .try_with_sort_information(vec![sort_exprs]) + .unwrap() + }) +} diff --git a/datafusion/core/tests/physical_optimizer/test_util.rs b/datafusion/core/tests/physical_optimizer/test_util.rs index ea4b80a7899c..3c3da36199b1 100644 --- a/datafusion/core/tests/physical_optimizer/test_util.rs +++ b/datafusion/core/tests/physical_optimizer/test_util.rs @@ -25,12 +25,17 @@ 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_physical_expr_common::sort_expr::LexOrdering; +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( @@ -174,3 +179,141 @@ impl QueryCase { 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 40074e8eecd8..c6c75a0e0939 100644 --- a/datafusion/physical-optimizer/Cargo.toml +++ b/datafusion/physical-optimizer/Cargo.toml @@ -48,6 +48,7 @@ futures = { workspace = true } itertools = { workspace = true } log = { workspace = true } recursive = { workspace = true, optional = true } +url = { workspace = true } [dev-dependencies] datafusion-expr = { workspace = true } diff --git a/datafusion/physical-optimizer/src/enforce_sorting/mod.rs b/datafusion/physical-optimizer/src/enforce_sorting/mod.rs new file mode 100644 index 000000000000..6853c3467b6c --- /dev/null +++ b/datafusion/physical-optimizer/src/enforce_sorting/mod.rs @@ -0,0 +1,657 @@ +// 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. + +//! EnforceSorting optimizer rule inspects the physical plan with respect +//! to local sorting requirements and does the following: +//! - Adds a [`SortExec`] when a requirement is not met, +//! - Removes an already-existing [`SortExec`] if it is possible to prove +//! that this sort is unnecessary +//! +//! The rule can work on valid *and* invalid physical plans with respect to +//! sorting requirements, but always produces a valid physical plan in this sense. +//! +//! A non-realistic but easy to follow example for sort removals: Assume that we +//! somehow get the fragment +//! +//! ```text +//! SortExec: expr=[nullable_col@0 ASC] +//! SortExec: expr=[non_nullable_col@1 ASC] +//! ``` +//! +//! in the physical plan. The first sort is unnecessary since its result is overwritten +//! by another [`SortExec`]. Therefore, this rule removes it from the physical plan. + +pub mod replace_with_order_preserving_variants; +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, +}; +use crate::enforce_sorting::sort_pushdown::{ + assign_initial_requirements, pushdown_sorts, SortPushDown, +}; +use crate::utils::{ + add_sort_above, add_sort_above_with_check, is_coalesce_partitions, is_limit, + is_repartition, is_sort, is_sort_preserving_merge, is_union, is_window, +}; +use crate::PhysicalOptimizerRule; +use datafusion_common::config::ConfigOptions; +use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; +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 itertools::izip; + +/// This rule inspects [`SortExec`]'s in the given physical plan and removes the +/// ones it can prove unnecessary. +#[derive(Default, Debug)] +pub struct EnforceSorting {} + +impl EnforceSorting { + #[allow(missing_docs)] + pub fn new() -> Self { + Self {} + } +} + +/// This object is used within the [`EnforceSorting`] rule to track the closest +/// [`SortExec`] descendant(s) for every child of a plan. The data attribute +/// stores whether the plan is a `SortExec` or is connected to a `SortExec` +/// via its children. +pub type PlanWithCorrespondingSort = PlanContext; + +fn update_sort_ctx_children( + mut node: PlanWithCorrespondingSort, + data: bool, +) -> Result { + for child_node in node.children.iter_mut() { + let plan = &child_node.plan; + child_node.data = if is_sort(plan) { + // Initiate connection: + true + } else if is_limit(plan) { + // There is no sort linkage for this path, it starts at a limit. + false + } else { + let is_spm = is_sort_preserving_merge(plan); + let required_orderings = plan.required_input_ordering(); + let flags = plan.maintains_input_order(); + // Add parent node to the tree if there is at least one child with + // a sort connection: + izip!(flags, required_orderings).any(|(maintains, required_ordering)| { + let propagates_ordering = + (maintains && required_ordering.is_none()) || is_spm; + let connected_to_sort = + child_node.children.iter().any(|child| child.data); + propagates_ordering && connected_to_sort + }) + } + } + + node.data = data; + node.update_plan_from_children() +} + +/// This object is used within the [`EnforceSorting`] rule to track the closest +/// [`CoalescePartitionsExec`] descendant(s) for every child of a plan. The data +/// attribute stores whether the plan is a `CoalescePartitionsExec` or is +/// connected to a `CoalescePartitionsExec` via its children. +pub type PlanWithCorrespondingCoalescePartitions = PlanContext; + +fn update_coalesce_ctx_children( + coalesce_context: &mut PlanWithCorrespondingCoalescePartitions, +) { + let children = &coalesce_context.children; + coalesce_context.data = if children.is_empty() { + // Plan has no children, it cannot be a `CoalescePartitionsExec`. + false + } else if is_coalesce_partitions(&coalesce_context.plan) { + // Initiate a connection: + true + } else { + children.iter().enumerate().any(|(idx, node)| { + // Only consider operators that don't require a single partition, + // and connected to some `CoalescePartitionsExec`: + node.data + && !matches!( + coalesce_context.plan.required_input_distribution()[idx], + Distribution::SinglePartition + ) + }) + }; +} + +/// The boolean flag `repartition_sorts` defined in the config indicates +/// whether we elect to transform [`CoalescePartitionsExec`] + [`SortExec`] cascades +/// into [`SortExec`] + [`SortPreservingMergeExec`] cascades, which enables us to +/// perform sorting in parallel. +impl PhysicalOptimizerRule for EnforceSorting { + fn optimize( + &self, + plan: Arc, + config: &ConfigOptions, + ) -> Result> { + let plan_requirements = PlanWithCorrespondingSort::new_default(plan); + // Execute a bottom-up traversal to enforce sorting requirements, + // remove unnecessary sorts, and optimize sort-sensitive operators: + let adjusted = plan_requirements.transform_up(ensure_sorting)?.data; + 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()?; + 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()?; + // Execute a top-down traversal to exploit sort push-down opportunities + // missed by the bottom-up traversal: + let mut sort_pushdown = SortPushDown::new_default(updated_plan.plan); + assign_initial_requirements(&mut sort_pushdown); + let adjusted = pushdown_sorts(sort_pushdown)?; + adjusted + .plan + .transform_up(|plan| Ok(Transformed::yes(replace_with_partial_sort(plan)?))) + .data() + } + + fn name(&self) -> &str { + "EnforceSorting" + } + + fn schema_check(&self) -> bool { + true + } +} + +fn replace_with_partial_sort( + plan: Arc, +) -> Result> { + let plan_any = plan.as_any(); + if let Some(sort_plan) = plan_any.downcast_ref::() { + let child = Arc::clone(sort_plan.children()[0]); + if !child.boundedness().is_unbounded() { + return Ok(plan); + } + + // here we're trying to find the common prefix for sorted columns that is required for the + // sort and already satisfied by the given ordering + let child_eq_properties = child.equivalence_properties(); + let sort_req = LexRequirement::from(sort_plan.expr().clone()); + + let mut common_prefix_length = 0; + while child_eq_properties.ordering_satisfy_requirement(&LexRequirement { + inner: sort_req[0..common_prefix_length + 1].to_vec(), + }) { + common_prefix_length += 1; + } + if common_prefix_length > 0 { + return Ok(Arc::new( + PartialSortExec::new( + LexOrdering::new(sort_plan.expr().to_vec()), + Arc::clone(sort_plan.input()), + common_prefix_length, + ) + .with_preserve_partitioning(sort_plan.preserve_partitioning()) + .with_fetch(sort_plan.fetch()), + )); + } + } + Ok(plan) +} + +/// This function turns plans of the form +/// ```text +/// "SortExec: expr=\[a@0 ASC\]", +/// " CoalescePartitionsExec", +/// " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", +/// ``` +/// to +/// ```text +/// "SortPreservingMergeExec: \[a@0 ASC\]", +/// " SortExec: expr=\[a@0 ASC\]", +/// " RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1", +/// ``` +/// by following connections from [`CoalescePartitionsExec`]s to [`SortExec`]s. +/// By performing sorting in parallel, we can increase performance in some scenarios. +pub fn parallelize_sorts( + mut requirements: PlanWithCorrespondingCoalescePartitions, +) -> Result> { + update_coalesce_ctx_children(&mut requirements); + + if requirements.children.is_empty() || !requirements.children[0].data { + // We only take an action when the plan is either a `SortExec`, a + // `SortPreservingMergeExec` or a `CoalescePartitionsExec`, and they + // all have a single child. Therefore, if the first child has no + // connection, we can return immediately. + Ok(Transformed::no(requirements)) + } else if (is_sort(&requirements.plan) + || is_sort_preserving_merge(&requirements.plan)) + && requirements.plan.output_partitioning().partition_count() <= 1 + { + // Take the initial sort expressions and requirements + let (sort_exprs, fetch) = get_sort_exprs(&requirements.plan)?; + let sort_reqs = LexRequirement::from(sort_exprs.clone()); + let sort_exprs = sort_exprs.clone(); + + // If there is a connection between a `CoalescePartitionsExec` and a + // global sort that satisfy the requirements (i.e. intermediate + // executors don't require single partition), then we can replace + // the `CoalescePartitionsExec` + `SortExec` cascade with a `SortExec` + // + `SortPreservingMergeExec` cascade to parallelize sorting. + requirements = remove_bottleneck_in_subplan(requirements)?; + // We also need to remove the self node since `remove_corresponding_coalesce_in_sub_plan` + // deals with the children and their children and so on. + requirements = requirements.children.swap_remove(0); + + requirements = add_sort_above_with_check(requirements, sort_reqs, fetch); + + let spm = + SortPreservingMergeExec::new(sort_exprs, Arc::clone(&requirements.plan)); + Ok(Transformed::yes( + PlanWithCorrespondingCoalescePartitions::new( + Arc::new(spm.with_fetch(fetch)), + false, + vec![requirements], + ), + )) + } else if is_coalesce_partitions(&requirements.plan) { + // There is an unnecessary `CoalescePartitionsExec` in the plan. + // This will handle the recursive `CoalescePartitionsExec` plans. + requirements = remove_bottleneck_in_subplan(requirements)?; + // For the removal of self node which is also a `CoalescePartitionsExec`. + requirements = requirements.children.swap_remove(0); + + Ok(Transformed::yes( + PlanWithCorrespondingCoalescePartitions::new( + Arc::new(CoalescePartitionsExec::new(Arc::clone(&requirements.plan))), + false, + vec![requirements], + ), + )) + } else { + Ok(Transformed::yes(requirements)) + } +} + +/// This function enforces sorting requirements and makes optimizations without +/// violating these requirements whenever possible. +pub fn ensure_sorting( + mut requirements: PlanWithCorrespondingSort, +) -> Result> { + requirements = update_sort_ctx_children(requirements, false)?; + + // Perform naive analysis at the beginning -- remove already-satisfied sorts: + if requirements.children.is_empty() { + return Ok(Transformed::no(requirements)); + } + let maybe_requirements = analyze_immediate_sort_removal(requirements); + requirements = if !maybe_requirements.transformed { + maybe_requirements.data + } else { + return Ok(maybe_requirements); + }; + + let plan = &requirements.plan; + let mut updated_children = vec![]; + for (idx, (required_ordering, mut child)) in plan + .required_input_ordering() + .into_iter() + .zip(requirements.children.into_iter()) + .enumerate() + { + let physical_ordering = child.plan.output_ordering(); + + if let Some(required) = required_ordering { + let eq_properties = child.plan.equivalence_properties(); + if !eq_properties.ordering_satisfy_requirement(&required) { + // Make sure we preserve the ordering requirements: + if physical_ordering.is_some() { + child = update_child_to_remove_unnecessary_sort(idx, child, plan)?; + } + child = add_sort_above(child, required, None); + child = update_sort_ctx_children(child, true)?; + } + } else if physical_ordering.is_none() + || !plan.maintains_input_order()[idx] + || is_union(plan) + { + // We have a `SortExec` whose effect may be neutralized by another + // order-imposing operator, remove this sort: + child = update_child_to_remove_unnecessary_sort(idx, child, plan)?; + } + updated_children.push(child); + } + requirements.children = updated_children; + // For window expressions, we can remove some sorts when we can + // calculate the result in reverse: + let child_node = &requirements.children[0]; + if is_window(plan) && child_node.data { + return adjust_window_sort_removal(requirements).map(Transformed::yes); + } else if is_sort_preserving_merge(plan) + && child_node.plan.output_partitioning().partition_count() <= 1 + { + // This `SortPreservingMergeExec` is unnecessary, input already has a + // single partition. + let child_node = requirements.children.swap_remove(0); + return Ok(Transformed::yes(child_node)); + } + + update_sort_ctx_children(requirements, false).map(Transformed::yes) +} + +/// Analyzes a given [`SortExec`] (`plan`) to determine whether its input +/// already has a finer ordering than it enforces. +fn analyze_immediate_sort_removal( + mut node: PlanWithCorrespondingSort, +) -> Transformed { + if let Some(sort_exec) = node.plan.as_any().downcast_ref::() { + let sort_input = sort_exec.input(); + // If this sort is unnecessary, we should remove it: + if sort_input.equivalence_properties().ordering_satisfy( + sort_exec + .properties() + .output_ordering() + .unwrap_or(LexOrdering::empty()), + ) { + node.plan = if !sort_exec.preserve_partitioning() + && sort_input.output_partitioning().partition_count() > 1 + { + // Replace the sort with a sort-preserving merge: + let expr = LexOrdering::new(sort_exec.expr().to_vec()); + Arc::new( + SortPreservingMergeExec::new(expr, Arc::clone(sort_input)) + .with_fetch(sort_exec.fetch()), + ) as _ + } else { + // Remove the sort: + node.children = node.children.swap_remove(0).children; + if let Some(fetch) = sort_exec.fetch() { + // If the sort has a fetch, we need to add a limit: + if sort_exec + .properties() + .output_partitioning() + .partition_count() + == 1 + { + Arc::new(GlobalLimitExec::new( + Arc::clone(sort_input), + 0, + Some(fetch), + )) + } else { + Arc::new(LocalLimitExec::new(Arc::clone(sort_input), fetch)) + } + } else { + Arc::clone(sort_input) + } + }; + for child in node.children.iter_mut() { + child.data = false; + } + node.data = false; + return Transformed::yes(node); + } + } + Transformed::no(node) +} + +/// Adjusts a [`WindowAggExec`] or a [`BoundedWindowAggExec`] to determine +/// whether it may allow removing a sort. +fn adjust_window_sort_removal( + mut window_tree: PlanWithCorrespondingSort, +) -> Result { + // Window operators have a single child we need to adjust: + let child_node = remove_corresponding_sort_from_sub_plan( + window_tree.children.swap_remove(0), + matches!( + window_tree.plan.required_input_distribution()[0], + Distribution::SinglePartition + ), + )?; + window_tree.children.push(child_node); + + let plan = window_tree.plan.as_any(); + let child_plan = &window_tree.children[0].plan; + let (window_expr, new_window) = + if let Some(exec) = plan.downcast_ref::() { + let window_expr = exec.window_expr(); + let new_window = + get_best_fitting_window(window_expr, child_plan, &exec.partition_keys)?; + (window_expr, new_window) + } else if let Some(exec) = plan.downcast_ref::() { + let window_expr = exec.window_expr(); + let new_window = + get_best_fitting_window(window_expr, child_plan, &exec.partition_keys)?; + (window_expr, new_window) + } else { + return plan_err!("Expected WindowAggExec or BoundedWindowAggExec"); + }; + + window_tree.plan = if let Some(new_window) = new_window { + // We were able to change the window to accommodate the input, use it: + new_window + } else { + // We were unable to change the window to accommodate the input, so we + // will insert a sort. + let reqs = window_tree + .plan + .required_input_ordering() + .swap_remove(0) + .unwrap_or_default(); + + // Satisfy the ordering requirement so that the window can run: + let mut child_node = window_tree.children.swap_remove(0); + child_node = add_sort_above(child_node, reqs, None); + let child_plan = Arc::clone(&child_node.plan); + window_tree.children.push(child_node); + + if window_expr.iter().all(|e| e.uses_bounded_memory()) { + Arc::new(BoundedWindowAggExec::try_new( + window_expr.to_vec(), + child_plan, + window_expr[0].partition_by().to_vec(), + InputOrderMode::Sorted, + )?) as _ + } else { + Arc::new(WindowAggExec::try_new( + window_expr.to_vec(), + child_plan, + window_expr[0].partition_by().to_vec(), + )?) as _ + } + }; + + window_tree.data = false; + Ok(window_tree) +} + +/// Removes parallelization-reducing, avoidable [`CoalescePartitionsExec`]s from +/// the plan in `node`. After the removal of such `CoalescePartitionsExec`s from +/// the plan, some of the remaining `RepartitionExec`s might become unnecessary. +/// Removes such `RepartitionExec`s from the plan as well. +fn remove_bottleneck_in_subplan( + mut requirements: PlanWithCorrespondingCoalescePartitions, +) -> Result { + let plan = &requirements.plan; + let children = &mut requirements.children; + if is_coalesce_partitions(&children[0].plan) { + // We can safely use the 0th index since we have a `CoalescePartitionsExec`. + let mut new_child_node = children[0].children.swap_remove(0); + while new_child_node.plan.output_partitioning() == plan.output_partitioning() + && is_repartition(&new_child_node.plan) + && is_repartition(plan) + { + new_child_node = new_child_node.children.swap_remove(0) + } + children[0] = new_child_node; + } else { + requirements.children = requirements + .children + .into_iter() + .map(|node| { + if node.data { + remove_bottleneck_in_subplan(node) + } else { + Ok(node) + } + }) + .collect::>()?; + } + let mut new_reqs = requirements.update_plan_from_children()?; + if let Some(repartition) = new_reqs.plan.as_any().downcast_ref::() { + let input_partitioning = repartition.input().output_partitioning(); + // We can remove this repartitioning operator if it is now a no-op: + let mut can_remove = input_partitioning.eq(repartition.partitioning()); + // We can also remove it if we ended up with an ineffective RR: + if let Partitioning::RoundRobinBatch(n_out) = repartition.partitioning() { + can_remove |= *n_out == input_partitioning.partition_count(); + } + if can_remove { + new_reqs = new_reqs.children.swap_remove(0) + } + } + Ok(new_reqs) +} + +/// Updates child to remove the unnecessary sort below it. +fn update_child_to_remove_unnecessary_sort( + child_idx: usize, + mut node: PlanWithCorrespondingSort, + parent: &Arc, +) -> Result { + if node.data { + let requires_single_partition = matches!( + parent.required_input_distribution()[child_idx], + Distribution::SinglePartition + ); + node = remove_corresponding_sort_from_sub_plan(node, requires_single_partition)?; + } + node.data = false; + Ok(node) +} + +/// Removes the sort from the plan in `node`. +fn remove_corresponding_sort_from_sub_plan( + mut node: PlanWithCorrespondingSort, + requires_single_partition: bool, +) -> Result { + // A `SortExec` is always at the bottom of the tree. + if let Some(sort_exec) = node.plan.as_any().downcast_ref::() { + // Do not remove sorts with fetch: + if sort_exec.fetch().is_none() { + node = node.children.swap_remove(0); + } + } else { + let mut any_connection = false; + let required_dist = node.plan.required_input_distribution(); + node.children = node + .children + .into_iter() + .enumerate() + .map(|(idx, child)| { + if child.data { + any_connection = true; + remove_corresponding_sort_from_sub_plan( + child, + matches!(required_dist[idx], Distribution::SinglePartition), + ) + } else { + Ok(child) + } + }) + .collect::>()?; + if any_connection || node.children.is_empty() { + node = update_sort_ctx_children(node, false)?; + } + + // Replace with variants that do not preserve order. + if is_sort_preserving_merge(&node.plan) { + node.children = node.children.swap_remove(0).children; + node.plan = Arc::clone(node.plan.children().swap_remove(0)); + } else if let Some(repartition) = + node.plan.as_any().downcast_ref::() + { + node.plan = Arc::new(RepartitionExec::try_new( + Arc::clone(&node.children[0].plan), + repartition.properties().output_partitioning().clone(), + )?) as _; + } + }; + // Deleting a merging sort may invalidate distribution requirements. + // Ensure that we stay compliant with such requirements: + if requires_single_partition && node.plan.output_partitioning().partition_count() > 1 + { + // If there is existing ordering, to preserve ordering use + // `SortPreservingMergeExec` instead of a `CoalescePartitionsExec`. + let plan = Arc::clone(&node.plan); + let fetch = plan.fetch(); + let plan = if let Some(ordering) = plan.output_ordering() { + Arc::new( + SortPreservingMergeExec::new(LexOrdering::new(ordering.to_vec()), plan) + .with_fetch(fetch), + ) as _ + } else { + Arc::new(CoalescePartitionsExec::new(plan)) as _ + }; + node = PlanWithCorrespondingSort::new(plan, false, vec![node]); + node = update_sort_ctx_children(node, false)?; + } + Ok(node) +} + +/// Converts an [ExecutionPlan] trait object to a [LexOrdering] reference when possible. +fn get_sort_exprs( + sort_any: &Arc, +) -> Result<(&LexOrdering, Option)> { + if let Some(sort_exec) = sort_any.as_any().downcast_ref::() { + Ok((sort_exec.expr(), sort_exec.fetch())) + } else if let Some(spm) = sort_any.as_any().downcast_ref::() + { + Ok((spm.expr(), spm.fetch())) + } else { + plan_err!("Given ExecutionPlan is not a SortExec or a SortPreservingMergeExec") + } +} 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 new file mode 100644 index 000000000000..1f1a35efb251 --- /dev/null +++ b/datafusion/physical-optimizer/src/enforce_sorting/replace_with_order_preserving_variants.rs @@ -0,0 +1,288 @@ +// 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. + +//! Optimizer rule that replaces executors that lose ordering with their +//! order-preserving variants when it is helpful; either in terms of +//! performance or to accommodate unbounded streams by fixing the pipeline. + +use std::sync::Arc; + +use crate::utils::{ + is_coalesce_partitions, is_repartition, is_sort, is_sort_preserving_merge, +}; + +use datafusion_common::config::ConfigOptions; +use datafusion_common::tree_node::Transformed; +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::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 +/// descendants to decide whether it is beneficial to replace order-losing (but +/// somewhat faster) variants of certain operators with their order-preserving +/// (but somewhat slower) cousins. +pub type OrderPreservationContext = PlanContext; + +/// Updates order-preservation data for all children of the given node. +pub fn update_children(opc: &mut OrderPreservationContext) { + for PlanContext { + plan, + children, + data, + } in opc.children.iter_mut() + { + let maintains_input_order = plan.maintains_input_order(); + let inspect_child = |idx| { + maintains_input_order[idx] + || is_coalesce_partitions(plan) + || is_repartition(plan) + }; + + // We cut the path towards nodes that do not maintain ordering. + for (idx, c) in children.iter_mut().enumerate() { + c.data &= inspect_child(idx); + } + + let plan_children = plan.children(); + *data = if plan_children.is_empty() { + false + } else if !children[0].data + && ((is_repartition(plan) && !maintains_input_order[0]) + || (is_coalesce_partitions(plan) + && plan_children[0].output_ordering().is_some())) + { + // We either have a RepartitionExec or a CoalescePartitionsExec + // and they lose their input ordering, so initiate connection: + true + } else { + // Maintain connection if there is a child with a connection, + // and operator can possibly maintain that connection (either + // in its current form or when we replace it with the corresponding + // order preserving operator). + children + .iter() + .enumerate() + .any(|(idx, c)| c.data && inspect_child(idx)) + } + } + opc.data = false; +} + +/// Calculates the updated plan by replacing operators that lose ordering +/// inside `sort_input` with their order-preserving variants. This will +/// generate an alternative plan, which will be accepted or rejected later on +/// depending on whether it helps us remove a `SortExec`. +fn plan_with_order_preserving_variants( + mut sort_input: OrderPreservationContext, + // Flag indicating that it is desirable to replace `RepartitionExec`s with + // `SortPreservingRepartitionExec`s: + is_spr_better: bool, + // Flag indicating that it is desirable to replace `CoalescePartitionsExec`s + // with `SortPreservingMergeExec`s: + is_spm_better: bool, + fetch: Option, +) -> Result { + sort_input.children = sort_input + .children + .into_iter() + .map(|node| { + // Update descendants in the given tree if there is a connection: + if node.data { + plan_with_order_preserving_variants( + node, + is_spr_better, + is_spm_better, + fetch, + ) + } else { + Ok(node) + } + }) + .collect::>()?; + sort_input.data = false; + + if is_repartition(&sort_input.plan) + && !sort_input.plan.maintains_input_order()[0] + && is_spr_better + { + // When a `RepartitionExec` doesn't preserve ordering, replace it with + // a sort-preserving variant if appropriate: + let child = Arc::clone(&sort_input.children[0].plan); + let partitioning = sort_input.plan.output_partitioning().clone(); + sort_input.plan = Arc::new( + RepartitionExec::try_new(child, partitioning)?.with_preserve_order(), + ) as _; + sort_input.children[0].data = true; + return Ok(sort_input); + } else if is_coalesce_partitions(&sort_input.plan) && is_spm_better { + let child = &sort_input.children[0].plan; + if let Some(ordering) = child.output_ordering() { + // When the input of a `CoalescePartitionsExec` has an ordering, + // replace it with a `SortPreservingMergeExec` if appropriate: + let spm = SortPreservingMergeExec::new(ordering.clone(), Arc::clone(child)) + .with_fetch(fetch); + sort_input.plan = Arc::new(spm) as _; + sort_input.children[0].data = true; + return Ok(sort_input); + } + } + + sort_input.update_plan_from_children() +} + +/// Calculates the updated plan by replacing operators that preserve ordering +/// inside `sort_input` with their order-breaking variants. This will restore +/// the original plan modified by [`plan_with_order_preserving_variants`]. +fn plan_with_order_breaking_variants( + mut sort_input: OrderPreservationContext, +) -> Result { + let plan = &sort_input.plan; + sort_input.children = izip!( + sort_input.children, + plan.maintains_input_order(), + plan.required_input_ordering() + ) + .map(|(node, maintains, required_ordering)| { + // Replace with non-order preserving variants as long as ordering is + // not required by intermediate operators: + if maintains + && (is_sort_preserving_merge(plan) + || !required_ordering.is_some_and(|required_ordering| { + node.plan + .equivalence_properties() + .ordering_satisfy_requirement(&required_ordering) + })) + { + plan_with_order_breaking_variants(node) + } else { + Ok(node) + } + }) + .collect::>()?; + sort_input.data = false; + + if is_repartition(plan) && plan.maintains_input_order()[0] { + // When a `RepartitionExec` preserves ordering, replace it with a + // non-sort-preserving variant: + let child = Arc::clone(&sort_input.children[0].plan); + let partitioning = plan.output_partitioning().clone(); + sort_input.plan = Arc::new(RepartitionExec::try_new(child, partitioning)?) as _; + } else if is_sort_preserving_merge(plan) { + // Replace `SortPreservingMergeExec` with a `CoalescePartitionsExec`: + let child = Arc::clone(&sort_input.children[0].plan); + let coalesce = CoalescePartitionsExec::new(child); + sort_input.plan = Arc::new(coalesce) as _; + } else { + return sort_input.update_plan_from_children(); + } + + sort_input.children[0].data = false; + Ok(sort_input) +} + +/// The `replace_with_order_preserving_variants` optimizer sub-rule tries to +/// remove `SortExec`s from the physical plan by replacing operators that do +/// not preserve ordering with their order-preserving variants; i.e. by replacing +/// ordinary `RepartitionExec`s with their sort-preserving variants or by replacing +/// `CoalescePartitionsExec`s with `SortPreservingMergeExec`s. +/// +/// If this replacement is helpful for removing a `SortExec`, it updates the plan. +/// Otherwise, it leaves the plan unchanged. +/// +/// NOTE: This optimizer sub-rule will only produce sort-preserving `RepartitionExec`s +/// if the query is bounded or if the config option `prefer_existing_sort` is +/// set to `true`. +/// +/// The algorithm flow is simply like this: +/// 1. Visit nodes of the physical plan bottom-up and look for `SortExec` nodes. +/// During the traversal, keep track of operators that maintain ordering (or +/// can maintain ordering when replaced by an order-preserving variant) until +/// a `SortExec` is found. +/// 2. When a `SortExec` is found, update the child of the `SortExec` by replacing +/// operators that do not preserve ordering in the tree with their order +/// preserving variants. +/// 3. Check if the `SortExec` is still necessary in the updated plan by comparing +/// its input ordering with the output ordering it imposes. We do this because +/// replacing operators that lose ordering with their order-preserving variants +/// enables us to preserve the previously lost ordering at the input of `SortExec`. +/// 4. If the `SortExec` in question turns out to be unnecessary, remove it and +/// use updated plan. Otherwise, use the original plan. +/// 5. Continue the bottom-up traversal until another `SortExec` is seen, or the +/// traversal is complete. +pub fn replace_with_order_preserving_variants( + mut requirements: OrderPreservationContext, + // A flag indicating that replacing `RepartitionExec`s with sort-preserving + // variants is desirable when it helps to remove a `SortExec` from the plan. + // If this flag is `false`, this replacement should only be made to fix the + // pipeline (streaming). + is_spr_better: bool, + // A flag indicating that replacing `CoalescePartitionsExec`s with + // `SortPreservingMergeExec`s is desirable when it helps to remove a + // `SortExec` from the plan. If this flag is `false`, this replacement + // should only be made to fix the pipeline (streaming). + is_spm_better: bool, + config: &ConfigOptions, +) -> Result> { + update_children(&mut requirements); + if !(is_sort(&requirements.plan) && requirements.children[0].data) { + return Ok(Transformed::no(requirements)); + } + + // For unbounded cases, we replace with the order-preserving variant in any + // case, as doing so helps fix the pipeline. Also replace if config allows. + let use_order_preserving_variant = config.optimizer.prefer_existing_sort + || (requirements.plan.boundedness().is_unbounded() + && requirements.plan.pipeline_behavior() == EmissionType::Final); + + // Create an alternate plan with order-preserving variants: + let mut alternate_plan = plan_with_order_preserving_variants( + requirements.children.swap_remove(0), + is_spr_better || use_order_preserving_variant, + is_spm_better || use_order_preserving_variant, + requirements.plan.fetch(), + )?; + + // If the alternate plan makes this sort unnecessary, accept the alternate: + if alternate_plan + .plan + .equivalence_properties() + .ordering_satisfy( + requirements + .plan + .output_ordering() + .unwrap_or(LexOrdering::empty()), + ) + { + for child in alternate_plan.children.iter_mut() { + child.data = false; + } + Ok(Transformed::yes(alternate_plan)) + } else { + // The alternate plan does not help, use faster order-breaking variants: + alternate_plan = plan_with_order_breaking_variants(alternate_plan)?; + alternate_plan.data = false; + requirements.children = vec![alternate_plan]; + Ok(Transformed::yes(requirements)) + } +} diff --git a/datafusion/core/src/physical_optimizer/sort_pushdown.rs b/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs similarity index 97% rename from datafusion/core/src/physical_optimizer/sort_pushdown.rs rename to datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs index e24e66250704..d2f7bcacd089 100644 --- a/datafusion/core/src/physical_optimizer/sort_pushdown.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs @@ -18,18 +18,11 @@ use std::fmt::Debug; use std::sync::Arc; -use super::utils::{add_sort_above, is_sort}; -use crate::physical_optimizer::utils::{is_sort_preserving_merge, is_union, is_window}; -use crate::physical_plan::filter::FilterExec; -use crate::physical_plan::joins::utils::calculate_join_output_ordering; -use crate::physical_plan::joins::SortMergeJoinExec; -use crate::physical_plan::projection::ProjectionExec; -use crate::physical_plan::repartition::RepartitionExec; -use crate::physical_plan::sorts::sort::SortExec; -use crate::physical_plan::tree_node::PlanContext; -use crate::physical_plan::{ExecutionPlan, ExecutionPlanProperties}; use arrow_schema::SchemaRef; +use crate::utils::{ + add_sort_above, is_sort, is_sort_preserving_merge, is_union, is_window, +}; use datafusion_common::tree_node::{ ConcreteTreeNode, Transformed, TreeNode, TreeNodeRecursion, }; @@ -39,8 +32,16 @@ use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::utils::collect_columns; use datafusion_physical_expr::PhysicalSortRequirement; use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; -use datafusion_physical_plan::joins::utils::ColumnIndex; -use datafusion_physical_plan::joins::HashJoinExec; +use datafusion_physical_plan::filter::FilterExec; +use datafusion_physical_plan::joins::utils::{ + calculate_join_output_ordering, ColumnIndex, +}; +use datafusion_physical_plan::joins::{HashJoinExec, SortMergeJoinExec}; +use datafusion_physical_plan::projection::ProjectionExec; +use datafusion_physical_plan::repartition::RepartitionExec; +use datafusion_physical_plan::sorts::sort::SortExec; +use datafusion_physical_plan::tree_node::PlanContext; +use datafusion_physical_plan::{ExecutionPlan, ExecutionPlanProperties}; /// This is a "data class" we use within the [`EnforceSorting`] rule to push /// down [`SortExec`] in the plan. In some cases, we can reduce the total @@ -48,7 +49,7 @@ use datafusion_physical_plan::joins::HashJoinExec; /// object carries the parent required ordering and the (optional) `fetch` value /// of the parent node as its data. /// -/// [`EnforceSorting`]: crate::physical_optimizer::enforce_sorting::EnforceSorting +/// [`EnforceSorting`]: crate::enforce_sorting::EnforceSorting #[derive(Default, Clone)] pub struct ParentRequirements { ordering_requirement: Option, @@ -70,7 +71,7 @@ pub fn assign_initial_requirements(node: &mut SortPushDown) { } } -pub(crate) fn pushdown_sorts(sort_pushdown: SortPushDown) -> Result { +pub fn pushdown_sorts(sort_pushdown: SortPushDown) -> Result { let mut new_node = pushdown_sorts_helper(sort_pushdown)?; while new_node.tnr == TreeNodeRecursion::Stop { new_node = pushdown_sorts_helper(new_node.data)?; diff --git a/datafusion/physical-optimizer/src/lib.rs b/datafusion/physical-optimizer/src/lib.rs index ccb18f679171..62a1f68c1366 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_sorting; pub mod join_selection; pub mod limit_pushdown; pub mod limited_distinct_aggregation; @@ -31,5 +32,6 @@ pub mod sanity_checker; pub mod test_utils; pub mod topk_aggregation; pub mod update_aggr_exprs; +pub mod utils; pub use optimizer::PhysicalOptimizerRule; diff --git a/datafusion/core/src/physical_optimizer/utils.rs b/datafusion/physical-optimizer/src/utils.rs similarity index 85% rename from datafusion/core/src/physical_optimizer/utils.rs rename to datafusion/physical-optimizer/src/utils.rs index 9f2c28d564f0..ed280452b53b 100644 --- a/datafusion/core/src/physical_optimizer/utils.rs +++ b/datafusion/physical-optimizer/src/utils.rs @@ -15,22 +15,18 @@ // specific language governing permissions and limitations // under the License. -//! Collection of utility functions that are leveraged by the query optimizer rules - use std::sync::Arc; -use crate::physical_plan::coalesce_partitions::CoalescePartitionsExec; -use crate::physical_plan::repartition::RepartitionExec; -use crate::physical_plan::sorts::sort::SortExec; -use crate::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; -use crate::physical_plan::union::UnionExec; -use crate::physical_plan::windows::{BoundedWindowAggExec, WindowAggExec}; -use crate::physical_plan::{ExecutionPlan, ExecutionPlanProperties}; - -use datafusion_physical_expr::LexRequirement; -use datafusion_physical_expr_common::sort_expr::LexOrdering; +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::sort::SortExec; +use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; use datafusion_physical_plan::tree_node::PlanContext; +use datafusion_physical_plan::union::UnionExec; +use datafusion_physical_plan::windows::{BoundedWindowAggExec, WindowAggExec}; +use datafusion_physical_plan::{ExecutionPlan, ExecutionPlanProperties}; /// This utility function adds a `SortExec` above an operator according to the /// given ordering requirements while preserving the original partitioning. @@ -72,10 +68,9 @@ pub fn add_sort_above_with_check( } } -/// Checks whether the given operator is a limit; -/// i.e. either a [`LocalLimitExec`] or a [`GlobalLimitExec`]. -pub fn is_limit(plan: &Arc) -> bool { - plan.as_any().is::() || plan.as_any().is::() +/// Checks whether the given operator is a [`SortExec`]. +pub fn is_sort(plan: &Arc) -> bool { + plan.as_any().is::() } /// Checks whether the given operator is a window; @@ -84,9 +79,9 @@ pub fn is_window(plan: &Arc) -> bool { plan.as_any().is::() || plan.as_any().is::() } -/// Checks whether the given operator is a [`SortExec`]. -pub fn is_sort(plan: &Arc) -> bool { - plan.as_any().is::() +/// Checks whether the given operator is a [`UnionExec`]. +pub fn is_union(plan: &Arc) -> bool { + plan.as_any().is::() } /// Checks whether the given operator is a [`SortPreservingMergeExec`]. @@ -99,12 +94,13 @@ pub fn is_coalesce_partitions(plan: &Arc) -> bool { plan.as_any().is::() } -/// Checks whether the given operator is a [`UnionExec`]. -pub fn is_union(plan: &Arc) -> bool { - plan.as_any().is::() -} - /// Checks whether the given operator is a [`RepartitionExec`]. pub fn is_repartition(plan: &Arc) -> bool { plan.as_any().is::() } + +/// Checks whether the given operator is a limit; +/// i.e. either a [`LocalLimitExec`] or a [`GlobalLimitExec`]. +pub fn is_limit(plan: &Arc) -> bool { + plan.as_any().is::() || plan.as_any().is::() +}