From 1f12ffe42fb1bed3093ade21e08a6391da7040c8 Mon Sep 17 00:00:00 2001 From: logan-keede Date: Sun, 19 Jan 2025 03:27:36 +0530 Subject: [PATCH 01/24] moving enforce_distribution --- datafusion/core/src/physical_optimizer/mod.rs | 2 +- datafusion/physical-optimizer/Cargo.toml | 1 + .../src}/enforce_distribution.rs | 34 +++--- datafusion/physical-optimizer/src/lib.rs | 3 +- datafusion/physical-optimizer/src/utils.rs | 110 ++++++++++++++++++ 5 files changed, 133 insertions(+), 17 deletions(-) rename datafusion/{core/src/physical_optimizer => physical-optimizer/src}/enforce_distribution.rs (99%) create mode 100644 datafusion/physical-optimizer/src/utils.rs diff --git a/datafusion/core/src/physical_optimizer/mod.rs b/datafusion/core/src/physical_optimizer/mod.rs index 63fe115e602c..964604054d26 100644 --- a/datafusion/core/src/physical_optimizer/mod.rs +++ b/datafusion/core/src/physical_optimizer/mod.rs @@ -22,7 +22,7 @@ //! //! [`ExecutionPlan`]: crate::physical_plan::ExecutionPlan -pub mod enforce_distribution; +// pub mod enforce_distribution; pub mod enforce_sorting; pub mod optimizer; pub mod projection_pushdown; diff --git a/datafusion/physical-optimizer/Cargo.toml b/datafusion/physical-optimizer/Cargo.toml index 40074e8eecd8..c61d2ceafab9 100644 --- a/datafusion/physical-optimizer/Cargo.toml +++ b/datafusion/physical-optimizer/Cargo.toml @@ -33,6 +33,7 @@ workspace = true [features] recursive_protection = ["dep:recursive"] +parquet = ["parquet"] [dependencies] arrow = { workspace = true } diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/physical-optimizer/src/enforce_distribution.rs similarity index 99% rename from datafusion/core/src/physical_optimizer/enforce_distribution.rs rename to datafusion/physical-optimizer/src/enforce_distribution.rs index fbadceba0948..b2e7deba09c2 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/physical-optimizer/src/enforce_distribution.rs @@ -24,24 +24,28 @@ use std::fmt::Debug; use std::sync::Arc; -use crate::config::ConfigOptions; -use crate::error::Result; -use crate::physical_optimizer::utils::{ +use crate::output_requirements::OutputRequirementExec; +use crate::optimizer::PhysicalOptimizerRule; +use crate::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::{ + +use datafusion_common::config::ConfigOptions; +use datafusion_common::error::Result; + +use datafusion_physical_plan::aggregates::{AggregateExec, AggregateMode, PhysicalGroupBy}; +use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; +use datafusion_physical_plan::joins::{ CrossJoinExec, HashJoinExec, PartitionMode, SortMergeJoinExec, }; -use crate::physical_plan::projection::ProjectionExec; -use crate::physical_plan::repartition::RepartitionExec; -use crate::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; -use crate::physical_plan::tree_node::PlanContext; -use crate::physical_plan::union::{can_interleave, InterleaveExec, UnionExec}; -use crate::physical_plan::windows::WindowAggExec; -use crate::physical_plan::{Distribution, ExecutionPlan, Partitioning}; +use datafusion_physical_plan::projection::ProjectionExec; +use datafusion_physical_plan::repartition::RepartitionExec; +use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; +use datafusion_physical_plan::tree_node::PlanContext; +use datafusion_physical_plan::union::{can_interleave, InterleaveExec, UnionExec}; +use datafusion_physical_plan::windows::WindowAggExec; +use datafusion_physical_plan::{Distribution, ExecutionPlan, Partitioning}; use arrow::compute::SortOptions; use datafusion_common::stats::Precision; @@ -53,8 +57,8 @@ use datafusion_physical_expr::{ physical_exprs_equal, EquivalenceProperties, PhysicalExpr, PhysicalExprRef, }; use datafusion_physical_expr_common::sort_expr::LexOrdering; -use datafusion_physical_optimizer::output_requirements::OutputRequirementExec; -use datafusion_physical_optimizer::PhysicalOptimizerRule; +// use datafusion_physical_optimizer::output_requirements::OutputRequirementExec; +// use datafusion_physical_optimizer::PhysicalOptimizerRule; use datafusion_physical_plan::execution_plan::EmissionType; use datafusion_physical_plan::windows::{get_best_fitting_window, BoundedWindowAggExec}; use datafusion_physical_plan::ExecutionPlanProperties; diff --git a/datafusion/physical-optimizer/src/lib.rs b/datafusion/physical-optimizer/src/lib.rs index ccb18f679171..f42da9a1e464 100644 --- a/datafusion/physical-optimizer/src/lib.rs +++ b/datafusion/physical-optimizer/src/lib.rs @@ -31,5 +31,6 @@ pub mod sanity_checker; pub mod test_utils; pub mod topk_aggregation; pub mod update_aggr_exprs; - +pub mod enforce_distribution; pub use optimizer::PhysicalOptimizerRule; +mod utils; diff --git a/datafusion/physical-optimizer/src/utils.rs b/datafusion/physical-optimizer/src/utils.rs new file mode 100644 index 000000000000..fa16b5422310 --- /dev/null +++ b/datafusion/physical-optimizer/src/utils.rs @@ -0,0 +1,110 @@ +// 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 utility functions that are leveraged by the query optimizer rules + +use std::sync::Arc; + +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::union::UnionExec; +// use datafusion_physical_plan::windows::{BoundedWindowAggExec, WindowAggExec}; +use datafusion_physical_plan::{ExecutionPlan, ExecutionPlanProperties}; + +use datafusion_physical_expr::LexRequirement; +use datafusion_physical_expr_common::sort_expr::LexOrdering; +// use datafusion_physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; +use datafusion_physical_plan::tree_node::PlanContext; + +/// This utility function adds a `SortExec` above an operator according to the +/// given ordering requirements while preserving the original partitioning. +pub fn add_sort_above( + node: PlanContext, + sort_requirements: LexRequirement, + fetch: Option, +) -> PlanContext { + let mut sort_expr = LexOrdering::from(sort_requirements); + sort_expr.retain(|sort_expr| { + !node + .plan + .equivalence_properties() + .is_expr_constant(&sort_expr.expr) + }); + let mut new_sort = SortExec::new(sort_expr, Arc::clone(&node.plan)).with_fetch(fetch); + if node.plan.output_partitioning().partition_count() > 1 { + new_sort = new_sort.with_preserve_partitioning(true); + } + PlanContext::new(Arc::new(new_sort), T::default(), vec![node]) +} + +/// This utility function adds a `SortExec` above an operator according to the +/// given ordering requirements while preserving the original partitioning. If +/// requirement is already satisfied no `SortExec` is added. +pub fn add_sort_above_with_check( + node: PlanContext, + sort_requirements: LexRequirement, + fetch: Option, +) -> PlanContext { + if !node + .plan + .equivalence_properties() + .ordering_satisfy_requirement(&sort_requirements) + { + add_sort_above(node, sort_requirements, fetch) + } else { + node + } +} + +/// 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 window; +/// i.e. either a [`WindowAggExec`] or a [`BoundedWindowAggExec`]. +// 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 [`SortPreservingMergeExec`]. +pub fn is_sort_preserving_merge(plan: &Arc) -> bool { + plan.as_any().is::() +} + +/// Checks whether the given operator is a [`CoalescePartitionsExec`]. +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::() +} From b58f9d65deb81bf0b3dd4b6a8a2a465c1d5f9eca Mon Sep 17 00:00:00 2001 From: logan-keede Date: Sun, 19 Jan 2025 03:36:05 +0530 Subject: [PATCH 02/24] formatting fix --- datafusion/core/src/physical_optimizer/mod.rs | 1 - datafusion/physical-optimizer/src/enforce_distribution.rs | 6 ++++-- datafusion/physical-optimizer/src/lib.rs | 2 +- 3 files changed, 5 insertions(+), 4 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/mod.rs b/datafusion/core/src/physical_optimizer/mod.rs index 964604054d26..ae2adda71026 100644 --- a/datafusion/core/src/physical_optimizer/mod.rs +++ b/datafusion/core/src/physical_optimizer/mod.rs @@ -22,7 +22,6 @@ //! //! [`ExecutionPlan`]: crate::physical_plan::ExecutionPlan -// pub mod enforce_distribution; pub mod enforce_sorting; pub mod optimizer; pub mod projection_pushdown; diff --git a/datafusion/physical-optimizer/src/enforce_distribution.rs b/datafusion/physical-optimizer/src/enforce_distribution.rs index b2e7deba09c2..58149a01df5f 100644 --- a/datafusion/physical-optimizer/src/enforce_distribution.rs +++ b/datafusion/physical-optimizer/src/enforce_distribution.rs @@ -24,8 +24,8 @@ use std::fmt::Debug; use std::sync::Arc; -use crate::output_requirements::OutputRequirementExec; use crate::optimizer::PhysicalOptimizerRule; +use crate::output_requirements::OutputRequirementExec; use crate::utils::{ add_sort_above_with_check, is_coalesce_partitions, is_repartition, is_sort_preserving_merge, @@ -34,7 +34,9 @@ use crate::utils::{ use datafusion_common::config::ConfigOptions; use datafusion_common::error::Result; -use datafusion_physical_plan::aggregates::{AggregateExec, AggregateMode, PhysicalGroupBy}; +use datafusion_physical_plan::aggregates::{ + AggregateExec, AggregateMode, PhysicalGroupBy, +}; use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; use datafusion_physical_plan::joins::{ CrossJoinExec, HashJoinExec, PartitionMode, SortMergeJoinExec, diff --git a/datafusion/physical-optimizer/src/lib.rs b/datafusion/physical-optimizer/src/lib.rs index f42da9a1e464..a5db5fbf4fd2 100644 --- a/datafusion/physical-optimizer/src/lib.rs +++ b/datafusion/physical-optimizer/src/lib.rs @@ -21,6 +21,7 @@ pub mod aggregate_statistics; pub mod coalesce_batches; pub mod combine_partial_final_agg; +pub mod enforce_distribution; pub mod join_selection; pub mod limit_pushdown; pub mod limited_distinct_aggregation; @@ -31,6 +32,5 @@ pub mod sanity_checker; pub mod test_utils; pub mod topk_aggregation; pub mod update_aggr_exprs; -pub mod enforce_distribution; pub use optimizer::PhysicalOptimizerRule; mod utils; From 0d571a76a19a1d8342ed3e89f40f3f599db0cf40 Mon Sep 17 00:00:00 2001 From: logan-keede <68557630+logan-keede@users.noreply.github.com> Date: Sun, 19 Jan 2025 03:55:10 +0530 Subject: [PATCH 03/24] Update Cargo.toml Fix circular dependency --- datafusion/physical-optimizer/Cargo.toml | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/datafusion/physical-optimizer/Cargo.toml b/datafusion/physical-optimizer/Cargo.toml index c61d2ceafab9..3a2f15bb6f50 100644 --- a/datafusion/physical-optimizer/Cargo.toml +++ b/datafusion/physical-optimizer/Cargo.toml @@ -33,7 +33,7 @@ workspace = true [features] recursive_protection = ["dep:recursive"] -parquet = ["parquet"] +parquet = ["dep:parquet"] [dependencies] arrow = { workspace = true } @@ -49,6 +49,7 @@ futures = { workspace = true } itertools = { workspace = true } log = { workspace = true } recursive = { workspace = true, optional = true } +parquet = { workspace = true, optional = true, default-features = true } [dev-dependencies] datafusion-expr = { workspace = true } From 24ab91364f34276a0f19b03256e4a198e0d3608d Mon Sep 17 00:00:00 2001 From: logan-keede Date: Mon, 20 Jan 2025 01:42:54 +0530 Subject: [PATCH 04/24] pulll source --- datafusion/physical-optimizer/Cargo.toml | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/datafusion/physical-optimizer/Cargo.toml b/datafusion/physical-optimizer/Cargo.toml index c61d2ceafab9..3a2f15bb6f50 100644 --- a/datafusion/physical-optimizer/Cargo.toml +++ b/datafusion/physical-optimizer/Cargo.toml @@ -33,7 +33,7 @@ workspace = true [features] recursive_protection = ["dep:recursive"] -parquet = ["parquet"] +parquet = ["dep:parquet"] [dependencies] arrow = { workspace = true } @@ -49,6 +49,7 @@ futures = { workspace = true } itertools = { workspace = true } log = { workspace = true } recursive = { workspace = true, optional = true } +parquet = { workspace = true, optional = true, default-features = true } [dev-dependencies] datafusion-expr = { workspace = true } From a7f8927166f4dcb39cd8300743a7af210e366d81 Mon Sep 17 00:00:00 2001 From: logan-keede Date: Mon, 20 Jan 2025 04:17:36 +0530 Subject: [PATCH 05/24] move_tests to core integration tests --- .../enforce_distribution.rs | 3251 +++++++++++++++++ .../core/tests/physical_optimizer/mod.rs | 1 + 2 files changed, 3252 insertions(+) create mode 100644 datafusion/core/tests/physical_optimizer/enforce_distribution.rs diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs new file mode 100644 index 000000000000..5575f50a38ec --- /dev/null +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -0,0 +1,3251 @@ +use std::ops::Deref; + +use super::*; +use datafusion::datasource::file_format::file_compression_type::FileCompressionType; +use datafusion::datasource::listing::PartitionedFile; +use datafusion::datasource::object_store::ObjectStoreUrl; +use datafusion::datasource::physical_plan::{CsvExec, FileScanConfig, ParquetExec}; +use datafusion::physical_plan::enforce_sorting::EnforceSorting; +use datafusion_physical_plan::coalesce_batches::CoalesceBatchesExec; +use datafusion_physical_plan::expressions::col; +use datafusion_physical_plan::filter::FilterExec; +use datafusion_physical_plan::joins::utils::JoinOn; +use datafusion_physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; +use datafusion_physical_plan::sorts::sort::SortExec; +use datafusion_physical_plan::{displayable, DisplayAs, DisplayFormatType, Statistics}; +use datafusion_physical_optimizer::output_requirements::OutputRequirements; +use datafusion_physical_optimizer::test_utils::{ + check_integrity, coalesce_partitions_exec, repartition_exec, +}; + +use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; +use datafusion_common::ScalarValue; +use datafusion_expr::Operator; +use datafusion_physical_expr::expressions::{BinaryExpr, Literal}; +use datafusion_physical_expr::{ + expressions::binary, expressions::lit, LexOrdering, PhysicalSortExpr, +}; +use datafusion_physical_expr_common::sort_expr::LexRequirement; +use datafusion_physical_plan::PlanProperties; + +/// Models operators like BoundedWindowExec that require an input +/// ordering but is easy to construct +#[derive(Debug)] +struct SortRequiredExec { + input: Arc, + expr: LexOrdering, + cache: PlanProperties, +} + +impl SortRequiredExec { + fn new_with_requirement( + input: Arc, + requirement: LexOrdering, + ) -> Self { + let cache = Self::compute_properties(&input); + Self { + input, + expr: requirement, + cache, + } + } + + /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. + fn compute_properties(input: &Arc) -> PlanProperties { + PlanProperties::new( + input.equivalence_properties().clone(), // Equivalence Properties + input.output_partitioning().clone(), // Output Partitioning + input.pipeline_behavior(), // Pipeline Behavior + input.boundedness(), // Boundedness + ) + } +} + +impl DisplayAs for SortRequiredExec { + fn fmt_as( + &self, + _t: DisplayFormatType, + f: &mut std::fmt::Formatter, + ) -> std::fmt::Result { + write!(f, "SortRequiredExec: [{}]", self.expr) + } +} + +impl ExecutionPlan for SortRequiredExec { + fn name(&self) -> &'static str { + "SortRequiredExec" + } + + fn as_any(&self) -> &dyn std::any::Any { + self + } + + fn properties(&self) -> &PlanProperties { + &self.cache + } + + fn benefits_from_input_partitioning(&self) -> Vec { + vec![false] + } + + fn children(&self) -> Vec<&Arc> { + vec![&self.input] + } + + // model that it requires the output ordering of its input + fn required_input_ordering(&self) -> Vec> { + if self.expr.is_empty() { + vec![None] + } else { + vec![Some(LexRequirement::from(self.expr.clone()))] + } + } + + fn with_new_children( + self: Arc, + mut children: Vec>, + ) -> Result> { + assert_eq!(children.len(), 1); + let child = children.pop().unwrap(); + Ok(Arc::new(Self::new_with_requirement( + child, + self.expr.clone(), + ))) + } + + fn execute( + &self, + _partition: usize, + _context: Arc, + ) -> Result { + unreachable!(); + } + + fn statistics(&self) -> Result { + self.input.statistics() + } +} + +pub(crate) fn schema() -> SchemaRef { + Arc::new(Schema::new(vec![ + Field::new("a", DataType::Int64, true), + Field::new("b", DataType::Int64, true), + Field::new("c", DataType::Int64, true), + Field::new("d", DataType::Int32, true), + Field::new("e", DataType::Boolean, true), + ])) +} + +fn parquet_exec() -> Arc { + parquet_exec_with_sort(vec![]) +} + +/// create a single parquet file that is sorted +pub(crate) fn parquet_exec_with_sort( + output_ordering: Vec, +) -> Arc { + ParquetExec::builder( + FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema()) + .with_file(PartitionedFile::new("x".to_string(), 100)) + .with_output_ordering(output_ordering), + ) + .build_arc() +} + +fn parquet_exec_multiple() -> Arc { + parquet_exec_multiple_sorted(vec![]) +} + +/// Created a sorted parquet exec with multiple files +fn parquet_exec_multiple_sorted( + output_ordering: Vec, +) -> Arc { + ParquetExec::builder( + FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema()) + .with_file_groups(vec![ + vec![PartitionedFile::new("x".to_string(), 100)], + vec![PartitionedFile::new("y".to_string(), 100)], + ]) + .with_output_ordering(output_ordering), + ) + .build_arc() +} + +fn csv_exec() -> Arc { + csv_exec_with_sort(vec![]) +} + +fn csv_exec_with_sort(output_ordering: Vec) -> Arc { + Arc::new( + CsvExec::builder( + FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema()) + .with_file(PartitionedFile::new("x".to_string(), 100)) + .with_output_ordering(output_ordering), + ) + .with_has_header(false) + .with_delimeter(b',') + .with_quote(b'"') + .with_escape(None) + .with_comment(None) + .with_newlines_in_values(false) + .with_file_compression_type(FileCompressionType::UNCOMPRESSED) + .build(), + ) +} + +fn csv_exec_multiple() -> Arc { + csv_exec_multiple_sorted(vec![]) +} + +// Created a sorted parquet exec with multiple files +fn csv_exec_multiple_sorted(output_ordering: Vec) -> Arc { + Arc::new( + CsvExec::builder( + FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema()) + .with_file_groups(vec![ + vec![PartitionedFile::new("x".to_string(), 100)], + vec![PartitionedFile::new("y".to_string(), 100)], + ]) + .with_output_ordering(output_ordering), + ) + .with_has_header(false) + .with_delimeter(b',') + .with_quote(b'"') + .with_escape(None) + .with_comment(None) + .with_newlines_in_values(false) + .with_file_compression_type(FileCompressionType::UNCOMPRESSED) + .build(), + ) +} + +fn projection_exec_with_alias( + input: Arc, + alias_pairs: Vec<(String, String)>, +) -> Arc { + let mut exprs = vec![]; + for (column, alias) in alias_pairs.iter() { + exprs.push((col(column, &input.schema()).unwrap(), alias.to_string())); + } + Arc::new(ProjectionExec::try_new(exprs, input).unwrap()) +} + +fn aggregate_exec_with_alias( + input: Arc, + alias_pairs: Vec<(String, String)>, +) -> Arc { + let schema = schema(); + let mut group_by_expr: Vec<(Arc, String)> = vec![]; + for (column, alias) in alias_pairs.iter() { + group_by_expr + .push((col(column, &input.schema()).unwrap(), alias.to_string())); + } + let group_by = PhysicalGroupBy::new_single(group_by_expr.clone()); + + let final_group_by_expr = group_by_expr + .iter() + .enumerate() + .map(|(index, (_col, name))| { + ( + Arc::new(Column::new(name, index)) as Arc, + name.clone(), + ) + }) + .collect::>(); + let final_grouping = PhysicalGroupBy::new_single(final_group_by_expr); + + Arc::new( + AggregateExec::try_new( + AggregateMode::FinalPartitioned, + final_grouping, + vec![], + vec![], + Arc::new( + AggregateExec::try_new( + AggregateMode::Partial, + group_by, + vec![], + vec![], + input, + schema.clone(), + ) + .unwrap(), + ), + schema, + ) + .unwrap(), + ) +} + +fn hash_join_exec( + left: Arc, + right: Arc, + join_on: &JoinOn, + join_type: &JoinType, +) -> Arc { + Arc::new( + HashJoinExec::try_new( + left, + right, + join_on.clone(), + None, + join_type, + None, + PartitionMode::Partitioned, + false, + ) + .unwrap(), + ) +} + +fn sort_merge_join_exec( + left: Arc, + right: Arc, + join_on: &JoinOn, + join_type: &JoinType, +) -> Arc { + Arc::new( + SortMergeJoinExec::try_new( + left, + right, + join_on.clone(), + None, + *join_type, + vec![SortOptions::default(); join_on.len()], + false, + ) + .unwrap(), + ) +} + +fn filter_exec(input: Arc) -> Arc { + let predicate = Arc::new(BinaryExpr::new( + col("c", &schema()).unwrap(), + Operator::Eq, + Arc::new(Literal::new(ScalarValue::Int64(Some(0)))), + )); + Arc::new(FilterExec::try_new(predicate, input).unwrap()) +} + +fn sort_exec( + sort_exprs: LexOrdering, + input: Arc, + preserve_partitioning: bool, +) -> Arc { + let new_sort = SortExec::new(sort_exprs, input) + .with_preserve_partitioning(preserve_partitioning); + Arc::new(new_sort) +} + +fn sort_preserving_merge_exec( + sort_exprs: LexOrdering, + input: Arc, +) -> Arc { + Arc::new(SortPreservingMergeExec::new(sort_exprs, input)) +} + +fn limit_exec(input: Arc) -> Arc { + Arc::new(GlobalLimitExec::new( + Arc::new(LocalLimitExec::new(input, 100)), + 0, + Some(100), + )) +} + +fn union_exec(input: Vec>) -> Arc { + Arc::new(UnionExec::new(input)) +} + +fn sort_required_exec_with_req( + input: Arc, + sort_exprs: LexOrdering, +) -> Arc { + Arc::new(SortRequiredExec::new_with_requirement(input, sort_exprs)) +} + +pub(crate) fn trim_plan_display(plan: &str) -> Vec<&str> { + plan.split('\n') + .map(|s| s.trim()) + .filter(|s| !s.is_empty()) + .collect() +} + +fn ensure_distribution_helper( + plan: Arc, + target_partitions: usize, + prefer_existing_sort: bool, +) -> Result> { + let distribution_context = DistributionContext::new_default(plan); + let mut config = ConfigOptions::new(); + config.execution.target_partitions = target_partitions; + config.optimizer.enable_round_robin_repartition = true; + config.optimizer.repartition_file_scans = false; + config.optimizer.repartition_file_min_size = 1024; + config.optimizer.prefer_existing_sort = prefer_existing_sort; + ensure_distribution(distribution_context, &config).map(|item| item.data.plan) +} + +/// Test whether plan matches with expected plan +macro_rules! plans_matches_expected { + ($EXPECTED_LINES: expr, $PLAN: expr) => { + let physical_plan = $PLAN; + let formatted = displayable(physical_plan.as_ref()).indent(true).to_string(); + let actual: Vec<&str> = formatted.trim().lines().collect(); + + let expected_plan_lines: Vec<&str> = $EXPECTED_LINES + .iter().map(|s| *s).collect(); + + assert_eq!( + expected_plan_lines, actual, + "\n**Original Plan Mismatch\n\nexpected:\n\n{expected_plan_lines:#?}\nactual:\n\n{actual:#?}\n\n" + ); + } +} + +/// Runs the repartition optimizer and asserts the plan against the expected +/// Arguments +/// * `EXPECTED_LINES` - Expected output plan +/// * `PLAN` - Input plan +/// * `FIRST_ENFORCE_DIST` - +/// true: (EnforceDistribution, EnforceDistribution, EnforceSorting) +/// false: else runs (EnforceSorting, EnforceDistribution, EnforceDistribution) +/// * `PREFER_EXISTING_SORT` (optional) - if true, will not repartition / resort data if it is already sorted +/// * `TARGET_PARTITIONS` (optional) - number of partitions to repartition to +/// * `REPARTITION_FILE_SCANS` (optional) - if true, will repartition file scans +/// * `REPARTITION_FILE_MIN_SIZE` (optional) - minimum file size to repartition +/// * `PREFER_EXISTING_UNION` (optional) - if true, will not attempt to convert Union to Interleave +macro_rules! assert_optimized { + ($EXPECTED_LINES: expr, $PLAN: expr, $FIRST_ENFORCE_DIST: expr) => { + assert_optimized!($EXPECTED_LINES, $PLAN, $FIRST_ENFORCE_DIST, false, 10, false, 1024, false); + }; + + ($EXPECTED_LINES: expr, $PLAN: expr, $FIRST_ENFORCE_DIST: expr, $PREFER_EXISTING_SORT: expr) => { + assert_optimized!($EXPECTED_LINES, $PLAN, $FIRST_ENFORCE_DIST, $PREFER_EXISTING_SORT, 10, false, 1024, false); + }; + + ($EXPECTED_LINES: expr, $PLAN: expr, $FIRST_ENFORCE_DIST: expr, $PREFER_EXISTING_SORT: expr, $PREFER_EXISTING_UNION: expr) => { + assert_optimized!($EXPECTED_LINES, $PLAN, $FIRST_ENFORCE_DIST, $PREFER_EXISTING_SORT, 10, false, 1024, $PREFER_EXISTING_UNION); + }; + + ($EXPECTED_LINES: expr, $PLAN: expr, $FIRST_ENFORCE_DIST: expr, $PREFER_EXISTING_SORT: expr, $TARGET_PARTITIONS: expr, $REPARTITION_FILE_SCANS: expr, $REPARTITION_FILE_MIN_SIZE: expr) => { + assert_optimized!($EXPECTED_LINES, $PLAN, $FIRST_ENFORCE_DIST, $PREFER_EXISTING_SORT, $TARGET_PARTITIONS, $REPARTITION_FILE_SCANS, $REPARTITION_FILE_MIN_SIZE, false); + }; + + ($EXPECTED_LINES: expr, $PLAN: expr, $FIRST_ENFORCE_DIST: expr, $PREFER_EXISTING_SORT: expr, $TARGET_PARTITIONS: expr, $REPARTITION_FILE_SCANS: expr, $REPARTITION_FILE_MIN_SIZE: expr, $PREFER_EXISTING_UNION: expr) => { + let expected_lines: Vec<&str> = $EXPECTED_LINES.iter().map(|s| *s).collect(); + + let mut config = ConfigOptions::new(); + config.execution.target_partitions = $TARGET_PARTITIONS; + config.optimizer.repartition_file_scans = $REPARTITION_FILE_SCANS; + config.optimizer.repartition_file_min_size = $REPARTITION_FILE_MIN_SIZE; + config.optimizer.prefer_existing_sort = $PREFER_EXISTING_SORT; + config.optimizer.prefer_existing_union = $PREFER_EXISTING_UNION; + // Use a small batch size, to trigger RoundRobin in tests + config.execution.batch_size = 1; + + // NOTE: These tests verify the joint `EnforceDistribution` + `EnforceSorting` cascade + // because they were written prior to the separation of `BasicEnforcement` into + // `EnforceSorting` and `EnforceDistribution`. + // TODO: Orthogonalize the tests here just to verify `EnforceDistribution` and create + // new tests for the cascade. + + // Add the ancillary output requirements operator at the start: + let optimizer = OutputRequirements::new_add_mode(); + let optimized = optimizer.optimize($PLAN.clone(), &config)?; + + // This file has 2 rules that use tree node, apply these rules to original plan consecutively + // After these operations tree nodes should be in a consistent state. + // This code block makes sure that these rules doesn't violate tree node integrity. + { + let adjusted = if config.optimizer.top_down_join_key_reordering { + // Run adjust_input_keys_ordering rule + let plan_requirements = + PlanWithKeyRequirements::new_default($PLAN.clone()); + let adjusted = plan_requirements + .transform_down(adjust_input_keys_ordering) + .data() + .and_then(check_integrity)?; + // TODO: End state payloads will be checked here. + adjusted.plan + } else { + // Run reorder_join_keys_to_inputs rule + $PLAN.clone().transform_up(|plan| { + Ok(Transformed::yes(reorder_join_keys_to_inputs(plan)?)) + }) + .data()? + }; + + // Then run ensure_distribution rule + DistributionContext::new_default(adjusted) + .transform_up(|distribution_context| { + ensure_distribution(distribution_context, &config) + }) + .data() + .and_then(check_integrity)?; + // TODO: End state payloads will be checked here. + } + + let optimized = if $FIRST_ENFORCE_DIST { + // Run enforce distribution rule first: + let optimizer = EnforceDistribution::new(); + let optimized = optimizer.optimize(optimized, &config)?; + // The rule should be idempotent. + // Re-running this rule shouldn't introduce unnecessary operators. + let optimizer = EnforceDistribution::new(); + let optimized = optimizer.optimize(optimized, &config)?; + // Run the enforce sorting rule: + let optimizer = EnforceSorting::new(); + let optimized = optimizer.optimize(optimized, &config)?; + optimized + } else { + // Run the enforce sorting rule first: + let optimizer = EnforceSorting::new(); + let optimized = optimizer.optimize(optimized, &config)?; + // Run enforce distribution rule: + let optimizer = EnforceDistribution::new(); + let optimized = optimizer.optimize(optimized, &config)?; + // The rule should be idempotent. + // Re-running this rule shouldn't introduce unnecessary operators. + let optimizer = EnforceDistribution::new(); + let optimized = optimizer.optimize(optimized, &config)?; + optimized + }; + + // Remove the ancillary output requirements operator when done: + let optimizer = OutputRequirements::new_remove_mode(); + let optimized = optimizer.optimize(optimized, &config)?; + + // Now format correctly + let plan = displayable(optimized.as_ref()).indent(true).to_string(); + let actual_lines = trim_plan_display(&plan); + + assert_eq!( + &expected_lines, &actual_lines, + "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", + expected_lines, actual_lines + ); + }; +} + +macro_rules! assert_plan_txt { + ($EXPECTED_LINES: expr, $PLAN: expr) => { + let expected_lines: Vec<&str> = $EXPECTED_LINES.iter().map(|s| *s).collect(); + // Now format correctly + let plan = displayable($PLAN.as_ref()).indent(true).to_string(); + let actual_lines = trim_plan_display(&plan); + + assert_eq!( + &expected_lines, &actual_lines, + "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", + expected_lines, actual_lines + ); + }; +} + +#[test] +fn multi_hash_joins() -> Result<()> { + let left = parquet_exec(); + let alias_pairs: Vec<(String, String)> = vec![ + ("a".to_string(), "a1".to_string()), + ("b".to_string(), "b1".to_string()), + ("c".to_string(), "c1".to_string()), + ("d".to_string(), "d1".to_string()), + ("e".to_string(), "e1".to_string()), + ]; + let right = projection_exec_with_alias(parquet_exec(), alias_pairs); + let join_types = vec![ + JoinType::Inner, + JoinType::Left, + JoinType::Right, + JoinType::Full, + JoinType::LeftSemi, + JoinType::LeftAnti, + JoinType::LeftMark, + JoinType::RightSemi, + JoinType::RightAnti, + ]; + + // Join on (a == b1) + let join_on = vec![( + Arc::new(Column::new_with_schema("a", &schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("b1", &right.schema()).unwrap()) as _, + )]; + + for join_type in join_types { + let join = hash_join_exec(left.clone(), right.clone(), &join_on, &join_type); + let join_plan = format!( + "HashJoinExec: mode=Partitioned, join_type={join_type}, on=[(a@0, b1@1)]" + ); + + match join_type { + JoinType::Inner + | JoinType::Left + | JoinType::Right + | JoinType::Full + | JoinType::LeftSemi + | JoinType::LeftAnti + | JoinType::LeftMark => { + // Join on (a == c) + let top_join_on = vec![( + Arc::new(Column::new_with_schema("a", &join.schema()).unwrap()) + as _, + Arc::new(Column::new_with_schema("c", &schema()).unwrap()) as _, + )]; + let top_join = hash_join_exec( + join.clone(), + parquet_exec(), + &top_join_on, + &join_type, + ); + let top_join_plan = + format!("HashJoinExec: mode=Partitioned, join_type={join_type}, on=[(a@0, c@2)]"); + + let expected = match join_type { + // Should include 3 RepartitionExecs + JoinType::Inner | JoinType::Left | JoinType::LeftSemi | JoinType::LeftAnti | JoinType::LeftMark => vec![ + top_join_plan.as_str(), + join_plan.as_str(), + "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ], + // Should include 4 RepartitionExecs + _ => vec![ + top_join_plan.as_str(), + "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", + join_plan.as_str(), + "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ], + }; + assert_optimized!(expected, top_join.clone(), true); + assert_optimized!(expected, top_join, false); + } + JoinType::RightSemi | JoinType::RightAnti => {} + } + + match join_type { + JoinType::Inner + | JoinType::Left + | JoinType::Right + | JoinType::Full + | JoinType::RightSemi + | JoinType::RightAnti => { + // This time we use (b1 == c) for top join + // Join on (b1 == c) + let top_join_on = vec![( + Arc::new(Column::new_with_schema("b1", &join.schema()).unwrap()) + as _, + Arc::new(Column::new_with_schema("c", &schema()).unwrap()) as _, + )]; + + let top_join = + hash_join_exec(join, parquet_exec(), &top_join_on, &join_type); + let top_join_plan = match join_type { + JoinType::RightSemi | JoinType::RightAnti => + format!("HashJoinExec: mode=Partitioned, join_type={join_type}, on=[(b1@1, c@2)]"), + _ => + format!("HashJoinExec: mode=Partitioned, join_type={join_type}, on=[(b1@6, c@2)]"), + }; + + let expected = match join_type { + // Should include 3 RepartitionExecs + JoinType::Inner | JoinType::Right | JoinType::RightSemi | JoinType::RightAnti => + vec![ + top_join_plan.as_str(), + join_plan.as_str(), + "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ], + // Should include 4 RepartitionExecs + _ => + vec![ + top_join_plan.as_str(), + "RepartitionExec: partitioning=Hash([b1@6], 10), input_partitions=10", + join_plan.as_str(), + "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ], + }; + assert_optimized!(expected, top_join.clone(), true); + assert_optimized!(expected, top_join, false); + } + JoinType::LeftSemi | JoinType::LeftAnti | JoinType::LeftMark => {} + } + } + + Ok(()) +} + +#[test] +fn multi_joins_after_alias() -> Result<()> { + let left = parquet_exec(); + let right = parquet_exec(); + + // Join on (a == b) + let join_on = vec![( + Arc::new(Column::new_with_schema("a", &schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("b", &schema()).unwrap()) as _, + )]; + let join = hash_join_exec(left, right.clone(), &join_on, &JoinType::Inner); + + // Projection(a as a1, a as a2) + let alias_pairs: Vec<(String, String)> = vec![ + ("a".to_string(), "a1".to_string()), + ("a".to_string(), "a2".to_string()), + ]; + let projection = projection_exec_with_alias(join, alias_pairs); + + // Join on (a1 == c) + let top_join_on = vec![( + Arc::new(Column::new_with_schema("a1", &projection.schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("c", &schema()).unwrap()) as _, + )]; + + let top_join = hash_join_exec( + projection.clone(), + right.clone(), + &top_join_on, + &JoinType::Inner, + ); + + // Output partition need to respect the Alias and should not introduce additional RepartitionExec + let expected = &[ + "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a1@0, c@2)]", + "ProjectionExec: expr=[a@0 as a1, a@0 as a2]", + "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, b@1)]", + "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "RepartitionExec: partitioning=Hash([b@1], 10), input_partitions=10", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + assert_optimized!(expected, top_join.clone(), true); + assert_optimized!(expected, top_join, false); + + // Join on (a2 == c) + let top_join_on = vec![( + Arc::new(Column::new_with_schema("a2", &projection.schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("c", &schema()).unwrap()) as _, + )]; + + let top_join = hash_join_exec(projection, right, &top_join_on, &JoinType::Inner); + + // Output partition need to respect the Alias and should not introduce additional RepartitionExec + let expected = &[ + "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a2@1, c@2)]", + "ProjectionExec: expr=[a@0 as a1, a@0 as a2]", + "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, b@1)]", + "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "RepartitionExec: partitioning=Hash([b@1], 10), input_partitions=10", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + assert_optimized!(expected, top_join.clone(), true); + assert_optimized!(expected, top_join, false); + + Ok(()) +} + +#[test] +fn multi_joins_after_multi_alias() -> Result<()> { + let left = parquet_exec(); + let right = parquet_exec(); + + // Join on (a == b) + let join_on = vec![( + Arc::new(Column::new_with_schema("a", &schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("b", &schema()).unwrap()) as _, + )]; + + let join = hash_join_exec(left, right.clone(), &join_on, &JoinType::Inner); + + // Projection(c as c1) + let alias_pairs: Vec<(String, String)> = + vec![("c".to_string(), "c1".to_string())]; + let projection = projection_exec_with_alias(join, alias_pairs); + + // Projection(c1 as a) + let alias_pairs: Vec<(String, String)> = + vec![("c1".to_string(), "a".to_string())]; + let projection2 = projection_exec_with_alias(projection, alias_pairs); + + // Join on (a == c) + let top_join_on = vec![( + Arc::new(Column::new_with_schema("a", &projection2.schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("c", &schema()).unwrap()) as _, + )]; + + let top_join = hash_join_exec(projection2, right, &top_join_on, &JoinType::Inner); + + // The Column 'a' has different meaning now after the two Projections + // The original Output partition can not satisfy the Join requirements and need to add an additional RepartitionExec + let expected = &[ + "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, c@2)]", + "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", + "ProjectionExec: expr=[c1@0 as a]", + "ProjectionExec: expr=[c@2 as c1]", + "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, b@1)]", + "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "RepartitionExec: partitioning=Hash([b@1], 10), input_partitions=10", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + + assert_optimized!(expected, top_join.clone(), true); + assert_optimized!(expected, top_join, false); + + Ok(()) +} + +#[test] +fn join_after_agg_alias() -> Result<()> { + // group by (a as a1) + let left = aggregate_exec_with_alias( + parquet_exec(), + vec![("a".to_string(), "a1".to_string())], + ); + // group by (a as a2) + let right = aggregate_exec_with_alias( + parquet_exec(), + vec![("a".to_string(), "a2".to_string())], + ); + + // Join on (a1 == a2) + let join_on = vec![( + Arc::new(Column::new_with_schema("a1", &left.schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("a2", &right.schema()).unwrap()) as _, + )]; + let join = hash_join_exec(left, right.clone(), &join_on, &JoinType::Inner); + + // Only two RepartitionExecs added + let expected = &[ + "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a1@0, a2@0)]", + "AggregateExec: mode=FinalPartitioned, gby=[a1@0 as a1], aggr=[]", + "RepartitionExec: partitioning=Hash([a1@0], 10), input_partitions=10", + "AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[]", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "AggregateExec: mode=FinalPartitioned, gby=[a2@0 as a2], aggr=[]", + "RepartitionExec: partitioning=Hash([a2@0], 10), input_partitions=10", + "AggregateExec: mode=Partial, gby=[a@0 as a2], aggr=[]", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + assert_optimized!(expected, join.clone(), true); + assert_optimized!(expected, join, false); + + Ok(()) +} + +#[test] +fn hash_join_key_ordering() -> Result<()> { + // group by (a as a1, b as b1) + let left = aggregate_exec_with_alias( + parquet_exec(), + vec![ + ("a".to_string(), "a1".to_string()), + ("b".to_string(), "b1".to_string()), + ], + ); + // group by (b, a) + let right = aggregate_exec_with_alias( + parquet_exec(), + vec![ + ("b".to_string(), "b".to_string()), + ("a".to_string(), "a".to_string()), + ], + ); + + // Join on (b1 == b && a1 == a) + let join_on = vec![ + ( + Arc::new(Column::new_with_schema("b1", &left.schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("b", &right.schema()).unwrap()) as _, + ), + ( + Arc::new(Column::new_with_schema("a1", &left.schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("a", &right.schema()).unwrap()) as _, + ), + ]; + let join = hash_join_exec(left, right.clone(), &join_on, &JoinType::Inner); + + // Only two RepartitionExecs added + let expected = &[ + "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(b1@1, b@0), (a1@0, a@1)]", + "ProjectionExec: expr=[a1@1 as a1, b1@0 as b1]", + "AggregateExec: mode=FinalPartitioned, gby=[b1@0 as b1, a1@1 as a1], aggr=[]", + "RepartitionExec: partitioning=Hash([b1@0, a1@1], 10), input_partitions=10", + "AggregateExec: mode=Partial, gby=[b@1 as b1, a@0 as a1], aggr=[]", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "AggregateExec: mode=FinalPartitioned, gby=[b@0 as b, a@1 as a], aggr=[]", + "RepartitionExec: partitioning=Hash([b@0, a@1], 10), input_partitions=10", + "AggregateExec: mode=Partial, gby=[b@1 as b, a@0 as a], aggr=[]", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + assert_optimized!(expected, join.clone(), true); + assert_optimized!(expected, join, false); + + Ok(()) +} + +#[test] +fn multi_hash_join_key_ordering() -> Result<()> { + let left = parquet_exec(); + let alias_pairs: Vec<(String, String)> = vec![ + ("a".to_string(), "a1".to_string()), + ("b".to_string(), "b1".to_string()), + ("c".to_string(), "c1".to_string()), + ]; + let right = projection_exec_with_alias(parquet_exec(), alias_pairs); + + // Join on (a == a1 and b == b1 and c == c1) + let join_on = vec![ + ( + Arc::new(Column::new_with_schema("a", &schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("a1", &right.schema()).unwrap()) as _, + ), + ( + Arc::new(Column::new_with_schema("b", &schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("b1", &right.schema()).unwrap()) as _, + ), + ( + Arc::new(Column::new_with_schema("c", &schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("c1", &right.schema()).unwrap()) as _, + ), + ]; + let bottom_left_join = + hash_join_exec(left.clone(), right.clone(), &join_on, &JoinType::Inner); + + // Projection(a as A, a as AA, b as B, c as C) + let alias_pairs: Vec<(String, String)> = vec![ + ("a".to_string(), "A".to_string()), + ("a".to_string(), "AA".to_string()), + ("b".to_string(), "B".to_string()), + ("c".to_string(), "C".to_string()), + ]; + let bottom_left_projection = + projection_exec_with_alias(bottom_left_join, alias_pairs); + + // Join on (c == c1 and b == b1 and a == a1) + let join_on = vec![ + ( + Arc::new(Column::new_with_schema("c", &schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("c1", &right.schema()).unwrap()) as _, + ), + ( + Arc::new(Column::new_with_schema("b", &schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("b1", &right.schema()).unwrap()) as _, + ), + ( + Arc::new(Column::new_with_schema("a", &schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("a1", &right.schema()).unwrap()) as _, + ), + ]; + let bottom_right_join = + hash_join_exec(left, right.clone(), &join_on, &JoinType::Inner); + + // Join on (B == b1 and C == c and AA = a1) + let top_join_on = vec![ + ( + Arc::new( + Column::new_with_schema("B", &bottom_left_projection.schema()) + .unwrap(), + ) as _, + Arc::new( + Column::new_with_schema("b1", &bottom_right_join.schema()).unwrap(), + ) as _, + ), + ( + Arc::new( + Column::new_with_schema("C", &bottom_left_projection.schema()) + .unwrap(), + ) as _, + Arc::new( + Column::new_with_schema("c", &bottom_right_join.schema()).unwrap(), + ) as _, + ), + ( + Arc::new( + Column::new_with_schema("AA", &bottom_left_projection.schema()) + .unwrap(), + ) as _, + Arc::new( + Column::new_with_schema("a1", &bottom_right_join.schema()).unwrap(), + ) as _, + ), + ]; + + let top_join = hash_join_exec( + bottom_left_projection.clone(), + bottom_right_join, + &top_join_on, + &JoinType::Inner, + ); + + let predicate: Arc = binary( + col("c", top_join.schema().deref())?, + Operator::Gt, + lit(1i64), + top_join.schema().deref(), + )?; + + let filter_top_join: Arc = + Arc::new(FilterExec::try_new(predicate, top_join)?); + + // The bottom joins' join key ordering is adjusted based on the top join. And the top join should not introduce additional RepartitionExec + let expected = &[ + "FilterExec: c@6 > 1", + "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(B@2, b1@6), (C@3, c@2), (AA@1, a1@5)]", + "ProjectionExec: expr=[a@0 as A, a@0 as AA, b@1 as B, c@2 as C]", + "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(b@1, b1@1), (c@2, c1@2), (a@0, a1@0)]", + "RepartitionExec: partitioning=Hash([b@1, c@2, a@0], 10), input_partitions=10", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "RepartitionExec: partitioning=Hash([b1@1, c1@2, a1@0], 10), input_partitions=10", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(b@1, b1@1), (c@2, c1@2), (a@0, a1@0)]", + "RepartitionExec: partitioning=Hash([b@1, c@2, a@0], 10), input_partitions=10", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "RepartitionExec: partitioning=Hash([b1@1, c1@2, a1@0], 10), input_partitions=10", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + assert_optimized!(expected, filter_top_join.clone(), true); + assert_optimized!(expected, filter_top_join, false); + + Ok(()) +} + +#[test] +fn reorder_join_keys_to_left_input() -> Result<()> { + let left = parquet_exec(); + let alias_pairs: Vec<(String, String)> = vec![ + ("a".to_string(), "a1".to_string()), + ("b".to_string(), "b1".to_string()), + ("c".to_string(), "c1".to_string()), + ]; + let right = projection_exec_with_alias(parquet_exec(), alias_pairs); + + // Join on (a == a1 and b == b1 and c == c1) + let join_on = vec![ + ( + Arc::new(Column::new_with_schema("a", &schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("a1", &right.schema()).unwrap()) as _, + ), + ( + Arc::new(Column::new_with_schema("b", &schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("b1", &right.schema()).unwrap()) as _, + ), + ( + Arc::new(Column::new_with_schema("c", &schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("c1", &right.schema()).unwrap()) as _, + ), + ]; + + let bottom_left_join = ensure_distribution_helper( + hash_join_exec(left.clone(), right.clone(), &join_on, &JoinType::Inner), + 10, + true, + )?; + + // Projection(a as A, a as AA, b as B, c as C) + let alias_pairs: Vec<(String, String)> = vec![ + ("a".to_string(), "A".to_string()), + ("a".to_string(), "AA".to_string()), + ("b".to_string(), "B".to_string()), + ("c".to_string(), "C".to_string()), + ]; + let bottom_left_projection = + projection_exec_with_alias(bottom_left_join, alias_pairs); + + // Join on (c == c1 and b == b1 and a == a1) + let join_on = vec![ + ( + Arc::new(Column::new_with_schema("c", &schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("c1", &right.schema()).unwrap()) as _, + ), + ( + Arc::new(Column::new_with_schema("b", &schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("b1", &right.schema()).unwrap()) as _, + ), + ( + Arc::new(Column::new_with_schema("a", &schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("a1", &right.schema()).unwrap()) as _, + ), + ]; + let bottom_right_join = ensure_distribution_helper( + hash_join_exec(left, right.clone(), &join_on, &JoinType::Inner), + 10, + true, + )?; + + // Join on (B == b1 and C == c and AA = a1) + let top_join_on = vec![ + ( + Arc::new( + Column::new_with_schema("B", &bottom_left_projection.schema()) + .unwrap(), + ) as _, + Arc::new( + Column::new_with_schema("b1", &bottom_right_join.schema()).unwrap(), + ) as _, + ), + ( + Arc::new( + Column::new_with_schema("C", &bottom_left_projection.schema()) + .unwrap(), + ) as _, + Arc::new( + Column::new_with_schema("c", &bottom_right_join.schema()).unwrap(), + ) as _, + ), + ( + Arc::new( + Column::new_with_schema("AA", &bottom_left_projection.schema()) + .unwrap(), + ) as _, + Arc::new( + Column::new_with_schema("a1", &bottom_right_join.schema()).unwrap(), + ) as _, + ), + ]; + + let join_types = vec![ + JoinType::Inner, + JoinType::Left, + JoinType::Right, + JoinType::Full, + JoinType::LeftSemi, + JoinType::LeftAnti, + JoinType::RightSemi, + JoinType::RightAnti, + ]; + + for join_type in join_types { + let top_join = hash_join_exec( + bottom_left_projection.clone(), + bottom_right_join.clone(), + &top_join_on, + &join_type, + ); + let top_join_plan = + format!("HashJoinExec: mode=Partitioned, join_type={:?}, on=[(AA@1, a1@5), (B@2, b1@6), (C@3, c@2)]", &join_type); + + let reordered = reorder_join_keys_to_inputs(top_join)?; + + // The top joins' join key ordering is adjusted based on the children inputs. + let expected = &[ + top_join_plan.as_str(), + "ProjectionExec: expr=[a@0 as A, a@0 as AA, b@1 as B, c@2 as C]", + "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a1@0), (b@1, b1@1), (c@2, c1@2)]", + "RepartitionExec: partitioning=Hash([a@0, b@1, c@2], 10), input_partitions=10", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "RepartitionExec: partitioning=Hash([a1@0, b1@1, c1@2], 10), input_partitions=10", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@2, c1@2), (b@1, b1@1), (a@0, a1@0)]", + "RepartitionExec: partitioning=Hash([c@2, b@1, a@0], 10), input_partitions=10", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "RepartitionExec: partitioning=Hash([c1@2, b1@1, a1@0], 10), input_partitions=10", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + + assert_plan_txt!(expected, reordered); + } + + Ok(()) +} + +#[test] +fn reorder_join_keys_to_right_input() -> Result<()> { + let left = parquet_exec(); + let alias_pairs: Vec<(String, String)> = vec![ + ("a".to_string(), "a1".to_string()), + ("b".to_string(), "b1".to_string()), + ("c".to_string(), "c1".to_string()), + ]; + let right = projection_exec_with_alias(parquet_exec(), alias_pairs); + + // Join on (a == a1 and b == b1) + let join_on = vec![ + ( + Arc::new(Column::new_with_schema("a", &schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("a1", &right.schema()).unwrap()) as _, + ), + ( + Arc::new(Column::new_with_schema("b", &schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("b1", &right.schema()).unwrap()) as _, + ), + ]; + let bottom_left_join = ensure_distribution_helper( + hash_join_exec(left.clone(), right.clone(), &join_on, &JoinType::Inner), + 10, + true, + )?; + + // Projection(a as A, a as AA, b as B, c as C) + let alias_pairs: Vec<(String, String)> = vec![ + ("a".to_string(), "A".to_string()), + ("a".to_string(), "AA".to_string()), + ("b".to_string(), "B".to_string()), + ("c".to_string(), "C".to_string()), + ]; + let bottom_left_projection = + projection_exec_with_alias(bottom_left_join, alias_pairs); + + // Join on (c == c1 and b == b1 and a == a1) + let join_on = vec![ + ( + Arc::new(Column::new_with_schema("c", &schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("c1", &right.schema()).unwrap()) as _, + ), + ( + Arc::new(Column::new_with_schema("b", &schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("b1", &right.schema()).unwrap()) as _, + ), + ( + Arc::new(Column::new_with_schema("a", &schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("a1", &right.schema()).unwrap()) as _, + ), + ]; + let bottom_right_join = ensure_distribution_helper( + hash_join_exec(left, right.clone(), &join_on, &JoinType::Inner), + 10, + true, + )?; + + // Join on (B == b1 and C == c and AA = a1) + let top_join_on = vec![ + ( + Arc::new( + Column::new_with_schema("B", &bottom_left_projection.schema()) + .unwrap(), + ) as _, + Arc::new( + Column::new_with_schema("b1", &bottom_right_join.schema()).unwrap(), + ) as _, + ), + ( + Arc::new( + Column::new_with_schema("C", &bottom_left_projection.schema()) + .unwrap(), + ) as _, + Arc::new( + Column::new_with_schema("c", &bottom_right_join.schema()).unwrap(), + ) as _, + ), + ( + Arc::new( + Column::new_with_schema("AA", &bottom_left_projection.schema()) + .unwrap(), + ) as _, + Arc::new( + Column::new_with_schema("a1", &bottom_right_join.schema()).unwrap(), + ) as _, + ), + ]; + + let join_types = vec![ + JoinType::Inner, + JoinType::Left, + JoinType::Right, + JoinType::Full, + JoinType::LeftSemi, + JoinType::LeftAnti, + JoinType::RightSemi, + JoinType::RightAnti, + ]; + + for join_type in join_types { + let top_join = hash_join_exec( + bottom_left_projection.clone(), + bottom_right_join.clone(), + &top_join_on, + &join_type, + ); + let top_join_plan = + format!("HashJoinExec: mode=Partitioned, join_type={:?}, on=[(C@3, c@2), (B@2, b1@6), (AA@1, a1@5)]", &join_type); + + let reordered = reorder_join_keys_to_inputs(top_join)?; + + // The top joins' join key ordering is adjusted based on the children inputs. + let expected = &[ + top_join_plan.as_str(), + "ProjectionExec: expr=[a@0 as A, a@0 as AA, b@1 as B, c@2 as C]", + "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a1@0), (b@1, b1@1)]", + "RepartitionExec: partitioning=Hash([a@0, b@1], 10), input_partitions=10", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "RepartitionExec: partitioning=Hash([a1@0, b1@1], 10), input_partitions=10", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@2, c1@2), (b@1, b1@1), (a@0, a1@0)]", + "RepartitionExec: partitioning=Hash([c@2, b@1, a@0], 10), input_partitions=10", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "RepartitionExec: partitioning=Hash([c1@2, b1@1, a1@0], 10), input_partitions=10", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + + assert_plan_txt!(expected, reordered); + } + + Ok(()) +} + +#[test] +fn multi_smj_joins() -> Result<()> { + let left = parquet_exec(); + let alias_pairs: Vec<(String, String)> = vec![ + ("a".to_string(), "a1".to_string()), + ("b".to_string(), "b1".to_string()), + ("c".to_string(), "c1".to_string()), + ("d".to_string(), "d1".to_string()), + ("e".to_string(), "e1".to_string()), + ]; + let right = projection_exec_with_alias(parquet_exec(), alias_pairs); + + // SortMergeJoin does not support RightSemi and RightAnti join now + let join_types = vec![ + JoinType::Inner, + JoinType::Left, + JoinType::Right, + JoinType::Full, + JoinType::LeftSemi, + JoinType::LeftAnti, + ]; + + // Join on (a == b1) + let join_on = vec![( + Arc::new(Column::new_with_schema("a", &schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("b1", &right.schema()).unwrap()) as _, + )]; + + for join_type in join_types { + let join = + sort_merge_join_exec(left.clone(), right.clone(), &join_on, &join_type); + let join_plan = + format!("SortMergeJoin: join_type={join_type}, on=[(a@0, b1@1)]"); + + // Top join on (a == c) + let top_join_on = vec![( + Arc::new(Column::new_with_schema("a", &join.schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("c", &schema()).unwrap()) as _, + )]; + let top_join = sort_merge_join_exec( + join.clone(), + parquet_exec(), + &top_join_on, + &join_type, + ); + let top_join_plan = + format!("SortMergeJoin: join_type={join_type}, on=[(a@0, c@2)]"); + + let expected = match join_type { + // Should include 6 RepartitionExecs (3 hash, 3 round-robin), 3 SortExecs + JoinType::Inner | JoinType::Left | JoinType::LeftSemi | JoinType::LeftAnti => + vec![ + top_join_plan.as_str(), + join_plan.as_str(), + "SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", + "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "SortExec: expr=[b1@1 ASC], preserve_partitioning=[true]", + "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "SortExec: expr=[c@2 ASC], preserve_partitioning=[true]", + "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ], + // Should include 7 RepartitionExecs (4 hash, 3 round-robin), 4 SortExecs + // Since ordering of the left child is not preserved after SortMergeJoin + // when mode is Right, RightSemi, RightAnti, Full + // - We need to add one additional SortExec after SortMergeJoin in contrast the test cases + // when mode is Inner, Left, LeftSemi, LeftAnti + // Similarly, since partitioning of the left side is not preserved + // when mode is Right, RightSemi, RightAnti, Full + // - We need to add one additional Hash Repartition after SortMergeJoin in contrast the test + // cases when mode is Inner, Left, LeftSemi, LeftAnti + _ => vec![ + top_join_plan.as_str(), + // Below 2 operators are differences introduced, when join mode is changed + "SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", + "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", + join_plan.as_str(), + "SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", + "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "SortExec: expr=[b1@1 ASC], preserve_partitioning=[true]", + "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "SortExec: expr=[c@2 ASC], preserve_partitioning=[true]", + "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ], + }; + assert_optimized!(expected, top_join.clone(), true, true); + + let expected_first_sort_enforcement = match join_type { + // Should include 6 RepartitionExecs (3 hash, 3 round-robin), 3 SortExecs + JoinType::Inner | JoinType::Left | JoinType::LeftSemi | JoinType::LeftAnti => + vec![ + top_join_plan.as_str(), + join_plan.as_str(), + "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10, preserve_order=true, sort_exprs=b1@1 ASC", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "SortExec: expr=[b1@1 ASC], preserve_partitioning=[false]", + "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10, preserve_order=true, sort_exprs=c@2 ASC", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ], + // Should include 8 RepartitionExecs (4 hash, 8 round-robin), 4 SortExecs + // Since ordering of the left child is not preserved after SortMergeJoin + // when mode is Right, RightSemi, RightAnti, Full + // - We need to add one additional SortExec after SortMergeJoin in contrast the test cases + // when mode is Inner, Left, LeftSemi, LeftAnti + // Similarly, since partitioning of the left side is not preserved + // when mode is Right, RightSemi, RightAnti, Full + // - We need to add one additional Hash Repartition and Roundrobin repartition after + // SortMergeJoin in contrast the test cases when mode is Inner, Left, LeftSemi, LeftAnti + _ => vec![ + top_join_plan.as_str(), + // Below 4 operators are differences introduced, when join mode is changed + "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", + "CoalescePartitionsExec", + join_plan.as_str(), + "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10, preserve_order=true, sort_exprs=b1@1 ASC", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "SortExec: expr=[b1@1 ASC], preserve_partitioning=[false]", + "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10, preserve_order=true, sort_exprs=c@2 ASC", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ], + }; + assert_optimized!(expected_first_sort_enforcement, top_join, false, true); + + match join_type { + JoinType::Inner | JoinType::Left | JoinType::Right | JoinType::Full => { + // This time we use (b1 == c) for top join + // Join on (b1 == c) + let top_join_on = vec![( + Arc::new(Column::new_with_schema("b1", &join.schema()).unwrap()) + as _, + Arc::new(Column::new_with_schema("c", &schema()).unwrap()) as _, + )]; + let top_join = sort_merge_join_exec( + join, + parquet_exec(), + &top_join_on, + &join_type, + ); + let top_join_plan = + format!("SortMergeJoin: join_type={join_type}, on=[(b1@6, c@2)]"); + + let expected = match join_type { + // Should include 6 RepartitionExecs(3 hash, 3 round-robin) and 3 SortExecs + JoinType::Inner | JoinType::Right => vec![ + top_join_plan.as_str(), + join_plan.as_str(), + "SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", + "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "SortExec: expr=[b1@1 ASC], preserve_partitioning=[true]", + "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "SortExec: expr=[c@2 ASC], preserve_partitioning=[true]", + "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ], + // Should include 7 RepartitionExecs (4 hash, 3 round-robin) and 4 SortExecs + JoinType::Left | JoinType::Full => vec![ + top_join_plan.as_str(), + "SortExec: expr=[b1@6 ASC], preserve_partitioning=[true]", + "RepartitionExec: partitioning=Hash([b1@6], 10), input_partitions=10", + join_plan.as_str(), + "SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", + "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "SortExec: expr=[b1@1 ASC], preserve_partitioning=[true]", + "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "SortExec: expr=[c@2 ASC], preserve_partitioning=[true]", + "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ], + // this match arm cannot be reached + _ => unreachable!() + }; + assert_optimized!(expected, top_join.clone(), true, true); + + let expected_first_sort_enforcement = match join_type { + // Should include 6 RepartitionExecs (3 of them preserves order) and 3 SortExecs + JoinType::Inner | JoinType::Right => vec![ + top_join_plan.as_str(), + join_plan.as_str(), + "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10, preserve_order=true, sort_exprs=b1@1 ASC", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "SortExec: expr=[b1@1 ASC], preserve_partitioning=[false]", + "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10, preserve_order=true, sort_exprs=c@2 ASC", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ], + // Should include 8 RepartitionExecs (4 of them preserves order) and 4 SortExecs + JoinType::Left | JoinType::Full => vec![ + top_join_plan.as_str(), + "RepartitionExec: partitioning=Hash([b1@6], 10), input_partitions=10, preserve_order=true, sort_exprs=b1@6 ASC", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "SortExec: expr=[b1@6 ASC], preserve_partitioning=[false]", + "CoalescePartitionsExec", + join_plan.as_str(), + "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10, preserve_order=true, sort_exprs=b1@1 ASC", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "SortExec: expr=[b1@1 ASC], preserve_partitioning=[false]", + "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10, preserve_order=true, sort_exprs=c@2 ASC", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ], + // this match arm cannot be reached + _ => unreachable!() + }; + assert_optimized!( + expected_first_sort_enforcement, + top_join, + false, + true + ); + } + _ => {} + } + } + + Ok(()) +} + +#[test] +fn smj_join_key_ordering() -> Result<()> { + // group by (a as a1, b as b1) + let left = aggregate_exec_with_alias( + parquet_exec(), + vec![ + ("a".to_string(), "a1".to_string()), + ("b".to_string(), "b1".to_string()), + ], + ); + //Projection(a1 as a3, b1 as b3) + let alias_pairs: Vec<(String, String)> = vec![ + ("a1".to_string(), "a3".to_string()), + ("b1".to_string(), "b3".to_string()), + ]; + let left = projection_exec_with_alias(left, alias_pairs); + + // group by (b, a) + let right = aggregate_exec_with_alias( + parquet_exec(), + vec![ + ("b".to_string(), "b".to_string()), + ("a".to_string(), "a".to_string()), + ], + ); + + //Projection(a as a2, b as b2) + let alias_pairs: Vec<(String, String)> = vec![ + ("a".to_string(), "a2".to_string()), + ("b".to_string(), "b2".to_string()), + ]; + let right = projection_exec_with_alias(right, alias_pairs); + + // Join on (b3 == b2 && a3 == a2) + let join_on = vec![ + ( + Arc::new(Column::new_with_schema("b3", &left.schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("b2", &right.schema()).unwrap()) as _, + ), + ( + Arc::new(Column::new_with_schema("a3", &left.schema()).unwrap()) as _, + Arc::new(Column::new_with_schema("a2", &right.schema()).unwrap()) as _, + ), + ]; + let join = sort_merge_join_exec(left, right.clone(), &join_on, &JoinType::Inner); + + // Only two RepartitionExecs added + let expected = &[ + "SortMergeJoin: join_type=Inner, on=[(b3@1, b2@1), (a3@0, a2@0)]", + "SortExec: expr=[b3@1 ASC, a3@0 ASC], preserve_partitioning=[true]", + "ProjectionExec: expr=[a1@0 as a3, b1@1 as b3]", + "ProjectionExec: expr=[a1@1 as a1, b1@0 as b1]", + "AggregateExec: mode=FinalPartitioned, gby=[b1@0 as b1, a1@1 as a1], aggr=[]", + "RepartitionExec: partitioning=Hash([b1@0, a1@1], 10), input_partitions=10", + "AggregateExec: mode=Partial, gby=[b@1 as b1, a@0 as a1], aggr=[]", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "SortExec: expr=[b2@1 ASC, a2@0 ASC], preserve_partitioning=[true]", + "ProjectionExec: expr=[a@1 as a2, b@0 as b2]", + "AggregateExec: mode=FinalPartitioned, gby=[b@0 as b, a@1 as a], aggr=[]", + "RepartitionExec: partitioning=Hash([b@0, a@1], 10), input_partitions=10", + "AggregateExec: mode=Partial, gby=[b@1 as b, a@0 as a], aggr=[]", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + assert_optimized!(expected, join.clone(), true, true); + + let expected_first_sort_enforcement = &[ + "SortMergeJoin: join_type=Inner, on=[(b3@1, b2@1), (a3@0, a2@0)]", + "RepartitionExec: partitioning=Hash([b3@1, a3@0], 10), input_partitions=10, preserve_order=true, sort_exprs=b3@1 ASC, a3@0 ASC", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "SortExec: expr=[b3@1 ASC, a3@0 ASC], preserve_partitioning=[false]", + "CoalescePartitionsExec", + "ProjectionExec: expr=[a1@0 as a3, b1@1 as b3]", + "ProjectionExec: expr=[a1@1 as a1, b1@0 as b1]", + "AggregateExec: mode=FinalPartitioned, gby=[b1@0 as b1, a1@1 as a1], aggr=[]", + "RepartitionExec: partitioning=Hash([b1@0, a1@1], 10), input_partitions=10", + "AggregateExec: mode=Partial, gby=[b@1 as b1, a@0 as a1], aggr=[]", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "RepartitionExec: partitioning=Hash([b2@1, a2@0], 10), input_partitions=10, preserve_order=true, sort_exprs=b2@1 ASC, a2@0 ASC", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "SortExec: expr=[b2@1 ASC, a2@0 ASC], preserve_partitioning=[false]", + "CoalescePartitionsExec", + "ProjectionExec: expr=[a@1 as a2, b@0 as b2]", + "AggregateExec: mode=FinalPartitioned, gby=[b@0 as b, a@1 as a], aggr=[]", + "RepartitionExec: partitioning=Hash([b@0, a@1], 10), input_partitions=10", + "AggregateExec: mode=Partial, gby=[b@1 as b, a@0 as a], aggr=[]", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + assert_optimized!(expected_first_sort_enforcement, join, false, true); + + Ok(()) +} + +#[test] +fn merge_does_not_need_sort() -> Result<()> { + // see https://github.com/apache/datafusion/issues/4331 + let schema = schema(); + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { + expr: col("a", &schema).unwrap(), + options: SortOptions::default(), + }]); + + // Scan some sorted parquet files + let exec = parquet_exec_multiple_sorted(vec![sort_key.clone()]); + + // CoalesceBatchesExec to mimic behavior after a filter + let exec = Arc::new(CoalesceBatchesExec::new(exec, 4096)); + + // Merge from multiple parquet files and keep the data sorted + let exec: Arc = + Arc::new(SortPreservingMergeExec::new(sort_key, exec)); + + // The optimizer should not add an additional SortExec as the + // data is already sorted + let expected = &[ + "SortPreservingMergeExec: [a@0 ASC]", + "CoalesceBatchesExec: target_batch_size=4096", + "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", + ]; + assert_optimized!(expected, exec, true); + + // In this case preserving ordering through order preserving operators is not desirable + // (according to flag: PREFER_EXISTING_SORT) + // hence in this case ordering lost during CoalescePartitionsExec and re-introduced with + // SortExec at the top. + let expected = &[ + "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", + "CoalescePartitionsExec", + "CoalesceBatchesExec: target_batch_size=4096", + "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", + ]; + assert_optimized!(expected, exec, false); + + Ok(()) +} + +#[test] +fn union_to_interleave() -> Result<()> { + // group by (a as a1) + let left = aggregate_exec_with_alias( + parquet_exec(), + vec![("a".to_string(), "a1".to_string())], + ); + // group by (a as a2) + let right = aggregate_exec_with_alias( + parquet_exec(), + vec![("a".to_string(), "a1".to_string())], + ); + + // Union + let plan = Arc::new(UnionExec::new(vec![left, right])); + + // final agg + let plan = + aggregate_exec_with_alias(plan, vec![("a1".to_string(), "a2".to_string())]); + + // Only two RepartitionExecs added, no final RepartitionExec required + let expected = &[ + "AggregateExec: mode=FinalPartitioned, gby=[a2@0 as a2], aggr=[]", + "AggregateExec: mode=Partial, gby=[a1@0 as a2], aggr=[]", + "InterleaveExec", + "AggregateExec: mode=FinalPartitioned, gby=[a1@0 as a1], aggr=[]", + "RepartitionExec: partitioning=Hash([a1@0], 10), input_partitions=10", + "AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[]", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "AggregateExec: mode=FinalPartitioned, gby=[a1@0 as a1], aggr=[]", + "RepartitionExec: partitioning=Hash([a1@0], 10), input_partitions=10", + "AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[]", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + assert_optimized!(expected, plan.clone(), true); + assert_optimized!(expected, plan.clone(), false); + + Ok(()) +} + +#[test] +fn union_not_to_interleave() -> Result<()> { + // group by (a as a1) + let left = aggregate_exec_with_alias( + parquet_exec(), + vec![("a".to_string(), "a1".to_string())], + ); + // group by (a as a2) + let right = aggregate_exec_with_alias( + parquet_exec(), + vec![("a".to_string(), "a1".to_string())], + ); + + // Union + let plan = Arc::new(UnionExec::new(vec![left, right])); + + // final agg + let plan = + aggregate_exec_with_alias(plan, vec![("a1".to_string(), "a2".to_string())]); + + // Only two RepartitionExecs added, no final RepartitionExec required + let expected = &[ + "AggregateExec: mode=FinalPartitioned, gby=[a2@0 as a2], aggr=[]", + "RepartitionExec: partitioning=Hash([a2@0], 10), input_partitions=20", + "AggregateExec: mode=Partial, gby=[a1@0 as a2], aggr=[]", + "UnionExec", + "AggregateExec: mode=FinalPartitioned, gby=[a1@0 as a1], aggr=[]", + "RepartitionExec: partitioning=Hash([a1@0], 10), input_partitions=10", + "AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[]", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "AggregateExec: mode=FinalPartitioned, gby=[a1@0 as a1], aggr=[]", + "RepartitionExec: partitioning=Hash([a1@0], 10), input_partitions=10", + "AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[]", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + // no sort in the plan but since we need it as a parameter, make it default false + let prefer_existing_sort = false; + let first_enforce_distribution = true; + let prefer_existing_union = true; + + assert_optimized!( + expected, + plan.clone(), + first_enforce_distribution, + prefer_existing_sort, + prefer_existing_union + ); + assert_optimized!( + expected, + plan, + !first_enforce_distribution, + prefer_existing_sort, + prefer_existing_union + ); + + Ok(()) +} + +#[test] +fn added_repartition_to_single_partition() -> Result<()> { + let alias = vec![("a".to_string(), "a".to_string())]; + let plan = aggregate_exec_with_alias(parquet_exec(), alias); + + let expected = [ + "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", + "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", + "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + assert_optimized!(expected, plan.clone(), true); + assert_optimized!(expected, plan, false); + + Ok(()) +} + +#[test] +fn repartition_deepest_node() -> Result<()> { + let alias = vec![("a".to_string(), "a".to_string())]; + let plan = aggregate_exec_with_alias(filter_exec(parquet_exec()), alias); + + let expected = &[ + "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", + "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", + "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", + "FilterExec: c@2 = 0", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + assert_optimized!(expected, plan.clone(), true); + assert_optimized!(expected, plan, false); + + Ok(()) +} + +#[test] + +fn repartition_unsorted_limit() -> Result<()> { + let plan = limit_exec(filter_exec(parquet_exec())); + + let expected = &[ + "GlobalLimitExec: skip=0, fetch=100", + "CoalescePartitionsExec", + "LocalLimitExec: fetch=100", + "FilterExec: c@2 = 0", + // nothing sorts the data, so the local limit doesn't require sorted data either + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + + assert_optimized!(expected, plan.clone(), true); + assert_optimized!(expected, plan, false); + + Ok(()) +} + +#[test] +fn repartition_sorted_limit() -> Result<()> { + let schema = schema(); + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { + expr: col("c", &schema).unwrap(), + options: SortOptions::default(), + }]); + let plan = limit_exec(sort_exec(sort_key, parquet_exec(), false)); + + let expected = &[ + "GlobalLimitExec: skip=0, fetch=100", + "LocalLimitExec: fetch=100", + // data is sorted so can't repartition here + "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + assert_optimized!(expected, plan.clone(), true); + assert_optimized!(expected, plan, false); + + Ok(()) +} + +#[test] +fn repartition_sorted_limit_with_filter() -> Result<()> { + let schema = schema(); + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { + expr: col("c", &schema).unwrap(), + options: SortOptions::default(), + }]); + let plan = sort_required_exec_with_req( + filter_exec(sort_exec(sort_key.clone(), parquet_exec(), false)), + sort_key, + ); + + let expected = &[ + "SortRequiredExec: [c@2 ASC]", + "FilterExec: c@2 = 0", + // We can use repartition here, ordering requirement by SortRequiredExec + // is still satisfied. + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + + assert_optimized!(expected, plan.clone(), true); + assert_optimized!(expected, plan, false); + + Ok(()) +} + +#[test] +fn repartition_ignores_limit() -> Result<()> { + let alias = vec![("a".to_string(), "a".to_string())]; + let plan = aggregate_exec_with_alias( + limit_exec(filter_exec(limit_exec(parquet_exec()))), + alias, + ); + + let expected = &[ + "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", + "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", + "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "GlobalLimitExec: skip=0, fetch=100", + "CoalescePartitionsExec", + "LocalLimitExec: fetch=100", + "FilterExec: c@2 = 0", + // repartition should happen prior to the filter to maximize parallelism + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "GlobalLimitExec: skip=0, fetch=100", + "LocalLimitExec: fetch=100", + // Expect no repartition to happen for local limit + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + assert_optimized!(expected, plan.clone(), true); + assert_optimized!(expected, plan, false); + + Ok(()) +} + +#[test] +fn repartition_ignores_union() -> Result<()> { + let plan = union_exec(vec![parquet_exec(); 5]); + + let expected = &[ + "UnionExec", + // Expect no repartition of ParquetExec + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + + assert_optimized!(expected, plan.clone(), true); + assert_optimized!(expected, plan, false); + + Ok(()) +} + +#[test] +fn repartition_through_sort_preserving_merge() -> Result<()> { + // sort preserving merge with non-sorted input + let schema = schema(); + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { + expr: col("c", &schema).unwrap(), + options: SortOptions::default(), + }]); + let plan = sort_preserving_merge_exec(sort_key, parquet_exec()); + + // need resort as the data was not sorted correctly + let expected = &[ + "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + assert_optimized!(expected, plan.clone(), true); + assert_optimized!(expected, plan, false); + + Ok(()) +} + +#[test] +fn repartition_ignores_sort_preserving_merge() -> Result<()> { + // sort preserving merge already sorted input, + let schema = schema(); + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { + expr: col("c", &schema).unwrap(), + options: SortOptions::default(), + }]); + let plan = sort_preserving_merge_exec( + sort_key.clone(), + parquet_exec_multiple_sorted(vec![sort_key]), + ); + + // should not sort (as the data was already sorted) + // should not repartition, since increased parallelism is not beneficial for SortPReservingMerge + let expected = &[ + "SortPreservingMergeExec: [c@2 ASC]", + "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + ]; + + assert_optimized!(expected, plan.clone(), true); + + let expected = &[ + "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", + "CoalescePartitionsExec", + "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + ]; + assert_optimized!(expected, plan, false); + + Ok(()) +} + +#[test] +fn repartition_ignores_sort_preserving_merge_with_union() -> Result<()> { + // 2 sorted parquet files unioned (partitions are concatenated, sort is preserved) + let schema = schema(); + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { + expr: col("c", &schema).unwrap(), + options: SortOptions::default(), + }]); + let input = union_exec(vec![parquet_exec_with_sort(vec![sort_key.clone()]); 2]); + let plan = sort_preserving_merge_exec(sort_key, input); + + // should not repartition / sort (as the data was already sorted) + let expected = &[ + "SortPreservingMergeExec: [c@2 ASC]", + "UnionExec", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + ]; + + assert_optimized!(expected, plan.clone(), true); + + let expected = &[ + "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", + "CoalescePartitionsExec", + "UnionExec", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + ]; + assert_optimized!(expected, plan, false); + + Ok(()) +} + +#[test] +fn repartition_does_not_destroy_sort() -> Result<()> { + // SortRequired + // Parquet(sorted) + let schema = schema(); + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { + expr: col("d", &schema).unwrap(), + options: SortOptions::default(), + }]); + let plan = sort_required_exec_with_req( + filter_exec(parquet_exec_with_sort(vec![sort_key.clone()])), + sort_key, + ); + + // during repartitioning ordering is preserved + let expected = &[ + "SortRequiredExec: [d@3 ASC]", + "FilterExec: c@2 = 0", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[d@3 ASC]", + ]; + + assert_optimized!(expected, plan.clone(), true, true); + assert_optimized!(expected, plan, false, true); + + Ok(()) +} + +#[test] +fn repartition_does_not_destroy_sort_more_complex() -> Result<()> { + // model a more complicated scenario where one child of a union can be repartitioned for performance + // but the other can not be + // + // Union + // SortRequired + // Parquet(sorted) + // Filter + // Parquet(unsorted) + + let schema = schema(); + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { + expr: col("c", &schema).unwrap(), + options: SortOptions::default(), + }]); + let input1 = sort_required_exec_with_req( + parquet_exec_with_sort(vec![sort_key.clone()]), + sort_key, + ); + let input2 = filter_exec(parquet_exec()); + let plan = union_exec(vec![input1, input2]); + + // should not repartition below the SortRequired as that + // branch doesn't benefit from increased parallelism + let expected = &[ + "UnionExec", + // union input 1: no repartitioning + "SortRequiredExec: [c@2 ASC]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + // union input 2: should repartition + "FilterExec: c@2 = 0", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + assert_optimized!(expected, plan.clone(), true); + assert_optimized!(expected, plan, false); + + Ok(()) +} + +#[test] +fn repartition_transitively_with_projection() -> Result<()> { + let schema = schema(); + let proj_exprs = vec![( + Arc::new(BinaryExpr::new( + col("a", &schema).unwrap(), + Operator::Plus, + col("b", &schema).unwrap(), + )) as Arc, + "sum".to_string(), + )]; + // non sorted input + let proj = Arc::new(ProjectionExec::try_new(proj_exprs, parquet_exec())?); + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { + expr: col("sum", &proj.schema()).unwrap(), + options: SortOptions::default(), + }]); + let plan = sort_preserving_merge_exec(sort_key, proj); + + let expected = &[ + "SortPreservingMergeExec: [sum@0 ASC]", + "SortExec: expr=[sum@0 ASC], preserve_partitioning=[true]", + // Since this projection is not trivial, increasing parallelism is beneficial + "ProjectionExec: expr=[a@0 + b@1 as sum]", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + + assert_optimized!(expected, plan.clone(), true); + + let expected_first_sort_enforcement = &[ + "SortExec: expr=[sum@0 ASC], preserve_partitioning=[false]", + "CoalescePartitionsExec", + // Since this projection is not trivial, increasing parallelism is beneficial + "ProjectionExec: expr=[a@0 + b@1 as sum]", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + assert_optimized!(expected_first_sort_enforcement, plan, false); + + Ok(()) +} + +#[test] +fn repartition_ignores_transitively_with_projection() -> Result<()> { + let schema = schema(); + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { + expr: col("c", &schema).unwrap(), + options: SortOptions::default(), + }]); + let alias = vec![ + ("a".to_string(), "a".to_string()), + ("b".to_string(), "b".to_string()), + ("c".to_string(), "c".to_string()), + ]; + // sorted input + let plan = sort_required_exec_with_req( + projection_exec_with_alias( + parquet_exec_multiple_sorted(vec![sort_key.clone()]), + alias, + ), + sort_key, + ); + + let expected = &[ + "SortRequiredExec: [c@2 ASC]", + // Since this projection is trivial, increasing parallelism is not beneficial + "ProjectionExec: expr=[a@0 as a, b@1 as b, c@2 as c]", + "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + ]; + assert_optimized!(expected, plan.clone(), true); + assert_optimized!(expected, plan, false); + + Ok(()) +} + +#[test] +fn repartition_transitively_past_sort_with_projection() -> Result<()> { + let schema = schema(); + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { + expr: col("c", &schema).unwrap(), + options: SortOptions::default(), + }]); + let alias = vec![ + ("a".to_string(), "a".to_string()), + ("b".to_string(), "b".to_string()), + ("c".to_string(), "c".to_string()), + ]; + let plan = sort_preserving_merge_exec( + sort_key.clone(), + sort_exec( + sort_key, + projection_exec_with_alias(parquet_exec(), alias), + true, + ), + ); + + let expected = &[ + "SortExec: expr=[c@2 ASC], preserve_partitioning=[true]", + // Since this projection is trivial, increasing parallelism is not beneficial + "ProjectionExec: expr=[a@0 as a, b@1 as b, c@2 as c]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + assert_optimized!(expected, plan.clone(), true); + assert_optimized!(expected, plan, false); + + Ok(()) +} + +#[test] +fn repartition_transitively_past_sort_with_filter() -> Result<()> { + let schema = schema(); + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { + expr: col("a", &schema).unwrap(), + options: SortOptions::default(), + }]); + let plan = sort_exec(sort_key, filter_exec(parquet_exec()), false); + + let expected = &[ + "SortPreservingMergeExec: [a@0 ASC]", + "SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", + // Expect repartition on the input to the sort (as it can benefit from additional parallelism) + "FilterExec: c@2 = 0", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + + assert_optimized!(expected, plan.clone(), true); + + let expected_first_sort_enforcement = &[ + "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", + "CoalescePartitionsExec", + "FilterExec: c@2 = 0", + // Expect repartition on the input of the filter (as it can benefit from additional parallelism) + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + assert_optimized!(expected_first_sort_enforcement, plan, false); + + Ok(()) +} + +#[test] +#[cfg(feature = "parquet")] +fn repartition_transitively_past_sort_with_projection_and_filter() -> Result<()> { + let schema = schema(); + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { + expr: col("a", &schema).unwrap(), + options: SortOptions::default(), + }]); + let plan = sort_exec( + sort_key, + projection_exec_with_alias( + filter_exec(parquet_exec()), + vec![ + ("a".to_string(), "a".to_string()), + ("b".to_string(), "b".to_string()), + ("c".to_string(), "c".to_string()), + ], + ), + false, + ); + + let expected = &[ + "SortPreservingMergeExec: [a@0 ASC]", + // Expect repartition on the input to the sort (as it can benefit from additional parallelism) + "SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", + "ProjectionExec: expr=[a@0 as a, b@1 as b, c@2 as c]", + "FilterExec: c@2 = 0", + // repartition is lowest down + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + + assert_optimized!(expected, plan.clone(), true); + + let expected_first_sort_enforcement = &[ + "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", + "CoalescePartitionsExec", + "ProjectionExec: expr=[a@0 as a, b@1 as b, c@2 as c]", + "FilterExec: c@2 = 0", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + assert_optimized!(expected_first_sort_enforcement, plan, false); + + Ok(()) +} + +#[test] +fn parallelization_single_partition() -> Result<()> { + let alias = vec![("a".to_string(), "a".to_string())]; + let plan_parquet = aggregate_exec_with_alias(parquet_exec(), alias.clone()); + let plan_csv = aggregate_exec_with_alias(csv_exec(), alias); + + let expected_parquet = [ + "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", + "RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2", + "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", + "ParquetExec: file_groups={2 groups: [[x:0..50], [x:50..100]]}, projection=[a, b, c, d, e]", + ]; + let expected_csv = [ + "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", + "RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2", + "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", + "CsvExec: file_groups={2 groups: [[x:0..50], [x:50..100]]}, projection=[a, b, c, d, e], has_header=false", + ]; + assert_optimized!(expected_parquet, plan_parquet, true, false, 2, true, 10); + assert_optimized!(expected_csv, plan_csv, true, false, 2, true, 10); + + Ok(()) +} + +#[test] +fn parallelization_multiple_files() -> Result<()> { + let schema = schema(); + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { + expr: col("a", &schema).unwrap(), + options: SortOptions::default(), + }]); + + let plan = filter_exec(parquet_exec_multiple_sorted(vec![sort_key.clone()])); + let plan = sort_required_exec_with_req(plan, sort_key); + + // The groups must have only contiguous ranges of rows from the same file + // if any group has rows from multiple files, the data is no longer sorted destroyed + // https://github.com/apache/datafusion/issues/8451 + let expected = [ + "SortRequiredExec: [a@0 ASC]", + "FilterExec: c@2 = 0", + "ParquetExec: file_groups={3 groups: [[x:0..50], [y:0..100], [x:50..100]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", ]; + let target_partitions = 3; + let repartition_size = 1; + assert_optimized!( + expected, + plan, + true, + true, + target_partitions, + true, + repartition_size, + false + ); + + let expected = [ + "SortRequiredExec: [a@0 ASC]", + "FilterExec: c@2 = 0", + "ParquetExec: file_groups={8 groups: [[x:0..25], [y:0..25], [x:25..50], [y:25..50], [x:50..75], [y:50..75], [x:75..100], [y:75..100]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", + ]; + let target_partitions = 8; + let repartition_size = 1; + assert_optimized!( + expected, + plan, + true, + true, + target_partitions, + true, + repartition_size, + false + ); + + Ok(()) +} + +#[test] +/// CsvExec on compressed csv file will not be partitioned +/// (Not able to decompress chunked csv file) +fn parallelization_compressed_csv() -> Result<()> { + let compression_types = [ + FileCompressionType::GZIP, + FileCompressionType::BZIP2, + FileCompressionType::XZ, + FileCompressionType::ZSTD, + FileCompressionType::UNCOMPRESSED, + ]; + + let expected_not_partitioned = [ + "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", + "RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2", + "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", + "RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", + "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + ]; + + let expected_partitioned = [ + "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", + "RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2", + "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", + "CsvExec: file_groups={2 groups: [[x:0..50], [x:50..100]]}, projection=[a, b, c, d, e], has_header=false", + ]; + + for compression_type in compression_types { + let expected = if compression_type.is_compressed() { + &expected_not_partitioned[..] + } else { + &expected_partitioned[..] + }; + + let plan = aggregate_exec_with_alias( + Arc::new( + CsvExec::builder( + FileScanConfig::new( + ObjectStoreUrl::parse("test:///").unwrap(), + schema(), + ) + .with_file(PartitionedFile::new("x".to_string(), 100)), + ) + .with_has_header(false) + .with_delimeter(b',') + .with_quote(b'"') + .with_escape(None) + .with_comment(None) + .with_newlines_in_values(false) + .with_file_compression_type(compression_type) + .build(), + ), + vec![("a".to_string(), "a".to_string())], + ); + assert_optimized!(expected, plan, true, false, 2, true, 10, false); + } + Ok(()) +} + +#[test] +fn parallelization_two_partitions() -> Result<()> { + let alias = vec![("a".to_string(), "a".to_string())]; + let plan_parquet = + aggregate_exec_with_alias(parquet_exec_multiple(), alias.clone()); + let plan_csv = aggregate_exec_with_alias(csv_exec_multiple(), alias); + + let expected_parquet = [ + "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", + "RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2", + "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", + // Plan already has two partitions + "ParquetExec: file_groups={2 groups: [[x:0..100], [y:0..100]]}, projection=[a, b, c, d, e]", + ]; + let expected_csv = [ + "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", + "RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2", + "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", + // Plan already has two partitions + "CsvExec: file_groups={2 groups: [[x:0..100], [y:0..100]]}, projection=[a, b, c, d, e], has_header=false", + ]; + assert_optimized!(expected_parquet, plan_parquet, true, false, 2, true, 10); + assert_optimized!(expected_csv, plan_csv, true, false, 2, true, 10); + Ok(()) +} + +#[test] +fn parallelization_two_partitions_into_four() -> Result<()> { + let alias = vec![("a".to_string(), "a".to_string())]; + let plan_parquet = + aggregate_exec_with_alias(parquet_exec_multiple(), alias.clone()); + let plan_csv = aggregate_exec_with_alias(csv_exec_multiple(), alias); + + let expected_parquet = [ + "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", + "RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4", + "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", + // Multiple source files splitted across partitions + "ParquetExec: file_groups={4 groups: [[x:0..50], [x:50..100], [y:0..50], [y:50..100]]}, projection=[a, b, c, d, e]", + ]; + let expected_csv = [ + "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", + "RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4", + "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", + // Multiple source files splitted across partitions + "CsvExec: file_groups={4 groups: [[x:0..50], [x:50..100], [y:0..50], [y:50..100]]}, projection=[a, b, c, d, e], has_header=false", + ]; + assert_optimized!(expected_parquet, plan_parquet, true, false, 4, true, 10); + assert_optimized!(expected_csv, plan_csv, true, false, 4, true, 10); + + Ok(()) +} + +#[test] +fn parallelization_sorted_limit() -> Result<()> { + let schema = schema(); + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { + expr: col("c", &schema).unwrap(), + options: SortOptions::default(), + }]); + let plan_parquet = limit_exec(sort_exec(sort_key.clone(), parquet_exec(), false)); + let plan_csv = limit_exec(sort_exec(sort_key, csv_exec(), false)); + + let expected_parquet = &[ + "GlobalLimitExec: skip=0, fetch=100", + "LocalLimitExec: fetch=100", + // data is sorted so can't repartition here + "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", + // Doesn't parallelize for SortExec without preserve_partitioning + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + let expected_csv = &[ + "GlobalLimitExec: skip=0, fetch=100", + "LocalLimitExec: fetch=100", + // data is sorted so can't repartition here + "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", + // Doesn't parallelize for SortExec without preserve_partitioning + "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + ]; + assert_optimized!(expected_parquet, plan_parquet, true); + assert_optimized!(expected_csv, plan_csv, true); + + Ok(()) +} + +#[test] +fn parallelization_limit_with_filter() -> Result<()> { + let schema = schema(); + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { + expr: col("c", &schema).unwrap(), + options: SortOptions::default(), + }]); + let plan_parquet = limit_exec(filter_exec(sort_exec( + sort_key.clone(), + parquet_exec(), + false, + ))); + let plan_csv = limit_exec(filter_exec(sort_exec(sort_key, csv_exec(), false))); + + let expected_parquet = &[ + "GlobalLimitExec: skip=0, fetch=100", + "CoalescePartitionsExec", + "LocalLimitExec: fetch=100", + "FilterExec: c@2 = 0", + // even though data is sorted, we can use repartition here. Since + // ordering is not used in subsequent stages anyway. + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", + // SortExec doesn't benefit from input partitioning + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + let expected_csv = &[ + "GlobalLimitExec: skip=0, fetch=100", + "CoalescePartitionsExec", + "LocalLimitExec: fetch=100", + "FilterExec: c@2 = 0", + // even though data is sorted, we can use repartition here. Since + // ordering is not used in subsequent stages anyway. + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", + // SortExec doesn't benefit from input partitioning + "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + ]; + assert_optimized!(expected_parquet, plan_parquet, true); + assert_optimized!(expected_csv, plan_csv, true); + + Ok(()) +} + +#[test] +fn parallelization_ignores_limit() -> Result<()> { + let alias = vec![("a".to_string(), "a".to_string())]; + let plan_parquet = aggregate_exec_with_alias( + limit_exec(filter_exec(limit_exec(parquet_exec()))), + alias.clone(), + ); + let plan_csv = aggregate_exec_with_alias( + limit_exec(filter_exec(limit_exec(csv_exec()))), + alias, + ); + + let expected_parquet = &[ + "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", + "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", + "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "GlobalLimitExec: skip=0, fetch=100", + "CoalescePartitionsExec", + "LocalLimitExec: fetch=100", + "FilterExec: c@2 = 0", + // repartition should happen prior to the filter to maximize parallelism + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "GlobalLimitExec: skip=0, fetch=100", + // Limit doesn't benefit from input partitioning - no parallelism + "LocalLimitExec: fetch=100", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + let expected_csv = &[ + "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", + "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", + "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "GlobalLimitExec: skip=0, fetch=100", + "CoalescePartitionsExec", + "LocalLimitExec: fetch=100", + "FilterExec: c@2 = 0", + // repartition should happen prior to the filter to maximize parallelism + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "GlobalLimitExec: skip=0, fetch=100", + // Limit doesn't benefit from input partitioning - no parallelism + "LocalLimitExec: fetch=100", + "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + ]; + assert_optimized!(expected_parquet, plan_parquet, true); + assert_optimized!(expected_csv, plan_csv, true); + + Ok(()) +} + +#[test] +fn parallelization_union_inputs() -> Result<()> { + let plan_parquet = union_exec(vec![parquet_exec(); 5]); + let plan_csv = union_exec(vec![csv_exec(); 5]); + + let expected_parquet = &[ + "UnionExec", + // Union doesn't benefit from input partitioning - no parallelism + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + let expected_csv = &[ + "UnionExec", + // Union doesn't benefit from input partitioning - no parallelism + "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", + ]; + assert_optimized!(expected_parquet, plan_parquet, true); + assert_optimized!(expected_csv, plan_csv, true); + + Ok(()) +} + +#[test] +fn parallelization_prior_to_sort_preserving_merge() -> Result<()> { + let schema = schema(); + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { + expr: col("c", &schema).unwrap(), + options: SortOptions::default(), + }]); + // sort preserving merge already sorted input, + let plan_parquet = sort_preserving_merge_exec( + sort_key.clone(), + parquet_exec_with_sort(vec![sort_key.clone()]), + ); + let plan_csv = sort_preserving_merge_exec( + sort_key.clone(), + csv_exec_with_sort(vec![sort_key]), + ); + + // parallelization is not beneficial for SortPreservingMerge + let expected_parquet = &[ + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + ]; + let expected_csv = &[ + "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], has_header=false", + ]; + assert_optimized!(expected_parquet, plan_parquet, true); + assert_optimized!(expected_csv, plan_csv, true); + + Ok(()) +} + +#[test] +fn parallelization_sort_preserving_merge_with_union() -> Result<()> { + let schema = schema(); + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { + expr: col("c", &schema).unwrap(), + options: SortOptions::default(), + }]); + // 2 sorted parquet files unioned (partitions are concatenated, sort is preserved) + let input_parquet = + union_exec(vec![parquet_exec_with_sort(vec![sort_key.clone()]); 2]); + let input_csv = union_exec(vec![csv_exec_with_sort(vec![sort_key.clone()]); 2]); + let plan_parquet = sort_preserving_merge_exec(sort_key.clone(), input_parquet); + let plan_csv = sort_preserving_merge_exec(sort_key, input_csv); + + // should not repartition (union doesn't benefit from increased parallelism) + // should not sort (as the data was already sorted) + let expected_parquet = &[ + "SortPreservingMergeExec: [c@2 ASC]", + "UnionExec", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + ]; + let expected_csv = &[ + "SortPreservingMergeExec: [c@2 ASC]", + "UnionExec", + "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], has_header=false", + "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], has_header=false", + ]; + assert_optimized!(expected_parquet, plan_parquet, true); + assert_optimized!(expected_csv, plan_csv, true); + + Ok(()) +} + +#[test] +fn parallelization_does_not_benefit() -> Result<()> { + let schema = schema(); + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { + expr: col("c", &schema).unwrap(), + options: SortOptions::default(), + }]); + // SortRequired + // Parquet(sorted) + let plan_parquet = sort_required_exec_with_req( + parquet_exec_with_sort(vec![sort_key.clone()]), + sort_key.clone(), + ); + let plan_csv = sort_required_exec_with_req( + csv_exec_with_sort(vec![sort_key.clone()]), + sort_key, + ); + + // no parallelization, because SortRequiredExec doesn't benefit from increased parallelism + let expected_parquet = &[ + "SortRequiredExec: [c@2 ASC]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + ]; + let expected_csv = &[ + "SortRequiredExec: [c@2 ASC]", + "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], has_header=false", + ]; + assert_optimized!(expected_parquet, plan_parquet, true); + assert_optimized!(expected_csv, plan_csv, true); + + Ok(()) +} + +#[test] +fn parallelization_ignores_transitively_with_projection_parquet() -> Result<()> { + // sorted input + let schema = schema(); + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { + expr: col("c", &schema).unwrap(), + options: SortOptions::default(), + }]); + + //Projection(a as a2, b as b2) + let alias_pairs: Vec<(String, String)> = vec![ + ("a".to_string(), "a2".to_string()), + ("c".to_string(), "c2".to_string()), + ]; + let proj_parquet = projection_exec_with_alias( + parquet_exec_with_sort(vec![sort_key]), + alias_pairs, + ); + let sort_key_after_projection = LexOrdering::new(vec![PhysicalSortExpr { + expr: col("c2", &proj_parquet.schema()).unwrap(), + options: SortOptions::default(), + }]); + let plan_parquet = + sort_preserving_merge_exec(sort_key_after_projection, proj_parquet); + let expected = &[ + "SortPreservingMergeExec: [c2@1 ASC]", + " ProjectionExec: expr=[a@0 as a2, c@2 as c2]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + ]; + plans_matches_expected!(expected, &plan_parquet); + + // data should not be repartitioned / resorted + let expected_parquet = &[ + "ProjectionExec: expr=[a@0 as a2, c@2 as c2]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + ]; + assert_optimized!(expected_parquet, plan_parquet, true); + + Ok(()) +} + +#[test] +fn parallelization_ignores_transitively_with_projection_csv() -> Result<()> { + // sorted input + let schema = schema(); + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { + expr: col("c", &schema).unwrap(), + options: SortOptions::default(), + }]); + + //Projection(a as a2, b as b2) + let alias_pairs: Vec<(String, String)> = vec![ + ("a".to_string(), "a2".to_string()), + ("c".to_string(), "c2".to_string()), + ]; + + let proj_csv = + projection_exec_with_alias(csv_exec_with_sort(vec![sort_key]), alias_pairs); + let sort_key_after_projection = LexOrdering::new(vec![PhysicalSortExpr { + expr: col("c2", &proj_csv.schema()).unwrap(), + options: SortOptions::default(), + }]); + let plan_csv = sort_preserving_merge_exec(sort_key_after_projection, proj_csv); + let expected = &[ + "SortPreservingMergeExec: [c2@1 ASC]", + " ProjectionExec: expr=[a@0 as a2, c@2 as c2]", + " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], has_header=false", + ]; + plans_matches_expected!(expected, &plan_csv); + + // data should not be repartitioned / resorted + let expected_csv = &[ + "ProjectionExec: expr=[a@0 as a2, c@2 as c2]", + "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], has_header=false", + ]; + assert_optimized!(expected_csv, plan_csv, true); + + Ok(()) +} + +#[test] +fn remove_redundant_roundrobins() -> Result<()> { + let input = parquet_exec(); + let repartition = repartition_exec(repartition_exec(input)); + let physical_plan = repartition_exec(filter_exec(repartition)); + let expected = &[ + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10", + " FilterExec: c@2 = 0", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + plans_matches_expected!(expected, &physical_plan); + + let expected = &[ + "FilterExec: c@2 = 0", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + assert_optimized!(expected, physical_plan.clone(), true); + assert_optimized!(expected, physical_plan, false); + + Ok(()) +} + +#[test] +fn remove_unnecessary_spm_after_filter() -> Result<()> { + let schema = schema(); + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { + expr: col("c", &schema).unwrap(), + options: SortOptions::default(), + }]); + let input = parquet_exec_multiple_sorted(vec![sort_key.clone()]); + let physical_plan = sort_preserving_merge_exec(sort_key, filter_exec(input)); + + // Original plan expects its output to be ordered by c@2 ASC. + // This is still satisfied since, after filter that column is constant. + let expected = &[ + "CoalescePartitionsExec", + "FilterExec: c@2 = 0", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2, preserve_order=true, sort_exprs=c@2 ASC", + "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + ]; + // last flag sets config.optimizer.PREFER_EXISTING_SORT + assert_optimized!(expected, physical_plan.clone(), true, true); + assert_optimized!(expected, physical_plan, false, true); + + Ok(()) +} + +#[test] +fn preserve_ordering_through_repartition() -> Result<()> { + let schema = schema(); + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { + expr: col("d", &schema).unwrap(), + options: SortOptions::default(), + }]); + let input = parquet_exec_multiple_sorted(vec![sort_key.clone()]); + let physical_plan = sort_preserving_merge_exec(sort_key, filter_exec(input)); + + let expected = &[ + "SortPreservingMergeExec: [d@3 ASC]", + "FilterExec: c@2 = 0", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2, preserve_order=true, sort_exprs=d@3 ASC", + "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[d@3 ASC]", + ]; + // last flag sets config.optimizer.PREFER_EXISTING_SORT + assert_optimized!(expected, physical_plan.clone(), true, true); + assert_optimized!(expected, physical_plan, false, true); + + Ok(()) +} + +#[test] +fn do_not_preserve_ordering_through_repartition() -> Result<()> { + let schema = schema(); + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { + expr: col("a", &schema).unwrap(), + options: SortOptions::default(), + }]); + let input = parquet_exec_multiple_sorted(vec![sort_key.clone()]); + let physical_plan = sort_preserving_merge_exec(sort_key, filter_exec(input)); + + let expected = &[ + "SortPreservingMergeExec: [a@0 ASC]", + "SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", + "FilterExec: c@2 = 0", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", + "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", + ]; + + assert_optimized!(expected, physical_plan.clone(), true); + + let expected = &[ + "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", + "CoalescePartitionsExec", + "FilterExec: c@2 = 0", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", + "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", + ]; + assert_optimized!(expected, physical_plan, false); + + Ok(()) +} + +#[test] +fn no_need_for_sort_after_filter() -> Result<()> { + let schema = schema(); + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { + expr: col("c", &schema).unwrap(), + options: SortOptions::default(), + }]); + let input = parquet_exec_multiple_sorted(vec![sort_key.clone()]); + let physical_plan = sort_preserving_merge_exec(sort_key, filter_exec(input)); + + let expected = &[ + // After CoalescePartitionsExec c is still constant. Hence c@2 ASC ordering is already satisfied. + "CoalescePartitionsExec", + // Since after this stage c is constant. c@2 ASC ordering is already satisfied. + "FilterExec: c@2 = 0", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", + "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + ]; + assert_optimized!(expected, physical_plan.clone(), true); + assert_optimized!(expected, physical_plan, false); + + Ok(()) +} + +#[test] +fn do_not_preserve_ordering_through_repartition2() -> Result<()> { + let schema = schema(); + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { + expr: col("c", &schema).unwrap(), + options: SortOptions::default(), + }]); + let input = parquet_exec_multiple_sorted(vec![sort_key]); + + let sort_req = LexOrdering::new(vec![PhysicalSortExpr { + expr: col("a", &schema).unwrap(), + options: SortOptions::default(), + }]); + let physical_plan = sort_preserving_merge_exec(sort_req, filter_exec(input)); + + let expected = &[ + "SortPreservingMergeExec: [a@0 ASC]", + "SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", + "FilterExec: c@2 = 0", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", + "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + ]; + + assert_optimized!(expected, physical_plan.clone(), true); + + let expected = &[ + "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", + "CoalescePartitionsExec", + "SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", + "FilterExec: c@2 = 0", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", + "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + ]; + assert_optimized!(expected, physical_plan, false); + + Ok(()) +} + +#[test] +fn do_not_preserve_ordering_through_repartition3() -> Result<()> { + let schema = schema(); + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { + expr: col("c", &schema).unwrap(), + options: SortOptions::default(), + }]); + let input = parquet_exec_multiple_sorted(vec![sort_key]); + let physical_plan = filter_exec(input); + + let expected = &[ + "FilterExec: c@2 = 0", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", + "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + ]; + assert_optimized!(expected, physical_plan.clone(), true); + assert_optimized!(expected, physical_plan, false); + + Ok(()) +} + +#[test] +fn do_not_put_sort_when_input_is_invalid() -> Result<()> { + let schema = schema(); + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { + expr: col("a", &schema).unwrap(), + options: SortOptions::default(), + }]); + let input = parquet_exec(); + let physical_plan = sort_required_exec_with_req(filter_exec(input), sort_key); + let expected = &[ + // Ordering requirement of sort required exec is NOT satisfied + // by existing ordering at the source. + "SortRequiredExec: [a@0 ASC]", + "FilterExec: c@2 = 0", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + assert_plan_txt!(expected, physical_plan); + + let expected = &[ + "SortRequiredExec: [a@0 ASC]", + // Since at the start of the rule ordering requirement is not satisfied + // EnforceDistribution rule doesn't satisfy this requirement either. + "FilterExec: c@2 = 0", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + + let mut config = ConfigOptions::new(); + config.execution.target_partitions = 10; + config.optimizer.enable_round_robin_repartition = true; + config.optimizer.prefer_existing_sort = false; + let dist_plan = EnforceDistribution::new().optimize(physical_plan, &config)?; + assert_plan_txt!(expected, dist_plan); + + Ok(()) +} + +#[test] +fn put_sort_when_input_is_valid() -> Result<()> { + let schema = schema(); + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { + expr: col("a", &schema).unwrap(), + options: SortOptions::default(), + }]); + let input = parquet_exec_multiple_sorted(vec![sort_key.clone()]); + let physical_plan = sort_required_exec_with_req(filter_exec(input), sort_key); + + let expected = &[ + // Ordering requirement of sort required exec is satisfied + // by existing ordering at the source. + "SortRequiredExec: [a@0 ASC]", + "FilterExec: c@2 = 0", + "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", + ]; + assert_plan_txt!(expected, physical_plan); + + let expected = &[ + // Since at the start of the rule ordering requirement is satisfied + // EnforceDistribution rule satisfy this requirement also. + "SortRequiredExec: [a@0 ASC]", + "FilterExec: c@2 = 0", + "ParquetExec: file_groups={10 groups: [[x:0..20], [y:0..20], [x:20..40], [y:20..40], [x:40..60], [y:40..60], [x:60..80], [y:60..80], [x:80..100], [y:80..100]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", + ]; + + let mut config = ConfigOptions::new(); + config.execution.target_partitions = 10; + config.optimizer.enable_round_robin_repartition = true; + config.optimizer.prefer_existing_sort = false; + let dist_plan = EnforceDistribution::new().optimize(physical_plan, &config)?; + assert_plan_txt!(expected, dist_plan); + + Ok(()) +} + +#[test] +fn do_not_add_unnecessary_hash() -> Result<()> { + let schema = schema(); + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { + expr: col("c", &schema).unwrap(), + options: SortOptions::default(), + }]); + let alias = vec![("a".to_string(), "a".to_string())]; + let input = parquet_exec_with_sort(vec![sort_key]); + let physical_plan = aggregate_exec_with_alias(input, alias); + + let expected = &[ + "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", + "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + ]; + // Make sure target partition number is 1. In this case hash repartition is unnecessary + assert_optimized!(expected, physical_plan.clone(), true, false, 1, false, 1024); + assert_optimized!(expected, physical_plan, false, false, 1, false, 1024); + + Ok(()) +} + +#[test] +fn do_not_add_unnecessary_hash2() -> Result<()> { + let schema = schema(); + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { + expr: col("c", &schema).unwrap(), + options: SortOptions::default(), + }]); + let alias = vec![("a".to_string(), "a".to_string())]; + let input = parquet_exec_multiple_sorted(vec![sort_key]); + let aggregate = aggregate_exec_with_alias(input, alias.clone()); + let physical_plan = aggregate_exec_with_alias(aggregate, alias); + + let expected = &[ + "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", + // Since hash requirements of this operator is satisfied. There shouldn't be + // a hash repartition here + "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", + "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", + "RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4", + "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", + "RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2", + "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", + ]; + // Make sure target partition number is larger than 2 (e.g partition number at the source). + assert_optimized!(expected, physical_plan.clone(), true, false, 4, false, 1024); + assert_optimized!(expected, physical_plan, false, false, 4, false, 1024); + + Ok(()) +} + +#[test] +fn optimize_away_unnecessary_repartition() -> Result<()> { + let physical_plan = coalesce_partitions_exec(repartition_exec(parquet_exec())); + let expected = &[ + "CoalescePartitionsExec", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + plans_matches_expected!(expected, physical_plan.clone()); + + let expected = + &["ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]"]; + assert_optimized!(expected, physical_plan.clone(), true); + assert_optimized!(expected, physical_plan, false); + + Ok(()) +} + +#[test] +fn optimize_away_unnecessary_repartition2() -> Result<()> { + let physical_plan = filter_exec(repartition_exec(coalesce_partitions_exec( + filter_exec(repartition_exec(parquet_exec())), + ))); + let expected = &[ + "FilterExec: c@2 = 0", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " CoalescePartitionsExec", + " FilterExec: c@2 = 0", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + plans_matches_expected!(expected, physical_plan.clone()); + + let expected = &[ + "FilterExec: c@2 = 0", + "FilterExec: c@2 = 0", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", + ]; + assert_optimized!(expected, physical_plan.clone(), true); + assert_optimized!(expected, physical_plan, false); + + Ok(()) +} diff --git a/datafusion/core/tests/physical_optimizer/mod.rs b/datafusion/core/tests/physical_optimizer/mod.rs index 1fac68e2505c..4f487ee88679 100644 --- a/datafusion/core/tests/physical_optimizer/mod.rs +++ b/datafusion/core/tests/physical_optimizer/mod.rs @@ -18,4 +18,5 @@ mod combine_partial_final_agg; mod limited_distinct_aggregation; mod sanity_checker; +mod enforce_distribution; pub(crate) mod test_util; From 0296d04634f9aa1f53d9f7fa3e8941d231ebb5dc Mon Sep 17 00:00:00 2001 From: logan-keede Date: Tue, 21 Jan 2025 04:04:51 +0530 Subject: [PATCH 06/24] remove tests from enforce_distribution.rs --- .../core/tests/physical_optimizer/mod.rs | 2 +- .../src/enforce_distribution.rs | 3257 ----------------- 2 files changed, 1 insertion(+), 3258 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/mod.rs b/datafusion/core/tests/physical_optimizer/mod.rs index 4f487ee88679..20ddb1999e28 100644 --- a/datafusion/core/tests/physical_optimizer/mod.rs +++ b/datafusion/core/tests/physical_optimizer/mod.rs @@ -18,5 +18,5 @@ mod combine_partial_final_agg; mod limited_distinct_aggregation; mod sanity_checker; -mod enforce_distribution; +// mod enforce_distribution; pub(crate) mod test_util; diff --git a/datafusion/physical-optimizer/src/enforce_distribution.rs b/datafusion/physical-optimizer/src/enforce_distribution.rs index 58149a01df5f..be4c6667df3f 100644 --- a/datafusion/physical-optimizer/src/enforce_distribution.rs +++ b/datafusion/physical-optimizer/src/enforce_distribution.rs @@ -1409,3260 +1409,3 @@ struct JoinKeyPairs { /// Keeps track of parent required key orderings. type PlanWithKeyRequirements = PlanContext>>; - -/// Since almost all of these tests explicitly use `ParquetExec` they only run with the parquet feature flag on -#[cfg(feature = "parquet")] -#[cfg(test)] -pub(crate) mod tests { - use std::ops::Deref; - - use super::*; - use crate::datasource::file_format::file_compression_type::FileCompressionType; - use crate::datasource::listing::PartitionedFile; - use crate::datasource::object_store::ObjectStoreUrl; - use crate::datasource::physical_plan::{CsvExec, FileScanConfig, ParquetExec}; - use crate::physical_optimizer::enforce_sorting::EnforceSorting; - use crate::physical_plan::coalesce_batches::CoalesceBatchesExec; - use crate::physical_plan::expressions::col; - use crate::physical_plan::filter::FilterExec; - use crate::physical_plan::joins::utils::JoinOn; - use crate::physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; - use crate::physical_plan::sorts::sort::SortExec; - use crate::physical_plan::{displayable, DisplayAs, DisplayFormatType, Statistics}; - use datafusion_physical_optimizer::output_requirements::OutputRequirements; - use datafusion_physical_optimizer::test_utils::{ - check_integrity, coalesce_partitions_exec, repartition_exec, - }; - - use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; - use datafusion_common::ScalarValue; - use datafusion_expr::Operator; - use datafusion_physical_expr::expressions::{BinaryExpr, Literal}; - use datafusion_physical_expr::{ - expressions::binary, expressions::lit, LexOrdering, PhysicalSortExpr, - }; - use datafusion_physical_expr_common::sort_expr::LexRequirement; - use datafusion_physical_plan::PlanProperties; - - /// Models operators like BoundedWindowExec that require an input - /// ordering but is easy to construct - #[derive(Debug)] - struct SortRequiredExec { - input: Arc, - expr: LexOrdering, - cache: PlanProperties, - } - - impl SortRequiredExec { - fn new_with_requirement( - input: Arc, - requirement: LexOrdering, - ) -> Self { - let cache = Self::compute_properties(&input); - Self { - input, - expr: requirement, - cache, - } - } - - /// This function creates the cache object that stores the plan properties such as schema, equivalence properties, ordering, partitioning, etc. - fn compute_properties(input: &Arc) -> PlanProperties { - PlanProperties::new( - input.equivalence_properties().clone(), // Equivalence Properties - input.output_partitioning().clone(), // Output Partitioning - input.pipeline_behavior(), // Pipeline Behavior - input.boundedness(), // Boundedness - ) - } - } - - impl DisplayAs for SortRequiredExec { - fn fmt_as( - &self, - _t: DisplayFormatType, - f: &mut std::fmt::Formatter, - ) -> std::fmt::Result { - write!(f, "SortRequiredExec: [{}]", self.expr) - } - } - - impl ExecutionPlan for SortRequiredExec { - fn name(&self) -> &'static str { - "SortRequiredExec" - } - - fn as_any(&self) -> &dyn std::any::Any { - self - } - - fn properties(&self) -> &PlanProperties { - &self.cache - } - - fn benefits_from_input_partitioning(&self) -> Vec { - vec![false] - } - - fn children(&self) -> Vec<&Arc> { - vec![&self.input] - } - - // model that it requires the output ordering of its input - fn required_input_ordering(&self) -> Vec> { - if self.expr.is_empty() { - vec![None] - } else { - vec![Some(LexRequirement::from(self.expr.clone()))] - } - } - - fn with_new_children( - self: Arc, - mut children: Vec>, - ) -> Result> { - assert_eq!(children.len(), 1); - let child = children.pop().unwrap(); - Ok(Arc::new(Self::new_with_requirement( - child, - self.expr.clone(), - ))) - } - - fn execute( - &self, - _partition: usize, - _context: Arc, - ) -> Result { - unreachable!(); - } - - fn statistics(&self) -> Result { - self.input.statistics() - } - } - - pub(crate) fn schema() -> SchemaRef { - Arc::new(Schema::new(vec![ - Field::new("a", DataType::Int64, true), - Field::new("b", DataType::Int64, true), - Field::new("c", DataType::Int64, true), - Field::new("d", DataType::Int32, true), - Field::new("e", DataType::Boolean, true), - ])) - } - - fn parquet_exec() -> Arc { - parquet_exec_with_sort(vec![]) - } - - /// create a single parquet file that is sorted - pub(crate) fn parquet_exec_with_sort( - output_ordering: Vec, - ) -> Arc { - ParquetExec::builder( - FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema()) - .with_file(PartitionedFile::new("x".to_string(), 100)) - .with_output_ordering(output_ordering), - ) - .build_arc() - } - - fn parquet_exec_multiple() -> Arc { - parquet_exec_multiple_sorted(vec![]) - } - - /// Created a sorted parquet exec with multiple files - fn parquet_exec_multiple_sorted( - output_ordering: Vec, - ) -> Arc { - ParquetExec::builder( - FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema()) - .with_file_groups(vec![ - vec![PartitionedFile::new("x".to_string(), 100)], - vec![PartitionedFile::new("y".to_string(), 100)], - ]) - .with_output_ordering(output_ordering), - ) - .build_arc() - } - - fn csv_exec() -> Arc { - csv_exec_with_sort(vec![]) - } - - fn csv_exec_with_sort(output_ordering: Vec) -> Arc { - Arc::new( - CsvExec::builder( - FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema()) - .with_file(PartitionedFile::new("x".to_string(), 100)) - .with_output_ordering(output_ordering), - ) - .with_has_header(false) - .with_delimeter(b',') - .with_quote(b'"') - .with_escape(None) - .with_comment(None) - .with_newlines_in_values(false) - .with_file_compression_type(FileCompressionType::UNCOMPRESSED) - .build(), - ) - } - - fn csv_exec_multiple() -> Arc { - csv_exec_multiple_sorted(vec![]) - } - - // Created a sorted parquet exec with multiple files - fn csv_exec_multiple_sorted(output_ordering: Vec) -> Arc { - Arc::new( - CsvExec::builder( - FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema()) - .with_file_groups(vec![ - vec![PartitionedFile::new("x".to_string(), 100)], - vec![PartitionedFile::new("y".to_string(), 100)], - ]) - .with_output_ordering(output_ordering), - ) - .with_has_header(false) - .with_delimeter(b',') - .with_quote(b'"') - .with_escape(None) - .with_comment(None) - .with_newlines_in_values(false) - .with_file_compression_type(FileCompressionType::UNCOMPRESSED) - .build(), - ) - } - - fn projection_exec_with_alias( - input: Arc, - alias_pairs: Vec<(String, String)>, - ) -> Arc { - let mut exprs = vec![]; - for (column, alias) in alias_pairs.iter() { - exprs.push((col(column, &input.schema()).unwrap(), alias.to_string())); - } - Arc::new(ProjectionExec::try_new(exprs, input).unwrap()) - } - - fn aggregate_exec_with_alias( - input: Arc, - alias_pairs: Vec<(String, String)>, - ) -> Arc { - let schema = schema(); - let mut group_by_expr: Vec<(Arc, String)> = vec![]; - for (column, alias) in alias_pairs.iter() { - group_by_expr - .push((col(column, &input.schema()).unwrap(), alias.to_string())); - } - let group_by = PhysicalGroupBy::new_single(group_by_expr.clone()); - - let final_group_by_expr = group_by_expr - .iter() - .enumerate() - .map(|(index, (_col, name))| { - ( - Arc::new(Column::new(name, index)) as Arc, - name.clone(), - ) - }) - .collect::>(); - let final_grouping = PhysicalGroupBy::new_single(final_group_by_expr); - - Arc::new( - AggregateExec::try_new( - AggregateMode::FinalPartitioned, - final_grouping, - vec![], - vec![], - Arc::new( - AggregateExec::try_new( - AggregateMode::Partial, - group_by, - vec![], - vec![], - input, - schema.clone(), - ) - .unwrap(), - ), - schema, - ) - .unwrap(), - ) - } - - fn hash_join_exec( - left: Arc, - right: Arc, - join_on: &JoinOn, - join_type: &JoinType, - ) -> Arc { - Arc::new( - HashJoinExec::try_new( - left, - right, - join_on.clone(), - None, - join_type, - None, - PartitionMode::Partitioned, - false, - ) - .unwrap(), - ) - } - - fn sort_merge_join_exec( - left: Arc, - right: Arc, - join_on: &JoinOn, - join_type: &JoinType, - ) -> Arc { - Arc::new( - SortMergeJoinExec::try_new( - left, - right, - join_on.clone(), - None, - *join_type, - vec![SortOptions::default(); join_on.len()], - false, - ) - .unwrap(), - ) - } - - fn filter_exec(input: Arc) -> Arc { - let predicate = Arc::new(BinaryExpr::new( - col("c", &schema()).unwrap(), - Operator::Eq, - Arc::new(Literal::new(ScalarValue::Int64(Some(0)))), - )); - Arc::new(FilterExec::try_new(predicate, input).unwrap()) - } - - fn sort_exec( - sort_exprs: LexOrdering, - input: Arc, - preserve_partitioning: bool, - ) -> Arc { - let new_sort = SortExec::new(sort_exprs, input) - .with_preserve_partitioning(preserve_partitioning); - Arc::new(new_sort) - } - - fn sort_preserving_merge_exec( - sort_exprs: LexOrdering, - input: Arc, - ) -> Arc { - Arc::new(SortPreservingMergeExec::new(sort_exprs, input)) - } - - fn limit_exec(input: Arc) -> Arc { - Arc::new(GlobalLimitExec::new( - Arc::new(LocalLimitExec::new(input, 100)), - 0, - Some(100), - )) - } - - fn union_exec(input: Vec>) -> Arc { - Arc::new(UnionExec::new(input)) - } - - fn sort_required_exec_with_req( - input: Arc, - sort_exprs: LexOrdering, - ) -> Arc { - Arc::new(SortRequiredExec::new_with_requirement(input, sort_exprs)) - } - - pub(crate) fn trim_plan_display(plan: &str) -> Vec<&str> { - plan.split('\n') - .map(|s| s.trim()) - .filter(|s| !s.is_empty()) - .collect() - } - - fn ensure_distribution_helper( - plan: Arc, - target_partitions: usize, - prefer_existing_sort: bool, - ) -> Result> { - let distribution_context = DistributionContext::new_default(plan); - let mut config = ConfigOptions::new(); - config.execution.target_partitions = target_partitions; - config.optimizer.enable_round_robin_repartition = true; - config.optimizer.repartition_file_scans = false; - config.optimizer.repartition_file_min_size = 1024; - config.optimizer.prefer_existing_sort = prefer_existing_sort; - ensure_distribution(distribution_context, &config).map(|item| item.data.plan) - } - - /// Test whether plan matches with expected plan - macro_rules! plans_matches_expected { - ($EXPECTED_LINES: expr, $PLAN: expr) => { - let physical_plan = $PLAN; - let formatted = displayable(physical_plan.as_ref()).indent(true).to_string(); - let actual: Vec<&str> = formatted.trim().lines().collect(); - - let expected_plan_lines: Vec<&str> = $EXPECTED_LINES - .iter().map(|s| *s).collect(); - - assert_eq!( - expected_plan_lines, actual, - "\n**Original Plan Mismatch\n\nexpected:\n\n{expected_plan_lines:#?}\nactual:\n\n{actual:#?}\n\n" - ); - } - } - - /// Runs the repartition optimizer and asserts the plan against the expected - /// Arguments - /// * `EXPECTED_LINES` - Expected output plan - /// * `PLAN` - Input plan - /// * `FIRST_ENFORCE_DIST` - - /// true: (EnforceDistribution, EnforceDistribution, EnforceSorting) - /// false: else runs (EnforceSorting, EnforceDistribution, EnforceDistribution) - /// * `PREFER_EXISTING_SORT` (optional) - if true, will not repartition / resort data if it is already sorted - /// * `TARGET_PARTITIONS` (optional) - number of partitions to repartition to - /// * `REPARTITION_FILE_SCANS` (optional) - if true, will repartition file scans - /// * `REPARTITION_FILE_MIN_SIZE` (optional) - minimum file size to repartition - /// * `PREFER_EXISTING_UNION` (optional) - if true, will not attempt to convert Union to Interleave - macro_rules! assert_optimized { - ($EXPECTED_LINES: expr, $PLAN: expr, $FIRST_ENFORCE_DIST: expr) => { - assert_optimized!($EXPECTED_LINES, $PLAN, $FIRST_ENFORCE_DIST, false, 10, false, 1024, false); - }; - - ($EXPECTED_LINES: expr, $PLAN: expr, $FIRST_ENFORCE_DIST: expr, $PREFER_EXISTING_SORT: expr) => { - assert_optimized!($EXPECTED_LINES, $PLAN, $FIRST_ENFORCE_DIST, $PREFER_EXISTING_SORT, 10, false, 1024, false); - }; - - ($EXPECTED_LINES: expr, $PLAN: expr, $FIRST_ENFORCE_DIST: expr, $PREFER_EXISTING_SORT: expr, $PREFER_EXISTING_UNION: expr) => { - assert_optimized!($EXPECTED_LINES, $PLAN, $FIRST_ENFORCE_DIST, $PREFER_EXISTING_SORT, 10, false, 1024, $PREFER_EXISTING_UNION); - }; - - ($EXPECTED_LINES: expr, $PLAN: expr, $FIRST_ENFORCE_DIST: expr, $PREFER_EXISTING_SORT: expr, $TARGET_PARTITIONS: expr, $REPARTITION_FILE_SCANS: expr, $REPARTITION_FILE_MIN_SIZE: expr) => { - assert_optimized!($EXPECTED_LINES, $PLAN, $FIRST_ENFORCE_DIST, $PREFER_EXISTING_SORT, $TARGET_PARTITIONS, $REPARTITION_FILE_SCANS, $REPARTITION_FILE_MIN_SIZE, false); - }; - - ($EXPECTED_LINES: expr, $PLAN: expr, $FIRST_ENFORCE_DIST: expr, $PREFER_EXISTING_SORT: expr, $TARGET_PARTITIONS: expr, $REPARTITION_FILE_SCANS: expr, $REPARTITION_FILE_MIN_SIZE: expr, $PREFER_EXISTING_UNION: expr) => { - let expected_lines: Vec<&str> = $EXPECTED_LINES.iter().map(|s| *s).collect(); - - let mut config = ConfigOptions::new(); - config.execution.target_partitions = $TARGET_PARTITIONS; - config.optimizer.repartition_file_scans = $REPARTITION_FILE_SCANS; - config.optimizer.repartition_file_min_size = $REPARTITION_FILE_MIN_SIZE; - config.optimizer.prefer_existing_sort = $PREFER_EXISTING_SORT; - config.optimizer.prefer_existing_union = $PREFER_EXISTING_UNION; - // Use a small batch size, to trigger RoundRobin in tests - config.execution.batch_size = 1; - - // NOTE: These tests verify the joint `EnforceDistribution` + `EnforceSorting` cascade - // because they were written prior to the separation of `BasicEnforcement` into - // `EnforceSorting` and `EnforceDistribution`. - // TODO: Orthogonalize the tests here just to verify `EnforceDistribution` and create - // new tests for the cascade. - - // Add the ancillary output requirements operator at the start: - let optimizer = OutputRequirements::new_add_mode(); - let optimized = optimizer.optimize($PLAN.clone(), &config)?; - - // This file has 2 rules that use tree node, apply these rules to original plan consecutively - // After these operations tree nodes should be in a consistent state. - // This code block makes sure that these rules doesn't violate tree node integrity. - { - let adjusted = if config.optimizer.top_down_join_key_reordering { - // Run adjust_input_keys_ordering rule - let plan_requirements = - PlanWithKeyRequirements::new_default($PLAN.clone()); - let adjusted = plan_requirements - .transform_down(adjust_input_keys_ordering) - .data() - .and_then(check_integrity)?; - // TODO: End state payloads will be checked here. - adjusted.plan - } else { - // Run reorder_join_keys_to_inputs rule - $PLAN.clone().transform_up(|plan| { - Ok(Transformed::yes(reorder_join_keys_to_inputs(plan)?)) - }) - .data()? - }; - - // Then run ensure_distribution rule - DistributionContext::new_default(adjusted) - .transform_up(|distribution_context| { - ensure_distribution(distribution_context, &config) - }) - .data() - .and_then(check_integrity)?; - // TODO: End state payloads will be checked here. - } - - let optimized = if $FIRST_ENFORCE_DIST { - // Run enforce distribution rule first: - let optimizer = EnforceDistribution::new(); - let optimized = optimizer.optimize(optimized, &config)?; - // The rule should be idempotent. - // Re-running this rule shouldn't introduce unnecessary operators. - let optimizer = EnforceDistribution::new(); - let optimized = optimizer.optimize(optimized, &config)?; - // Run the enforce sorting rule: - let optimizer = EnforceSorting::new(); - let optimized = optimizer.optimize(optimized, &config)?; - optimized - } else { - // Run the enforce sorting rule first: - let optimizer = EnforceSorting::new(); - let optimized = optimizer.optimize(optimized, &config)?; - // Run enforce distribution rule: - let optimizer = EnforceDistribution::new(); - let optimized = optimizer.optimize(optimized, &config)?; - // The rule should be idempotent. - // Re-running this rule shouldn't introduce unnecessary operators. - let optimizer = EnforceDistribution::new(); - let optimized = optimizer.optimize(optimized, &config)?; - optimized - }; - - // Remove the ancillary output requirements operator when done: - let optimizer = OutputRequirements::new_remove_mode(); - let optimized = optimizer.optimize(optimized, &config)?; - - // Now format correctly - let plan = displayable(optimized.as_ref()).indent(true).to_string(); - let actual_lines = trim_plan_display(&plan); - - assert_eq!( - &expected_lines, &actual_lines, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected_lines, actual_lines - ); - }; - } - - macro_rules! assert_plan_txt { - ($EXPECTED_LINES: expr, $PLAN: expr) => { - let expected_lines: Vec<&str> = $EXPECTED_LINES.iter().map(|s| *s).collect(); - // Now format correctly - let plan = displayable($PLAN.as_ref()).indent(true).to_string(); - let actual_lines = trim_plan_display(&plan); - - assert_eq!( - &expected_lines, &actual_lines, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected_lines, actual_lines - ); - }; - } - - #[test] - fn multi_hash_joins() -> Result<()> { - let left = parquet_exec(); - let alias_pairs: Vec<(String, String)> = vec![ - ("a".to_string(), "a1".to_string()), - ("b".to_string(), "b1".to_string()), - ("c".to_string(), "c1".to_string()), - ("d".to_string(), "d1".to_string()), - ("e".to_string(), "e1".to_string()), - ]; - let right = projection_exec_with_alias(parquet_exec(), alias_pairs); - let join_types = vec![ - JoinType::Inner, - JoinType::Left, - JoinType::Right, - JoinType::Full, - JoinType::LeftSemi, - JoinType::LeftAnti, - JoinType::LeftMark, - JoinType::RightSemi, - JoinType::RightAnti, - ]; - - // Join on (a == b1) - let join_on = vec![( - Arc::new(Column::new_with_schema("a", &schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("b1", &right.schema()).unwrap()) as _, - )]; - - for join_type in join_types { - let join = hash_join_exec(left.clone(), right.clone(), &join_on, &join_type); - let join_plan = format!( - "HashJoinExec: mode=Partitioned, join_type={join_type}, on=[(a@0, b1@1)]" - ); - - match join_type { - JoinType::Inner - | JoinType::Left - | JoinType::Right - | JoinType::Full - | JoinType::LeftSemi - | JoinType::LeftAnti - | JoinType::LeftMark => { - // Join on (a == c) - let top_join_on = vec![( - Arc::new(Column::new_with_schema("a", &join.schema()).unwrap()) - as _, - Arc::new(Column::new_with_schema("c", &schema()).unwrap()) as _, - )]; - let top_join = hash_join_exec( - join.clone(), - parquet_exec(), - &top_join_on, - &join_type, - ); - let top_join_plan = - format!("HashJoinExec: mode=Partitioned, join_type={join_type}, on=[(a@0, c@2)]"); - - let expected = match join_type { - // Should include 3 RepartitionExecs - JoinType::Inner | JoinType::Left | JoinType::LeftSemi | JoinType::LeftAnti | JoinType::LeftMark => vec![ - top_join_plan.as_str(), - join_plan.as_str(), - "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ], - // Should include 4 RepartitionExecs - _ => vec![ - top_join_plan.as_str(), - "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - join_plan.as_str(), - "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ], - }; - assert_optimized!(expected, top_join.clone(), true); - assert_optimized!(expected, top_join, false); - } - JoinType::RightSemi | JoinType::RightAnti => {} - } - - match join_type { - JoinType::Inner - | JoinType::Left - | JoinType::Right - | JoinType::Full - | JoinType::RightSemi - | JoinType::RightAnti => { - // This time we use (b1 == c) for top join - // Join on (b1 == c) - let top_join_on = vec![( - Arc::new(Column::new_with_schema("b1", &join.schema()).unwrap()) - as _, - Arc::new(Column::new_with_schema("c", &schema()).unwrap()) as _, - )]; - - let top_join = - hash_join_exec(join, parquet_exec(), &top_join_on, &join_type); - let top_join_plan = match join_type { - JoinType::RightSemi | JoinType::RightAnti => - format!("HashJoinExec: mode=Partitioned, join_type={join_type}, on=[(b1@1, c@2)]"), - _ => - format!("HashJoinExec: mode=Partitioned, join_type={join_type}, on=[(b1@6, c@2)]"), - }; - - let expected = match join_type { - // Should include 3 RepartitionExecs - JoinType::Inner | JoinType::Right | JoinType::RightSemi | JoinType::RightAnti => - vec![ - top_join_plan.as_str(), - join_plan.as_str(), - "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ], - // Should include 4 RepartitionExecs - _ => - vec![ - top_join_plan.as_str(), - "RepartitionExec: partitioning=Hash([b1@6], 10), input_partitions=10", - join_plan.as_str(), - "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ], - }; - assert_optimized!(expected, top_join.clone(), true); - assert_optimized!(expected, top_join, false); - } - JoinType::LeftSemi | JoinType::LeftAnti | JoinType::LeftMark => {} - } - } - - Ok(()) - } - - #[test] - fn multi_joins_after_alias() -> Result<()> { - let left = parquet_exec(); - let right = parquet_exec(); - - // Join on (a == b) - let join_on = vec![( - Arc::new(Column::new_with_schema("a", &schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("b", &schema()).unwrap()) as _, - )]; - let join = hash_join_exec(left, right.clone(), &join_on, &JoinType::Inner); - - // Projection(a as a1, a as a2) - let alias_pairs: Vec<(String, String)> = vec![ - ("a".to_string(), "a1".to_string()), - ("a".to_string(), "a2".to_string()), - ]; - let projection = projection_exec_with_alias(join, alias_pairs); - - // Join on (a1 == c) - let top_join_on = vec![( - Arc::new(Column::new_with_schema("a1", &projection.schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("c", &schema()).unwrap()) as _, - )]; - - let top_join = hash_join_exec( - projection.clone(), - right.clone(), - &top_join_on, - &JoinType::Inner, - ); - - // Output partition need to respect the Alias and should not introduce additional RepartitionExec - let expected = &[ - "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a1@0, c@2)]", - "ProjectionExec: expr=[a@0 as a1, a@0 as a2]", - "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, b@1)]", - "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "RepartitionExec: partitioning=Hash([b@1], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - assert_optimized!(expected, top_join.clone(), true); - assert_optimized!(expected, top_join, false); - - // Join on (a2 == c) - let top_join_on = vec![( - Arc::new(Column::new_with_schema("a2", &projection.schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("c", &schema()).unwrap()) as _, - )]; - - let top_join = hash_join_exec(projection, right, &top_join_on, &JoinType::Inner); - - // Output partition need to respect the Alias and should not introduce additional RepartitionExec - let expected = &[ - "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a2@1, c@2)]", - "ProjectionExec: expr=[a@0 as a1, a@0 as a2]", - "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, b@1)]", - "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "RepartitionExec: partitioning=Hash([b@1], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - assert_optimized!(expected, top_join.clone(), true); - assert_optimized!(expected, top_join, false); - - Ok(()) - } - - #[test] - fn multi_joins_after_multi_alias() -> Result<()> { - let left = parquet_exec(); - let right = parquet_exec(); - - // Join on (a == b) - let join_on = vec![( - Arc::new(Column::new_with_schema("a", &schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("b", &schema()).unwrap()) as _, - )]; - - let join = hash_join_exec(left, right.clone(), &join_on, &JoinType::Inner); - - // Projection(c as c1) - let alias_pairs: Vec<(String, String)> = - vec![("c".to_string(), "c1".to_string())]; - let projection = projection_exec_with_alias(join, alias_pairs); - - // Projection(c1 as a) - let alias_pairs: Vec<(String, String)> = - vec![("c1".to_string(), "a".to_string())]; - let projection2 = projection_exec_with_alias(projection, alias_pairs); - - // Join on (a == c) - let top_join_on = vec![( - Arc::new(Column::new_with_schema("a", &projection2.schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("c", &schema()).unwrap()) as _, - )]; - - let top_join = hash_join_exec(projection2, right, &top_join_on, &JoinType::Inner); - - // The Column 'a' has different meaning now after the two Projections - // The original Output partition can not satisfy the Join requirements and need to add an additional RepartitionExec - let expected = &[ - "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, c@2)]", - "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - "ProjectionExec: expr=[c1@0 as a]", - "ProjectionExec: expr=[c@2 as c1]", - "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, b@1)]", - "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "RepartitionExec: partitioning=Hash([b@1], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - - assert_optimized!(expected, top_join.clone(), true); - assert_optimized!(expected, top_join, false); - - Ok(()) - } - - #[test] - fn join_after_agg_alias() -> Result<()> { - // group by (a as a1) - let left = aggregate_exec_with_alias( - parquet_exec(), - vec![("a".to_string(), "a1".to_string())], - ); - // group by (a as a2) - let right = aggregate_exec_with_alias( - parquet_exec(), - vec![("a".to_string(), "a2".to_string())], - ); - - // Join on (a1 == a2) - let join_on = vec![( - Arc::new(Column::new_with_schema("a1", &left.schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("a2", &right.schema()).unwrap()) as _, - )]; - let join = hash_join_exec(left, right.clone(), &join_on, &JoinType::Inner); - - // Only two RepartitionExecs added - let expected = &[ - "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a1@0, a2@0)]", - "AggregateExec: mode=FinalPartitioned, gby=[a1@0 as a1], aggr=[]", - "RepartitionExec: partitioning=Hash([a1@0], 10), input_partitions=10", - "AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[]", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "AggregateExec: mode=FinalPartitioned, gby=[a2@0 as a2], aggr=[]", - "RepartitionExec: partitioning=Hash([a2@0], 10), input_partitions=10", - "AggregateExec: mode=Partial, gby=[a@0 as a2], aggr=[]", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - assert_optimized!(expected, join.clone(), true); - assert_optimized!(expected, join, false); - - Ok(()) - } - - #[test] - fn hash_join_key_ordering() -> Result<()> { - // group by (a as a1, b as b1) - let left = aggregate_exec_with_alias( - parquet_exec(), - vec![ - ("a".to_string(), "a1".to_string()), - ("b".to_string(), "b1".to_string()), - ], - ); - // group by (b, a) - let right = aggregate_exec_with_alias( - parquet_exec(), - vec![ - ("b".to_string(), "b".to_string()), - ("a".to_string(), "a".to_string()), - ], - ); - - // Join on (b1 == b && a1 == a) - let join_on = vec![ - ( - Arc::new(Column::new_with_schema("b1", &left.schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("b", &right.schema()).unwrap()) as _, - ), - ( - Arc::new(Column::new_with_schema("a1", &left.schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("a", &right.schema()).unwrap()) as _, - ), - ]; - let join = hash_join_exec(left, right.clone(), &join_on, &JoinType::Inner); - - // Only two RepartitionExecs added - let expected = &[ - "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(b1@1, b@0), (a1@0, a@1)]", - "ProjectionExec: expr=[a1@1 as a1, b1@0 as b1]", - "AggregateExec: mode=FinalPartitioned, gby=[b1@0 as b1, a1@1 as a1], aggr=[]", - "RepartitionExec: partitioning=Hash([b1@0, a1@1], 10), input_partitions=10", - "AggregateExec: mode=Partial, gby=[b@1 as b1, a@0 as a1], aggr=[]", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "AggregateExec: mode=FinalPartitioned, gby=[b@0 as b, a@1 as a], aggr=[]", - "RepartitionExec: partitioning=Hash([b@0, a@1], 10), input_partitions=10", - "AggregateExec: mode=Partial, gby=[b@1 as b, a@0 as a], aggr=[]", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - assert_optimized!(expected, join.clone(), true); - assert_optimized!(expected, join, false); - - Ok(()) - } - - #[test] - fn multi_hash_join_key_ordering() -> Result<()> { - let left = parquet_exec(); - let alias_pairs: Vec<(String, String)> = vec![ - ("a".to_string(), "a1".to_string()), - ("b".to_string(), "b1".to_string()), - ("c".to_string(), "c1".to_string()), - ]; - let right = projection_exec_with_alias(parquet_exec(), alias_pairs); - - // Join on (a == a1 and b == b1 and c == c1) - let join_on = vec![ - ( - Arc::new(Column::new_with_schema("a", &schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("a1", &right.schema()).unwrap()) as _, - ), - ( - Arc::new(Column::new_with_schema("b", &schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("b1", &right.schema()).unwrap()) as _, - ), - ( - Arc::new(Column::new_with_schema("c", &schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("c1", &right.schema()).unwrap()) as _, - ), - ]; - let bottom_left_join = - hash_join_exec(left.clone(), right.clone(), &join_on, &JoinType::Inner); - - // Projection(a as A, a as AA, b as B, c as C) - let alias_pairs: Vec<(String, String)> = vec![ - ("a".to_string(), "A".to_string()), - ("a".to_string(), "AA".to_string()), - ("b".to_string(), "B".to_string()), - ("c".to_string(), "C".to_string()), - ]; - let bottom_left_projection = - projection_exec_with_alias(bottom_left_join, alias_pairs); - - // Join on (c == c1 and b == b1 and a == a1) - let join_on = vec![ - ( - Arc::new(Column::new_with_schema("c", &schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("c1", &right.schema()).unwrap()) as _, - ), - ( - Arc::new(Column::new_with_schema("b", &schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("b1", &right.schema()).unwrap()) as _, - ), - ( - Arc::new(Column::new_with_schema("a", &schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("a1", &right.schema()).unwrap()) as _, - ), - ]; - let bottom_right_join = - hash_join_exec(left, right.clone(), &join_on, &JoinType::Inner); - - // Join on (B == b1 and C == c and AA = a1) - let top_join_on = vec![ - ( - Arc::new( - Column::new_with_schema("B", &bottom_left_projection.schema()) - .unwrap(), - ) as _, - Arc::new( - Column::new_with_schema("b1", &bottom_right_join.schema()).unwrap(), - ) as _, - ), - ( - Arc::new( - Column::new_with_schema("C", &bottom_left_projection.schema()) - .unwrap(), - ) as _, - Arc::new( - Column::new_with_schema("c", &bottom_right_join.schema()).unwrap(), - ) as _, - ), - ( - Arc::new( - Column::new_with_schema("AA", &bottom_left_projection.schema()) - .unwrap(), - ) as _, - Arc::new( - Column::new_with_schema("a1", &bottom_right_join.schema()).unwrap(), - ) as _, - ), - ]; - - let top_join = hash_join_exec( - bottom_left_projection.clone(), - bottom_right_join, - &top_join_on, - &JoinType::Inner, - ); - - let predicate: Arc = binary( - col("c", top_join.schema().deref())?, - Operator::Gt, - lit(1i64), - top_join.schema().deref(), - )?; - - let filter_top_join: Arc = - Arc::new(FilterExec::try_new(predicate, top_join)?); - - // The bottom joins' join key ordering is adjusted based on the top join. And the top join should not introduce additional RepartitionExec - let expected = &[ - "FilterExec: c@6 > 1", - "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(B@2, b1@6), (C@3, c@2), (AA@1, a1@5)]", - "ProjectionExec: expr=[a@0 as A, a@0 as AA, b@1 as B, c@2 as C]", - "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(b@1, b1@1), (c@2, c1@2), (a@0, a1@0)]", - "RepartitionExec: partitioning=Hash([b@1, c@2, a@0], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "RepartitionExec: partitioning=Hash([b1@1, c1@2, a1@0], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(b@1, b1@1), (c@2, c1@2), (a@0, a1@0)]", - "RepartitionExec: partitioning=Hash([b@1, c@2, a@0], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "RepartitionExec: partitioning=Hash([b1@1, c1@2, a1@0], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - assert_optimized!(expected, filter_top_join.clone(), true); - assert_optimized!(expected, filter_top_join, false); - - Ok(()) - } - - #[test] - fn reorder_join_keys_to_left_input() -> Result<()> { - let left = parquet_exec(); - let alias_pairs: Vec<(String, String)> = vec![ - ("a".to_string(), "a1".to_string()), - ("b".to_string(), "b1".to_string()), - ("c".to_string(), "c1".to_string()), - ]; - let right = projection_exec_with_alias(parquet_exec(), alias_pairs); - - // Join on (a == a1 and b == b1 and c == c1) - let join_on = vec![ - ( - Arc::new(Column::new_with_schema("a", &schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("a1", &right.schema()).unwrap()) as _, - ), - ( - Arc::new(Column::new_with_schema("b", &schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("b1", &right.schema()).unwrap()) as _, - ), - ( - Arc::new(Column::new_with_schema("c", &schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("c1", &right.schema()).unwrap()) as _, - ), - ]; - - let bottom_left_join = ensure_distribution_helper( - hash_join_exec(left.clone(), right.clone(), &join_on, &JoinType::Inner), - 10, - true, - )?; - - // Projection(a as A, a as AA, b as B, c as C) - let alias_pairs: Vec<(String, String)> = vec![ - ("a".to_string(), "A".to_string()), - ("a".to_string(), "AA".to_string()), - ("b".to_string(), "B".to_string()), - ("c".to_string(), "C".to_string()), - ]; - let bottom_left_projection = - projection_exec_with_alias(bottom_left_join, alias_pairs); - - // Join on (c == c1 and b == b1 and a == a1) - let join_on = vec![ - ( - Arc::new(Column::new_with_schema("c", &schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("c1", &right.schema()).unwrap()) as _, - ), - ( - Arc::new(Column::new_with_schema("b", &schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("b1", &right.schema()).unwrap()) as _, - ), - ( - Arc::new(Column::new_with_schema("a", &schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("a1", &right.schema()).unwrap()) as _, - ), - ]; - let bottom_right_join = ensure_distribution_helper( - hash_join_exec(left, right.clone(), &join_on, &JoinType::Inner), - 10, - true, - )?; - - // Join on (B == b1 and C == c and AA = a1) - let top_join_on = vec![ - ( - Arc::new( - Column::new_with_schema("B", &bottom_left_projection.schema()) - .unwrap(), - ) as _, - Arc::new( - Column::new_with_schema("b1", &bottom_right_join.schema()).unwrap(), - ) as _, - ), - ( - Arc::new( - Column::new_with_schema("C", &bottom_left_projection.schema()) - .unwrap(), - ) as _, - Arc::new( - Column::new_with_schema("c", &bottom_right_join.schema()).unwrap(), - ) as _, - ), - ( - Arc::new( - Column::new_with_schema("AA", &bottom_left_projection.schema()) - .unwrap(), - ) as _, - Arc::new( - Column::new_with_schema("a1", &bottom_right_join.schema()).unwrap(), - ) as _, - ), - ]; - - let join_types = vec![ - JoinType::Inner, - JoinType::Left, - JoinType::Right, - JoinType::Full, - JoinType::LeftSemi, - JoinType::LeftAnti, - JoinType::RightSemi, - JoinType::RightAnti, - ]; - - for join_type in join_types { - let top_join = hash_join_exec( - bottom_left_projection.clone(), - bottom_right_join.clone(), - &top_join_on, - &join_type, - ); - let top_join_plan = - format!("HashJoinExec: mode=Partitioned, join_type={:?}, on=[(AA@1, a1@5), (B@2, b1@6), (C@3, c@2)]", &join_type); - - let reordered = reorder_join_keys_to_inputs(top_join)?; - - // The top joins' join key ordering is adjusted based on the children inputs. - let expected = &[ - top_join_plan.as_str(), - "ProjectionExec: expr=[a@0 as A, a@0 as AA, b@1 as B, c@2 as C]", - "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a1@0), (b@1, b1@1), (c@2, c1@2)]", - "RepartitionExec: partitioning=Hash([a@0, b@1, c@2], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "RepartitionExec: partitioning=Hash([a1@0, b1@1, c1@2], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@2, c1@2), (b@1, b1@1), (a@0, a1@0)]", - "RepartitionExec: partitioning=Hash([c@2, b@1, a@0], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "RepartitionExec: partitioning=Hash([c1@2, b1@1, a1@0], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - - assert_plan_txt!(expected, reordered); - } - - Ok(()) - } - - #[test] - fn reorder_join_keys_to_right_input() -> Result<()> { - let left = parquet_exec(); - let alias_pairs: Vec<(String, String)> = vec![ - ("a".to_string(), "a1".to_string()), - ("b".to_string(), "b1".to_string()), - ("c".to_string(), "c1".to_string()), - ]; - let right = projection_exec_with_alias(parquet_exec(), alias_pairs); - - // Join on (a == a1 and b == b1) - let join_on = vec![ - ( - Arc::new(Column::new_with_schema("a", &schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("a1", &right.schema()).unwrap()) as _, - ), - ( - Arc::new(Column::new_with_schema("b", &schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("b1", &right.schema()).unwrap()) as _, - ), - ]; - let bottom_left_join = ensure_distribution_helper( - hash_join_exec(left.clone(), right.clone(), &join_on, &JoinType::Inner), - 10, - true, - )?; - - // Projection(a as A, a as AA, b as B, c as C) - let alias_pairs: Vec<(String, String)> = vec![ - ("a".to_string(), "A".to_string()), - ("a".to_string(), "AA".to_string()), - ("b".to_string(), "B".to_string()), - ("c".to_string(), "C".to_string()), - ]; - let bottom_left_projection = - projection_exec_with_alias(bottom_left_join, alias_pairs); - - // Join on (c == c1 and b == b1 and a == a1) - let join_on = vec![ - ( - Arc::new(Column::new_with_schema("c", &schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("c1", &right.schema()).unwrap()) as _, - ), - ( - Arc::new(Column::new_with_schema("b", &schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("b1", &right.schema()).unwrap()) as _, - ), - ( - Arc::new(Column::new_with_schema("a", &schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("a1", &right.schema()).unwrap()) as _, - ), - ]; - let bottom_right_join = ensure_distribution_helper( - hash_join_exec(left, right.clone(), &join_on, &JoinType::Inner), - 10, - true, - )?; - - // Join on (B == b1 and C == c and AA = a1) - let top_join_on = vec![ - ( - Arc::new( - Column::new_with_schema("B", &bottom_left_projection.schema()) - .unwrap(), - ) as _, - Arc::new( - Column::new_with_schema("b1", &bottom_right_join.schema()).unwrap(), - ) as _, - ), - ( - Arc::new( - Column::new_with_schema("C", &bottom_left_projection.schema()) - .unwrap(), - ) as _, - Arc::new( - Column::new_with_schema("c", &bottom_right_join.schema()).unwrap(), - ) as _, - ), - ( - Arc::new( - Column::new_with_schema("AA", &bottom_left_projection.schema()) - .unwrap(), - ) as _, - Arc::new( - Column::new_with_schema("a1", &bottom_right_join.schema()).unwrap(), - ) as _, - ), - ]; - - let join_types = vec![ - JoinType::Inner, - JoinType::Left, - JoinType::Right, - JoinType::Full, - JoinType::LeftSemi, - JoinType::LeftAnti, - JoinType::RightSemi, - JoinType::RightAnti, - ]; - - for join_type in join_types { - let top_join = hash_join_exec( - bottom_left_projection.clone(), - bottom_right_join.clone(), - &top_join_on, - &join_type, - ); - let top_join_plan = - format!("HashJoinExec: mode=Partitioned, join_type={:?}, on=[(C@3, c@2), (B@2, b1@6), (AA@1, a1@5)]", &join_type); - - let reordered = reorder_join_keys_to_inputs(top_join)?; - - // The top joins' join key ordering is adjusted based on the children inputs. - let expected = &[ - top_join_plan.as_str(), - "ProjectionExec: expr=[a@0 as A, a@0 as AA, b@1 as B, c@2 as C]", - "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a1@0), (b@1, b1@1)]", - "RepartitionExec: partitioning=Hash([a@0, b@1], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "RepartitionExec: partitioning=Hash([a1@0, b1@1], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "HashJoinExec: mode=Partitioned, join_type=Inner, on=[(c@2, c1@2), (b@1, b1@1), (a@0, a1@0)]", - "RepartitionExec: partitioning=Hash([c@2, b@1, a@0], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "RepartitionExec: partitioning=Hash([c1@2, b1@1, a1@0], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - - assert_plan_txt!(expected, reordered); - } - - Ok(()) - } - - #[test] - fn multi_smj_joins() -> Result<()> { - let left = parquet_exec(); - let alias_pairs: Vec<(String, String)> = vec![ - ("a".to_string(), "a1".to_string()), - ("b".to_string(), "b1".to_string()), - ("c".to_string(), "c1".to_string()), - ("d".to_string(), "d1".to_string()), - ("e".to_string(), "e1".to_string()), - ]; - let right = projection_exec_with_alias(parquet_exec(), alias_pairs); - - // SortMergeJoin does not support RightSemi and RightAnti join now - let join_types = vec![ - JoinType::Inner, - JoinType::Left, - JoinType::Right, - JoinType::Full, - JoinType::LeftSemi, - JoinType::LeftAnti, - ]; - - // Join on (a == b1) - let join_on = vec![( - Arc::new(Column::new_with_schema("a", &schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("b1", &right.schema()).unwrap()) as _, - )]; - - for join_type in join_types { - let join = - sort_merge_join_exec(left.clone(), right.clone(), &join_on, &join_type); - let join_plan = - format!("SortMergeJoin: join_type={join_type}, on=[(a@0, b1@1)]"); - - // Top join on (a == c) - let top_join_on = vec![( - Arc::new(Column::new_with_schema("a", &join.schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("c", &schema()).unwrap()) as _, - )]; - let top_join = sort_merge_join_exec( - join.clone(), - parquet_exec(), - &top_join_on, - &join_type, - ); - let top_join_plan = - format!("SortMergeJoin: join_type={join_type}, on=[(a@0, c@2)]"); - - let expected = match join_type { - // Should include 6 RepartitionExecs (3 hash, 3 round-robin), 3 SortExecs - JoinType::Inner | JoinType::Left | JoinType::LeftSemi | JoinType::LeftAnti => - vec![ - top_join_plan.as_str(), - join_plan.as_str(), - "SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", - "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "SortExec: expr=[b1@1 ASC], preserve_partitioning=[true]", - "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "SortExec: expr=[c@2 ASC], preserve_partitioning=[true]", - "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ], - // Should include 7 RepartitionExecs (4 hash, 3 round-robin), 4 SortExecs - // Since ordering of the left child is not preserved after SortMergeJoin - // when mode is Right, RightSemi, RightAnti, Full - // - We need to add one additional SortExec after SortMergeJoin in contrast the test cases - // when mode is Inner, Left, LeftSemi, LeftAnti - // Similarly, since partitioning of the left side is not preserved - // when mode is Right, RightSemi, RightAnti, Full - // - We need to add one additional Hash Repartition after SortMergeJoin in contrast the test - // cases when mode is Inner, Left, LeftSemi, LeftAnti - _ => vec![ - top_join_plan.as_str(), - // Below 2 operators are differences introduced, when join mode is changed - "SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", - "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - join_plan.as_str(), - "SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", - "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "SortExec: expr=[b1@1 ASC], preserve_partitioning=[true]", - "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "SortExec: expr=[c@2 ASC], preserve_partitioning=[true]", - "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ], - }; - assert_optimized!(expected, top_join.clone(), true, true); - - let expected_first_sort_enforcement = match join_type { - // Should include 6 RepartitionExecs (3 hash, 3 round-robin), 3 SortExecs - JoinType::Inner | JoinType::Left | JoinType::LeftSemi | JoinType::LeftAnti => - vec![ - top_join_plan.as_str(), - join_plan.as_str(), - "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10, preserve_order=true, sort_exprs=b1@1 ASC", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "SortExec: expr=[b1@1 ASC], preserve_partitioning=[false]", - "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10, preserve_order=true, sort_exprs=c@2 ASC", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ], - // Should include 8 RepartitionExecs (4 hash, 8 round-robin), 4 SortExecs - // Since ordering of the left child is not preserved after SortMergeJoin - // when mode is Right, RightSemi, RightAnti, Full - // - We need to add one additional SortExec after SortMergeJoin in contrast the test cases - // when mode is Inner, Left, LeftSemi, LeftAnti - // Similarly, since partitioning of the left side is not preserved - // when mode is Right, RightSemi, RightAnti, Full - // - We need to add one additional Hash Repartition and Roundrobin repartition after - // SortMergeJoin in contrast the test cases when mode is Inner, Left, LeftSemi, LeftAnti - _ => vec![ - top_join_plan.as_str(), - // Below 4 operators are differences introduced, when join mode is changed - "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - "CoalescePartitionsExec", - join_plan.as_str(), - "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10, preserve_order=true, sort_exprs=b1@1 ASC", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "SortExec: expr=[b1@1 ASC], preserve_partitioning=[false]", - "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10, preserve_order=true, sort_exprs=c@2 ASC", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ], - }; - assert_optimized!(expected_first_sort_enforcement, top_join, false, true); - - match join_type { - JoinType::Inner | JoinType::Left | JoinType::Right | JoinType::Full => { - // This time we use (b1 == c) for top join - // Join on (b1 == c) - let top_join_on = vec![( - Arc::new(Column::new_with_schema("b1", &join.schema()).unwrap()) - as _, - Arc::new(Column::new_with_schema("c", &schema()).unwrap()) as _, - )]; - let top_join = sort_merge_join_exec( - join, - parquet_exec(), - &top_join_on, - &join_type, - ); - let top_join_plan = - format!("SortMergeJoin: join_type={join_type}, on=[(b1@6, c@2)]"); - - let expected = match join_type { - // Should include 6 RepartitionExecs(3 hash, 3 round-robin) and 3 SortExecs - JoinType::Inner | JoinType::Right => vec![ - top_join_plan.as_str(), - join_plan.as_str(), - "SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", - "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "SortExec: expr=[b1@1 ASC], preserve_partitioning=[true]", - "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "SortExec: expr=[c@2 ASC], preserve_partitioning=[true]", - "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ], - // Should include 7 RepartitionExecs (4 hash, 3 round-robin) and 4 SortExecs - JoinType::Left | JoinType::Full => vec![ - top_join_plan.as_str(), - "SortExec: expr=[b1@6 ASC], preserve_partitioning=[true]", - "RepartitionExec: partitioning=Hash([b1@6], 10), input_partitions=10", - join_plan.as_str(), - "SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", - "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "SortExec: expr=[b1@1 ASC], preserve_partitioning=[true]", - "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "SortExec: expr=[c@2 ASC], preserve_partitioning=[true]", - "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ], - // this match arm cannot be reached - _ => unreachable!() - }; - assert_optimized!(expected, top_join.clone(), true, true); - - let expected_first_sort_enforcement = match join_type { - // Should include 6 RepartitionExecs (3 of them preserves order) and 3 SortExecs - JoinType::Inner | JoinType::Right => vec![ - top_join_plan.as_str(), - join_plan.as_str(), - "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10, preserve_order=true, sort_exprs=b1@1 ASC", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "SortExec: expr=[b1@1 ASC], preserve_partitioning=[false]", - "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10, preserve_order=true, sort_exprs=c@2 ASC", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ], - // Should include 8 RepartitionExecs (4 of them preserves order) and 4 SortExecs - JoinType::Left | JoinType::Full => vec![ - top_join_plan.as_str(), - "RepartitionExec: partitioning=Hash([b1@6], 10), input_partitions=10, preserve_order=true, sort_exprs=b1@6 ASC", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "SortExec: expr=[b1@6 ASC], preserve_partitioning=[false]", - "CoalescePartitionsExec", - join_plan.as_str(), - "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10, preserve_order=true, sort_exprs=a@0 ASC", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "RepartitionExec: partitioning=Hash([b1@1], 10), input_partitions=10, preserve_order=true, sort_exprs=b1@1 ASC", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "SortExec: expr=[b1@1 ASC], preserve_partitioning=[false]", - "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "RepartitionExec: partitioning=Hash([c@2], 10), input_partitions=10, preserve_order=true, sort_exprs=c@2 ASC", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ], - // this match arm cannot be reached - _ => unreachable!() - }; - assert_optimized!( - expected_first_sort_enforcement, - top_join, - false, - true - ); - } - _ => {} - } - } - - Ok(()) - } - - #[test] - fn smj_join_key_ordering() -> Result<()> { - // group by (a as a1, b as b1) - let left = aggregate_exec_with_alias( - parquet_exec(), - vec![ - ("a".to_string(), "a1".to_string()), - ("b".to_string(), "b1".to_string()), - ], - ); - //Projection(a1 as a3, b1 as b3) - let alias_pairs: Vec<(String, String)> = vec![ - ("a1".to_string(), "a3".to_string()), - ("b1".to_string(), "b3".to_string()), - ]; - let left = projection_exec_with_alias(left, alias_pairs); - - // group by (b, a) - let right = aggregate_exec_with_alias( - parquet_exec(), - vec![ - ("b".to_string(), "b".to_string()), - ("a".to_string(), "a".to_string()), - ], - ); - - //Projection(a as a2, b as b2) - let alias_pairs: Vec<(String, String)> = vec![ - ("a".to_string(), "a2".to_string()), - ("b".to_string(), "b2".to_string()), - ]; - let right = projection_exec_with_alias(right, alias_pairs); - - // Join on (b3 == b2 && a3 == a2) - let join_on = vec![ - ( - Arc::new(Column::new_with_schema("b3", &left.schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("b2", &right.schema()).unwrap()) as _, - ), - ( - Arc::new(Column::new_with_schema("a3", &left.schema()).unwrap()) as _, - Arc::new(Column::new_with_schema("a2", &right.schema()).unwrap()) as _, - ), - ]; - let join = sort_merge_join_exec(left, right.clone(), &join_on, &JoinType::Inner); - - // Only two RepartitionExecs added - let expected = &[ - "SortMergeJoin: join_type=Inner, on=[(b3@1, b2@1), (a3@0, a2@0)]", - "SortExec: expr=[b3@1 ASC, a3@0 ASC], preserve_partitioning=[true]", - "ProjectionExec: expr=[a1@0 as a3, b1@1 as b3]", - "ProjectionExec: expr=[a1@1 as a1, b1@0 as b1]", - "AggregateExec: mode=FinalPartitioned, gby=[b1@0 as b1, a1@1 as a1], aggr=[]", - "RepartitionExec: partitioning=Hash([b1@0, a1@1], 10), input_partitions=10", - "AggregateExec: mode=Partial, gby=[b@1 as b1, a@0 as a1], aggr=[]", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "SortExec: expr=[b2@1 ASC, a2@0 ASC], preserve_partitioning=[true]", - "ProjectionExec: expr=[a@1 as a2, b@0 as b2]", - "AggregateExec: mode=FinalPartitioned, gby=[b@0 as b, a@1 as a], aggr=[]", - "RepartitionExec: partitioning=Hash([b@0, a@1], 10), input_partitions=10", - "AggregateExec: mode=Partial, gby=[b@1 as b, a@0 as a], aggr=[]", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - assert_optimized!(expected, join.clone(), true, true); - - let expected_first_sort_enforcement = &[ - "SortMergeJoin: join_type=Inner, on=[(b3@1, b2@1), (a3@0, a2@0)]", - "RepartitionExec: partitioning=Hash([b3@1, a3@0], 10), input_partitions=10, preserve_order=true, sort_exprs=b3@1 ASC, a3@0 ASC", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "SortExec: expr=[b3@1 ASC, a3@0 ASC], preserve_partitioning=[false]", - "CoalescePartitionsExec", - "ProjectionExec: expr=[a1@0 as a3, b1@1 as b3]", - "ProjectionExec: expr=[a1@1 as a1, b1@0 as b1]", - "AggregateExec: mode=FinalPartitioned, gby=[b1@0 as b1, a1@1 as a1], aggr=[]", - "RepartitionExec: partitioning=Hash([b1@0, a1@1], 10), input_partitions=10", - "AggregateExec: mode=Partial, gby=[b@1 as b1, a@0 as a1], aggr=[]", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "RepartitionExec: partitioning=Hash([b2@1, a2@0], 10), input_partitions=10, preserve_order=true, sort_exprs=b2@1 ASC, a2@0 ASC", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "SortExec: expr=[b2@1 ASC, a2@0 ASC], preserve_partitioning=[false]", - "CoalescePartitionsExec", - "ProjectionExec: expr=[a@1 as a2, b@0 as b2]", - "AggregateExec: mode=FinalPartitioned, gby=[b@0 as b, a@1 as a], aggr=[]", - "RepartitionExec: partitioning=Hash([b@0, a@1], 10), input_partitions=10", - "AggregateExec: mode=Partial, gby=[b@1 as b, a@0 as a], aggr=[]", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - assert_optimized!(expected_first_sort_enforcement, join, false, true); - - Ok(()) - } - - #[test] - fn merge_does_not_need_sort() -> Result<()> { - // see https://github.com/apache/datafusion/issues/4331 - let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("a", &schema).unwrap(), - options: SortOptions::default(), - }]); - - // Scan some sorted parquet files - let exec = parquet_exec_multiple_sorted(vec![sort_key.clone()]); - - // CoalesceBatchesExec to mimic behavior after a filter - let exec = Arc::new(CoalesceBatchesExec::new(exec, 4096)); - - // Merge from multiple parquet files and keep the data sorted - let exec: Arc = - Arc::new(SortPreservingMergeExec::new(sort_key, exec)); - - // The optimizer should not add an additional SortExec as the - // data is already sorted - let expected = &[ - "SortPreservingMergeExec: [a@0 ASC]", - "CoalesceBatchesExec: target_batch_size=4096", - "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", - ]; - assert_optimized!(expected, exec, true); - - // In this case preserving ordering through order preserving operators is not desirable - // (according to flag: PREFER_EXISTING_SORT) - // hence in this case ordering lost during CoalescePartitionsExec and re-introduced with - // SortExec at the top. - let expected = &[ - "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - "CoalescePartitionsExec", - "CoalesceBatchesExec: target_batch_size=4096", - "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", - ]; - assert_optimized!(expected, exec, false); - - Ok(()) - } - - #[test] - fn union_to_interleave() -> Result<()> { - // group by (a as a1) - let left = aggregate_exec_with_alias( - parquet_exec(), - vec![("a".to_string(), "a1".to_string())], - ); - // group by (a as a2) - let right = aggregate_exec_with_alias( - parquet_exec(), - vec![("a".to_string(), "a1".to_string())], - ); - - // Union - let plan = Arc::new(UnionExec::new(vec![left, right])); - - // final agg - let plan = - aggregate_exec_with_alias(plan, vec![("a1".to_string(), "a2".to_string())]); - - // Only two RepartitionExecs added, no final RepartitionExec required - let expected = &[ - "AggregateExec: mode=FinalPartitioned, gby=[a2@0 as a2], aggr=[]", - "AggregateExec: mode=Partial, gby=[a1@0 as a2], aggr=[]", - "InterleaveExec", - "AggregateExec: mode=FinalPartitioned, gby=[a1@0 as a1], aggr=[]", - "RepartitionExec: partitioning=Hash([a1@0], 10), input_partitions=10", - "AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[]", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "AggregateExec: mode=FinalPartitioned, gby=[a1@0 as a1], aggr=[]", - "RepartitionExec: partitioning=Hash([a1@0], 10), input_partitions=10", - "AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[]", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - assert_optimized!(expected, plan.clone(), true); - assert_optimized!(expected, plan.clone(), false); - - Ok(()) - } - - #[test] - fn union_not_to_interleave() -> Result<()> { - // group by (a as a1) - let left = aggregate_exec_with_alias( - parquet_exec(), - vec![("a".to_string(), "a1".to_string())], - ); - // group by (a as a2) - let right = aggregate_exec_with_alias( - parquet_exec(), - vec![("a".to_string(), "a1".to_string())], - ); - - // Union - let plan = Arc::new(UnionExec::new(vec![left, right])); - - // final agg - let plan = - aggregate_exec_with_alias(plan, vec![("a1".to_string(), "a2".to_string())]); - - // Only two RepartitionExecs added, no final RepartitionExec required - let expected = &[ - "AggregateExec: mode=FinalPartitioned, gby=[a2@0 as a2], aggr=[]", - "RepartitionExec: partitioning=Hash([a2@0], 10), input_partitions=20", - "AggregateExec: mode=Partial, gby=[a1@0 as a2], aggr=[]", - "UnionExec", - "AggregateExec: mode=FinalPartitioned, gby=[a1@0 as a1], aggr=[]", - "RepartitionExec: partitioning=Hash([a1@0], 10), input_partitions=10", - "AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[]", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "AggregateExec: mode=FinalPartitioned, gby=[a1@0 as a1], aggr=[]", - "RepartitionExec: partitioning=Hash([a1@0], 10), input_partitions=10", - "AggregateExec: mode=Partial, gby=[a@0 as a1], aggr=[]", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - // no sort in the plan but since we need it as a parameter, make it default false - let prefer_existing_sort = false; - let first_enforce_distribution = true; - let prefer_existing_union = true; - - assert_optimized!( - expected, - plan.clone(), - first_enforce_distribution, - prefer_existing_sort, - prefer_existing_union - ); - assert_optimized!( - expected, - plan, - !first_enforce_distribution, - prefer_existing_sort, - prefer_existing_union - ); - - Ok(()) - } - - #[test] - fn added_repartition_to_single_partition() -> Result<()> { - let alias = vec![("a".to_string(), "a".to_string())]; - let plan = aggregate_exec_with_alias(parquet_exec(), alias); - - let expected = [ - "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", - "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - assert_optimized!(expected, plan.clone(), true); - assert_optimized!(expected, plan, false); - - Ok(()) - } - - #[test] - fn repartition_deepest_node() -> Result<()> { - let alias = vec![("a".to_string(), "a".to_string())]; - let plan = aggregate_exec_with_alias(filter_exec(parquet_exec()), alias); - - let expected = &[ - "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", - "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", - "FilterExec: c@2 = 0", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - assert_optimized!(expected, plan.clone(), true); - assert_optimized!(expected, plan, false); - - Ok(()) - } - - #[test] - - fn repartition_unsorted_limit() -> Result<()> { - let plan = limit_exec(filter_exec(parquet_exec())); - - let expected = &[ - "GlobalLimitExec: skip=0, fetch=100", - "CoalescePartitionsExec", - "LocalLimitExec: fetch=100", - "FilterExec: c@2 = 0", - // nothing sorts the data, so the local limit doesn't require sorted data either - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - - assert_optimized!(expected, plan.clone(), true); - assert_optimized!(expected, plan, false); - - Ok(()) - } - - #[test] - fn repartition_sorted_limit() -> Result<()> { - let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("c", &schema).unwrap(), - options: SortOptions::default(), - }]); - let plan = limit_exec(sort_exec(sort_key, parquet_exec(), false)); - - let expected = &[ - "GlobalLimitExec: skip=0, fetch=100", - "LocalLimitExec: fetch=100", - // data is sorted so can't repartition here - "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - assert_optimized!(expected, plan.clone(), true); - assert_optimized!(expected, plan, false); - - Ok(()) - } - - #[test] - fn repartition_sorted_limit_with_filter() -> Result<()> { - let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("c", &schema).unwrap(), - options: SortOptions::default(), - }]); - let plan = sort_required_exec_with_req( - filter_exec(sort_exec(sort_key.clone(), parquet_exec(), false)), - sort_key, - ); - - let expected = &[ - "SortRequiredExec: [c@2 ASC]", - "FilterExec: c@2 = 0", - // We can use repartition here, ordering requirement by SortRequiredExec - // is still satisfied. - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - - assert_optimized!(expected, plan.clone(), true); - assert_optimized!(expected, plan, false); - - Ok(()) - } - - #[test] - fn repartition_ignores_limit() -> Result<()> { - let alias = vec![("a".to_string(), "a".to_string())]; - let plan = aggregate_exec_with_alias( - limit_exec(filter_exec(limit_exec(parquet_exec()))), - alias, - ); - - let expected = &[ - "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", - "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "GlobalLimitExec: skip=0, fetch=100", - "CoalescePartitionsExec", - "LocalLimitExec: fetch=100", - "FilterExec: c@2 = 0", - // repartition should happen prior to the filter to maximize parallelism - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "GlobalLimitExec: skip=0, fetch=100", - "LocalLimitExec: fetch=100", - // Expect no repartition to happen for local limit - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - assert_optimized!(expected, plan.clone(), true); - assert_optimized!(expected, plan, false); - - Ok(()) - } - - #[test] - fn repartition_ignores_union() -> Result<()> { - let plan = union_exec(vec![parquet_exec(); 5]); - - let expected = &[ - "UnionExec", - // Expect no repartition of ParquetExec - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - - assert_optimized!(expected, plan.clone(), true); - assert_optimized!(expected, plan, false); - - Ok(()) - } - - #[test] - fn repartition_through_sort_preserving_merge() -> Result<()> { - // sort preserving merge with non-sorted input - let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("c", &schema).unwrap(), - options: SortOptions::default(), - }]); - let plan = sort_preserving_merge_exec(sort_key, parquet_exec()); - - // need resort as the data was not sorted correctly - let expected = &[ - "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - assert_optimized!(expected, plan.clone(), true); - assert_optimized!(expected, plan, false); - - Ok(()) - } - - #[test] - fn repartition_ignores_sort_preserving_merge() -> Result<()> { - // sort preserving merge already sorted input, - let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("c", &schema).unwrap(), - options: SortOptions::default(), - }]); - let plan = sort_preserving_merge_exec( - sort_key.clone(), - parquet_exec_multiple_sorted(vec![sort_key]), - ); - - // should not sort (as the data was already sorted) - // should not repartition, since increased parallelism is not beneficial for SortPReservingMerge - let expected = &[ - "SortPreservingMergeExec: [c@2 ASC]", - "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", - ]; - - assert_optimized!(expected, plan.clone(), true); - - let expected = &[ - "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - "CoalescePartitionsExec", - "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", - ]; - assert_optimized!(expected, plan, false); - - Ok(()) - } - - #[test] - fn repartition_ignores_sort_preserving_merge_with_union() -> Result<()> { - // 2 sorted parquet files unioned (partitions are concatenated, sort is preserved) - let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("c", &schema).unwrap(), - options: SortOptions::default(), - }]); - let input = union_exec(vec![parquet_exec_with_sort(vec![sort_key.clone()]); 2]); - let plan = sort_preserving_merge_exec(sort_key, input); - - // should not repartition / sort (as the data was already sorted) - let expected = &[ - "SortPreservingMergeExec: [c@2 ASC]", - "UnionExec", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", - ]; - - assert_optimized!(expected, plan.clone(), true); - - let expected = &[ - "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - "CoalescePartitionsExec", - "UnionExec", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", - ]; - assert_optimized!(expected, plan, false); - - Ok(()) - } - - #[test] - fn repartition_does_not_destroy_sort() -> Result<()> { - // SortRequired - // Parquet(sorted) - let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("d", &schema).unwrap(), - options: SortOptions::default(), - }]); - let plan = sort_required_exec_with_req( - filter_exec(parquet_exec_with_sort(vec![sort_key.clone()])), - sort_key, - ); - - // during repartitioning ordering is preserved - let expected = &[ - "SortRequiredExec: [d@3 ASC]", - "FilterExec: c@2 = 0", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[d@3 ASC]", - ]; - - assert_optimized!(expected, plan.clone(), true, true); - assert_optimized!(expected, plan, false, true); - - Ok(()) - } - - #[test] - fn repartition_does_not_destroy_sort_more_complex() -> Result<()> { - // model a more complicated scenario where one child of a union can be repartitioned for performance - // but the other can not be - // - // Union - // SortRequired - // Parquet(sorted) - // Filter - // Parquet(unsorted) - - let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("c", &schema).unwrap(), - options: SortOptions::default(), - }]); - let input1 = sort_required_exec_with_req( - parquet_exec_with_sort(vec![sort_key.clone()]), - sort_key, - ); - let input2 = filter_exec(parquet_exec()); - let plan = union_exec(vec![input1, input2]); - - // should not repartition below the SortRequired as that - // branch doesn't benefit from increased parallelism - let expected = &[ - "UnionExec", - // union input 1: no repartitioning - "SortRequiredExec: [c@2 ASC]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", - // union input 2: should repartition - "FilterExec: c@2 = 0", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - assert_optimized!(expected, plan.clone(), true); - assert_optimized!(expected, plan, false); - - Ok(()) - } - - #[test] - fn repartition_transitively_with_projection() -> Result<()> { - let schema = schema(); - let proj_exprs = vec![( - Arc::new(BinaryExpr::new( - col("a", &schema).unwrap(), - Operator::Plus, - col("b", &schema).unwrap(), - )) as Arc, - "sum".to_string(), - )]; - // non sorted input - let proj = Arc::new(ProjectionExec::try_new(proj_exprs, parquet_exec())?); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("sum", &proj.schema()).unwrap(), - options: SortOptions::default(), - }]); - let plan = sort_preserving_merge_exec(sort_key, proj); - - let expected = &[ - "SortPreservingMergeExec: [sum@0 ASC]", - "SortExec: expr=[sum@0 ASC], preserve_partitioning=[true]", - // Since this projection is not trivial, increasing parallelism is beneficial - "ProjectionExec: expr=[a@0 + b@1 as sum]", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - - assert_optimized!(expected, plan.clone(), true); - - let expected_first_sort_enforcement = &[ - "SortExec: expr=[sum@0 ASC], preserve_partitioning=[false]", - "CoalescePartitionsExec", - // Since this projection is not trivial, increasing parallelism is beneficial - "ProjectionExec: expr=[a@0 + b@1 as sum]", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - assert_optimized!(expected_first_sort_enforcement, plan, false); - - Ok(()) - } - - #[test] - fn repartition_ignores_transitively_with_projection() -> Result<()> { - let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("c", &schema).unwrap(), - options: SortOptions::default(), - }]); - let alias = vec![ - ("a".to_string(), "a".to_string()), - ("b".to_string(), "b".to_string()), - ("c".to_string(), "c".to_string()), - ]; - // sorted input - let plan = sort_required_exec_with_req( - projection_exec_with_alias( - parquet_exec_multiple_sorted(vec![sort_key.clone()]), - alias, - ), - sort_key, - ); - - let expected = &[ - "SortRequiredExec: [c@2 ASC]", - // Since this projection is trivial, increasing parallelism is not beneficial - "ProjectionExec: expr=[a@0 as a, b@1 as b, c@2 as c]", - "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", - ]; - assert_optimized!(expected, plan.clone(), true); - assert_optimized!(expected, plan, false); - - Ok(()) - } - - #[test] - fn repartition_transitively_past_sort_with_projection() -> Result<()> { - let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("c", &schema).unwrap(), - options: SortOptions::default(), - }]); - let alias = vec![ - ("a".to_string(), "a".to_string()), - ("b".to_string(), "b".to_string()), - ("c".to_string(), "c".to_string()), - ]; - let plan = sort_preserving_merge_exec( - sort_key.clone(), - sort_exec( - sort_key, - projection_exec_with_alias(parquet_exec(), alias), - true, - ), - ); - - let expected = &[ - "SortExec: expr=[c@2 ASC], preserve_partitioning=[true]", - // Since this projection is trivial, increasing parallelism is not beneficial - "ProjectionExec: expr=[a@0 as a, b@1 as b, c@2 as c]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - assert_optimized!(expected, plan.clone(), true); - assert_optimized!(expected, plan, false); - - Ok(()) - } - - #[test] - fn repartition_transitively_past_sort_with_filter() -> Result<()> { - let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("a", &schema).unwrap(), - options: SortOptions::default(), - }]); - let plan = sort_exec(sort_key, filter_exec(parquet_exec()), false); - - let expected = &[ - "SortPreservingMergeExec: [a@0 ASC]", - "SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", - // Expect repartition on the input to the sort (as it can benefit from additional parallelism) - "FilterExec: c@2 = 0", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - - assert_optimized!(expected, plan.clone(), true); - - let expected_first_sort_enforcement = &[ - "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - "CoalescePartitionsExec", - "FilterExec: c@2 = 0", - // Expect repartition on the input of the filter (as it can benefit from additional parallelism) - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - assert_optimized!(expected_first_sort_enforcement, plan, false); - - Ok(()) - } - - #[test] - #[cfg(feature = "parquet")] - fn repartition_transitively_past_sort_with_projection_and_filter() -> Result<()> { - let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("a", &schema).unwrap(), - options: SortOptions::default(), - }]); - let plan = sort_exec( - sort_key, - projection_exec_with_alias( - filter_exec(parquet_exec()), - vec![ - ("a".to_string(), "a".to_string()), - ("b".to_string(), "b".to_string()), - ("c".to_string(), "c".to_string()), - ], - ), - false, - ); - - let expected = &[ - "SortPreservingMergeExec: [a@0 ASC]", - // Expect repartition on the input to the sort (as it can benefit from additional parallelism) - "SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", - "ProjectionExec: expr=[a@0 as a, b@1 as b, c@2 as c]", - "FilterExec: c@2 = 0", - // repartition is lowest down - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - - assert_optimized!(expected, plan.clone(), true); - - let expected_first_sort_enforcement = &[ - "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - "CoalescePartitionsExec", - "ProjectionExec: expr=[a@0 as a, b@1 as b, c@2 as c]", - "FilterExec: c@2 = 0", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - assert_optimized!(expected_first_sort_enforcement, plan, false); - - Ok(()) - } - - #[test] - fn parallelization_single_partition() -> Result<()> { - let alias = vec![("a".to_string(), "a".to_string())]; - let plan_parquet = aggregate_exec_with_alias(parquet_exec(), alias.clone()); - let plan_csv = aggregate_exec_with_alias(csv_exec(), alias); - - let expected_parquet = [ - "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", - "RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2", - "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", - "ParquetExec: file_groups={2 groups: [[x:0..50], [x:50..100]]}, projection=[a, b, c, d, e]", - ]; - let expected_csv = [ - "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", - "RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2", - "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", - "CsvExec: file_groups={2 groups: [[x:0..50], [x:50..100]]}, projection=[a, b, c, d, e], has_header=false", - ]; - assert_optimized!(expected_parquet, plan_parquet, true, false, 2, true, 10); - assert_optimized!(expected_csv, plan_csv, true, false, 2, true, 10); - - Ok(()) - } - - #[test] - fn parallelization_multiple_files() -> Result<()> { - let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("a", &schema).unwrap(), - options: SortOptions::default(), - }]); - - let plan = filter_exec(parquet_exec_multiple_sorted(vec![sort_key.clone()])); - let plan = sort_required_exec_with_req(plan, sort_key); - - // The groups must have only contiguous ranges of rows from the same file - // if any group has rows from multiple files, the data is no longer sorted destroyed - // https://github.com/apache/datafusion/issues/8451 - let expected = [ - "SortRequiredExec: [a@0 ASC]", - "FilterExec: c@2 = 0", - "ParquetExec: file_groups={3 groups: [[x:0..50], [y:0..100], [x:50..100]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", ]; - let target_partitions = 3; - let repartition_size = 1; - assert_optimized!( - expected, - plan, - true, - true, - target_partitions, - true, - repartition_size, - false - ); - - let expected = [ - "SortRequiredExec: [a@0 ASC]", - "FilterExec: c@2 = 0", - "ParquetExec: file_groups={8 groups: [[x:0..25], [y:0..25], [x:25..50], [y:25..50], [x:50..75], [y:50..75], [x:75..100], [y:75..100]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", - ]; - let target_partitions = 8; - let repartition_size = 1; - assert_optimized!( - expected, - plan, - true, - true, - target_partitions, - true, - repartition_size, - false - ); - - Ok(()) - } - - #[test] - /// CsvExec on compressed csv file will not be partitioned - /// (Not able to decompress chunked csv file) - fn parallelization_compressed_csv() -> Result<()> { - let compression_types = [ - FileCompressionType::GZIP, - FileCompressionType::BZIP2, - FileCompressionType::XZ, - FileCompressionType::ZSTD, - FileCompressionType::UNCOMPRESSED, - ]; - - let expected_not_partitioned = [ - "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", - "RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2", - "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", - "RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", - "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - ]; - - let expected_partitioned = [ - "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", - "RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2", - "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", - "CsvExec: file_groups={2 groups: [[x:0..50], [x:50..100]]}, projection=[a, b, c, d, e], has_header=false", - ]; - - for compression_type in compression_types { - let expected = if compression_type.is_compressed() { - &expected_not_partitioned[..] - } else { - &expected_partitioned[..] - }; - - let plan = aggregate_exec_with_alias( - Arc::new( - CsvExec::builder( - FileScanConfig::new( - ObjectStoreUrl::parse("test:///").unwrap(), - schema(), - ) - .with_file(PartitionedFile::new("x".to_string(), 100)), - ) - .with_has_header(false) - .with_delimeter(b',') - .with_quote(b'"') - .with_escape(None) - .with_comment(None) - .with_newlines_in_values(false) - .with_file_compression_type(compression_type) - .build(), - ), - vec![("a".to_string(), "a".to_string())], - ); - assert_optimized!(expected, plan, true, false, 2, true, 10, false); - } - Ok(()) - } - - #[test] - fn parallelization_two_partitions() -> Result<()> { - let alias = vec![("a".to_string(), "a".to_string())]; - let plan_parquet = - aggregate_exec_with_alias(parquet_exec_multiple(), alias.clone()); - let plan_csv = aggregate_exec_with_alias(csv_exec_multiple(), alias); - - let expected_parquet = [ - "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", - "RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2", - "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", - // Plan already has two partitions - "ParquetExec: file_groups={2 groups: [[x:0..100], [y:0..100]]}, projection=[a, b, c, d, e]", - ]; - let expected_csv = [ - "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", - "RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2", - "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", - // Plan already has two partitions - "CsvExec: file_groups={2 groups: [[x:0..100], [y:0..100]]}, projection=[a, b, c, d, e], has_header=false", - ]; - assert_optimized!(expected_parquet, plan_parquet, true, false, 2, true, 10); - assert_optimized!(expected_csv, plan_csv, true, false, 2, true, 10); - Ok(()) - } - - #[test] - fn parallelization_two_partitions_into_four() -> Result<()> { - let alias = vec![("a".to_string(), "a".to_string())]; - let plan_parquet = - aggregate_exec_with_alias(parquet_exec_multiple(), alias.clone()); - let plan_csv = aggregate_exec_with_alias(csv_exec_multiple(), alias); - - let expected_parquet = [ - "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", - "RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4", - "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", - // Multiple source files splitted across partitions - "ParquetExec: file_groups={4 groups: [[x:0..50], [x:50..100], [y:0..50], [y:50..100]]}, projection=[a, b, c, d, e]", - ]; - let expected_csv = [ - "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", - "RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4", - "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", - // Multiple source files splitted across partitions - "CsvExec: file_groups={4 groups: [[x:0..50], [x:50..100], [y:0..50], [y:50..100]]}, projection=[a, b, c, d, e], has_header=false", - ]; - assert_optimized!(expected_parquet, plan_parquet, true, false, 4, true, 10); - assert_optimized!(expected_csv, plan_csv, true, false, 4, true, 10); - - Ok(()) - } - - #[test] - fn parallelization_sorted_limit() -> Result<()> { - let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("c", &schema).unwrap(), - options: SortOptions::default(), - }]); - let plan_parquet = limit_exec(sort_exec(sort_key.clone(), parquet_exec(), false)); - let plan_csv = limit_exec(sort_exec(sort_key, csv_exec(), false)); - - let expected_parquet = &[ - "GlobalLimitExec: skip=0, fetch=100", - "LocalLimitExec: fetch=100", - // data is sorted so can't repartition here - "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - // Doesn't parallelize for SortExec without preserve_partitioning - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - let expected_csv = &[ - "GlobalLimitExec: skip=0, fetch=100", - "LocalLimitExec: fetch=100", - // data is sorted so can't repartition here - "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - // Doesn't parallelize for SortExec without preserve_partitioning - "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - ]; - assert_optimized!(expected_parquet, plan_parquet, true); - assert_optimized!(expected_csv, plan_csv, true); - - Ok(()) - } - - #[test] - fn parallelization_limit_with_filter() -> Result<()> { - let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("c", &schema).unwrap(), - options: SortOptions::default(), - }]); - let plan_parquet = limit_exec(filter_exec(sort_exec( - sort_key.clone(), - parquet_exec(), - false, - ))); - let plan_csv = limit_exec(filter_exec(sort_exec(sort_key, csv_exec(), false))); - - let expected_parquet = &[ - "GlobalLimitExec: skip=0, fetch=100", - "CoalescePartitionsExec", - "LocalLimitExec: fetch=100", - "FilterExec: c@2 = 0", - // even though data is sorted, we can use repartition here. Since - // ordering is not used in subsequent stages anyway. - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - // SortExec doesn't benefit from input partitioning - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - let expected_csv = &[ - "GlobalLimitExec: skip=0, fetch=100", - "CoalescePartitionsExec", - "LocalLimitExec: fetch=100", - "FilterExec: c@2 = 0", - // even though data is sorted, we can use repartition here. Since - // ordering is not used in subsequent stages anyway. - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "SortExec: expr=[c@2 ASC], preserve_partitioning=[false]", - // SortExec doesn't benefit from input partitioning - "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - ]; - assert_optimized!(expected_parquet, plan_parquet, true); - assert_optimized!(expected_csv, plan_csv, true); - - Ok(()) - } - - #[test] - fn parallelization_ignores_limit() -> Result<()> { - let alias = vec![("a".to_string(), "a".to_string())]; - let plan_parquet = aggregate_exec_with_alias( - limit_exec(filter_exec(limit_exec(parquet_exec()))), - alias.clone(), - ); - let plan_csv = aggregate_exec_with_alias( - limit_exec(filter_exec(limit_exec(csv_exec()))), - alias, - ); - - let expected_parquet = &[ - "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", - "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "GlobalLimitExec: skip=0, fetch=100", - "CoalescePartitionsExec", - "LocalLimitExec: fetch=100", - "FilterExec: c@2 = 0", - // repartition should happen prior to the filter to maximize parallelism - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "GlobalLimitExec: skip=0, fetch=100", - // Limit doesn't benefit from input partitioning - no parallelism - "LocalLimitExec: fetch=100", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - let expected_csv = &[ - "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", - "RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=10", - "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "GlobalLimitExec: skip=0, fetch=100", - "CoalescePartitionsExec", - "LocalLimitExec: fetch=100", - "FilterExec: c@2 = 0", - // repartition should happen prior to the filter to maximize parallelism - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "GlobalLimitExec: skip=0, fetch=100", - // Limit doesn't benefit from input partitioning - no parallelism - "LocalLimitExec: fetch=100", - "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - ]; - assert_optimized!(expected_parquet, plan_parquet, true); - assert_optimized!(expected_csv, plan_csv, true); - - Ok(()) - } - - #[test] - fn parallelization_union_inputs() -> Result<()> { - let plan_parquet = union_exec(vec![parquet_exec(); 5]); - let plan_csv = union_exec(vec![csv_exec(); 5]); - - let expected_parquet = &[ - "UnionExec", - // Union doesn't benefit from input partitioning - no parallelism - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - let expected_csv = &[ - "UnionExec", - // Union doesn't benefit from input partitioning - no parallelism - "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], has_header=false", - ]; - assert_optimized!(expected_parquet, plan_parquet, true); - assert_optimized!(expected_csv, plan_csv, true); - - Ok(()) - } - - #[test] - fn parallelization_prior_to_sort_preserving_merge() -> Result<()> { - let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("c", &schema).unwrap(), - options: SortOptions::default(), - }]); - // sort preserving merge already sorted input, - let plan_parquet = sort_preserving_merge_exec( - sort_key.clone(), - parquet_exec_with_sort(vec![sort_key.clone()]), - ); - let plan_csv = sort_preserving_merge_exec( - sort_key.clone(), - csv_exec_with_sort(vec![sort_key]), - ); - - // parallelization is not beneficial for SortPreservingMerge - let expected_parquet = &[ - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", - ]; - let expected_csv = &[ - "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], has_header=false", - ]; - assert_optimized!(expected_parquet, plan_parquet, true); - assert_optimized!(expected_csv, plan_csv, true); - - Ok(()) - } - - #[test] - fn parallelization_sort_preserving_merge_with_union() -> Result<()> { - let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("c", &schema).unwrap(), - options: SortOptions::default(), - }]); - // 2 sorted parquet files unioned (partitions are concatenated, sort is preserved) - let input_parquet = - union_exec(vec![parquet_exec_with_sort(vec![sort_key.clone()]); 2]); - let input_csv = union_exec(vec![csv_exec_with_sort(vec![sort_key.clone()]); 2]); - let plan_parquet = sort_preserving_merge_exec(sort_key.clone(), input_parquet); - let plan_csv = sort_preserving_merge_exec(sort_key, input_csv); - - // should not repartition (union doesn't benefit from increased parallelism) - // should not sort (as the data was already sorted) - let expected_parquet = &[ - "SortPreservingMergeExec: [c@2 ASC]", - "UnionExec", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", - ]; - let expected_csv = &[ - "SortPreservingMergeExec: [c@2 ASC]", - "UnionExec", - "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], has_header=false", - "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], has_header=false", - ]; - assert_optimized!(expected_parquet, plan_parquet, true); - assert_optimized!(expected_csv, plan_csv, true); - - Ok(()) - } - - #[test] - fn parallelization_does_not_benefit() -> Result<()> { - let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("c", &schema).unwrap(), - options: SortOptions::default(), - }]); - // SortRequired - // Parquet(sorted) - let plan_parquet = sort_required_exec_with_req( - parquet_exec_with_sort(vec![sort_key.clone()]), - sort_key.clone(), - ); - let plan_csv = sort_required_exec_with_req( - csv_exec_with_sort(vec![sort_key.clone()]), - sort_key, - ); - - // no parallelization, because SortRequiredExec doesn't benefit from increased parallelism - let expected_parquet = &[ - "SortRequiredExec: [c@2 ASC]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", - ]; - let expected_csv = &[ - "SortRequiredExec: [c@2 ASC]", - "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], has_header=false", - ]; - assert_optimized!(expected_parquet, plan_parquet, true); - assert_optimized!(expected_csv, plan_csv, true); - - Ok(()) - } - - #[test] - fn parallelization_ignores_transitively_with_projection_parquet() -> Result<()> { - // sorted input - let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("c", &schema).unwrap(), - options: SortOptions::default(), - }]); - - //Projection(a as a2, b as b2) - let alias_pairs: Vec<(String, String)> = vec![ - ("a".to_string(), "a2".to_string()), - ("c".to_string(), "c2".to_string()), - ]; - let proj_parquet = projection_exec_with_alias( - parquet_exec_with_sort(vec![sort_key]), - alias_pairs, - ); - let sort_key_after_projection = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("c2", &proj_parquet.schema()).unwrap(), - options: SortOptions::default(), - }]); - let plan_parquet = - sort_preserving_merge_exec(sort_key_after_projection, proj_parquet); - let expected = &[ - "SortPreservingMergeExec: [c2@1 ASC]", - " ProjectionExec: expr=[a@0 as a2, c@2 as c2]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", - ]; - plans_matches_expected!(expected, &plan_parquet); - - // data should not be repartitioned / resorted - let expected_parquet = &[ - "ProjectionExec: expr=[a@0 as a2, c@2 as c2]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", - ]; - assert_optimized!(expected_parquet, plan_parquet, true); - - Ok(()) - } - - #[test] - fn parallelization_ignores_transitively_with_projection_csv() -> Result<()> { - // sorted input - let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("c", &schema).unwrap(), - options: SortOptions::default(), - }]); - - //Projection(a as a2, b as b2) - let alias_pairs: Vec<(String, String)> = vec![ - ("a".to_string(), "a2".to_string()), - ("c".to_string(), "c2".to_string()), - ]; - - let proj_csv = - projection_exec_with_alias(csv_exec_with_sort(vec![sort_key]), alias_pairs); - let sort_key_after_projection = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("c2", &proj_csv.schema()).unwrap(), - options: SortOptions::default(), - }]); - let plan_csv = sort_preserving_merge_exec(sort_key_after_projection, proj_csv); - let expected = &[ - "SortPreservingMergeExec: [c2@1 ASC]", - " ProjectionExec: expr=[a@0 as a2, c@2 as c2]", - " CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], has_header=false", - ]; - plans_matches_expected!(expected, &plan_csv); - - // data should not be repartitioned / resorted - let expected_csv = &[ - "ProjectionExec: expr=[a@0 as a2, c@2 as c2]", - "CsvExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC], has_header=false", - ]; - assert_optimized!(expected_csv, plan_csv, true); - - Ok(()) - } - - #[test] - fn remove_redundant_roundrobins() -> Result<()> { - let input = parquet_exec(); - let repartition = repartition_exec(repartition_exec(input)); - let physical_plan = repartition_exec(filter_exec(repartition)); - let expected = &[ - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10", - " FilterExec: c@2 = 0", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - plans_matches_expected!(expected, &physical_plan); - - let expected = &[ - "FilterExec: c@2 = 0", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - assert_optimized!(expected, physical_plan.clone(), true); - assert_optimized!(expected, physical_plan, false); - - Ok(()) - } - - #[test] - fn remove_unnecessary_spm_after_filter() -> Result<()> { - let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("c", &schema).unwrap(), - options: SortOptions::default(), - }]); - let input = parquet_exec_multiple_sorted(vec![sort_key.clone()]); - let physical_plan = sort_preserving_merge_exec(sort_key, filter_exec(input)); - - // Original plan expects its output to be ordered by c@2 ASC. - // This is still satisfied since, after filter that column is constant. - let expected = &[ - "CoalescePartitionsExec", - "FilterExec: c@2 = 0", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2, preserve_order=true, sort_exprs=c@2 ASC", - "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", - ]; - // last flag sets config.optimizer.PREFER_EXISTING_SORT - assert_optimized!(expected, physical_plan.clone(), true, true); - assert_optimized!(expected, physical_plan, false, true); - - Ok(()) - } - - #[test] - fn preserve_ordering_through_repartition() -> Result<()> { - let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("d", &schema).unwrap(), - options: SortOptions::default(), - }]); - let input = parquet_exec_multiple_sorted(vec![sort_key.clone()]); - let physical_plan = sort_preserving_merge_exec(sort_key, filter_exec(input)); - - let expected = &[ - "SortPreservingMergeExec: [d@3 ASC]", - "FilterExec: c@2 = 0", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2, preserve_order=true, sort_exprs=d@3 ASC", - "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[d@3 ASC]", - ]; - // last flag sets config.optimizer.PREFER_EXISTING_SORT - assert_optimized!(expected, physical_plan.clone(), true, true); - assert_optimized!(expected, physical_plan, false, true); - - Ok(()) - } - - #[test] - fn do_not_preserve_ordering_through_repartition() -> Result<()> { - let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("a", &schema).unwrap(), - options: SortOptions::default(), - }]); - let input = parquet_exec_multiple_sorted(vec![sort_key.clone()]); - let physical_plan = sort_preserving_merge_exec(sort_key, filter_exec(input)); - - let expected = &[ - "SortPreservingMergeExec: [a@0 ASC]", - "SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", - "FilterExec: c@2 = 0", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", - "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", - ]; - - assert_optimized!(expected, physical_plan.clone(), true); - - let expected = &[ - "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - "CoalescePartitionsExec", - "FilterExec: c@2 = 0", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", - "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", - ]; - assert_optimized!(expected, physical_plan, false); - - Ok(()) - } - - #[test] - fn no_need_for_sort_after_filter() -> Result<()> { - let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("c", &schema).unwrap(), - options: SortOptions::default(), - }]); - let input = parquet_exec_multiple_sorted(vec![sort_key.clone()]); - let physical_plan = sort_preserving_merge_exec(sort_key, filter_exec(input)); - - let expected = &[ - // After CoalescePartitionsExec c is still constant. Hence c@2 ASC ordering is already satisfied. - "CoalescePartitionsExec", - // Since after this stage c is constant. c@2 ASC ordering is already satisfied. - "FilterExec: c@2 = 0", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", - "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", - ]; - assert_optimized!(expected, physical_plan.clone(), true); - assert_optimized!(expected, physical_plan, false); - - Ok(()) - } - - #[test] - fn do_not_preserve_ordering_through_repartition2() -> Result<()> { - let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("c", &schema).unwrap(), - options: SortOptions::default(), - }]); - let input = parquet_exec_multiple_sorted(vec![sort_key]); - - let sort_req = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("a", &schema).unwrap(), - options: SortOptions::default(), - }]); - let physical_plan = sort_preserving_merge_exec(sort_req, filter_exec(input)); - - let expected = &[ - "SortPreservingMergeExec: [a@0 ASC]", - "SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", - "FilterExec: c@2 = 0", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", - "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", - ]; - - assert_optimized!(expected, physical_plan.clone(), true); - - let expected = &[ - "SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - "CoalescePartitionsExec", - "SortExec: expr=[a@0 ASC], preserve_partitioning=[true]", - "FilterExec: c@2 = 0", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", - "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", - ]; - assert_optimized!(expected, physical_plan, false); - - Ok(()) - } - - #[test] - fn do_not_preserve_ordering_through_repartition3() -> Result<()> { - let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("c", &schema).unwrap(), - options: SortOptions::default(), - }]); - let input = parquet_exec_multiple_sorted(vec![sort_key]); - let physical_plan = filter_exec(input); - - let expected = &[ - "FilterExec: c@2 = 0", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", - "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", - ]; - assert_optimized!(expected, physical_plan.clone(), true); - assert_optimized!(expected, physical_plan, false); - - Ok(()) - } - - #[test] - fn do_not_put_sort_when_input_is_invalid() -> Result<()> { - let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("a", &schema).unwrap(), - options: SortOptions::default(), - }]); - let input = parquet_exec(); - let physical_plan = sort_required_exec_with_req(filter_exec(input), sort_key); - let expected = &[ - // Ordering requirement of sort required exec is NOT satisfied - // by existing ordering at the source. - "SortRequiredExec: [a@0 ASC]", - "FilterExec: c@2 = 0", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - assert_plan_txt!(expected, physical_plan); - - let expected = &[ - "SortRequiredExec: [a@0 ASC]", - // Since at the start of the rule ordering requirement is not satisfied - // EnforceDistribution rule doesn't satisfy this requirement either. - "FilterExec: c@2 = 0", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - - let mut config = ConfigOptions::new(); - config.execution.target_partitions = 10; - config.optimizer.enable_round_robin_repartition = true; - config.optimizer.prefer_existing_sort = false; - let dist_plan = EnforceDistribution::new().optimize(physical_plan, &config)?; - assert_plan_txt!(expected, dist_plan); - - Ok(()) - } - - #[test] - fn put_sort_when_input_is_valid() -> Result<()> { - let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("a", &schema).unwrap(), - options: SortOptions::default(), - }]); - let input = parquet_exec_multiple_sorted(vec![sort_key.clone()]); - let physical_plan = sort_required_exec_with_req(filter_exec(input), sort_key); - - let expected = &[ - // Ordering requirement of sort required exec is satisfied - // by existing ordering at the source. - "SortRequiredExec: [a@0 ASC]", - "FilterExec: c@2 = 0", - "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", - ]; - assert_plan_txt!(expected, physical_plan); - - let expected = &[ - // Since at the start of the rule ordering requirement is satisfied - // EnforceDistribution rule satisfy this requirement also. - "SortRequiredExec: [a@0 ASC]", - "FilterExec: c@2 = 0", - "ParquetExec: file_groups={10 groups: [[x:0..20], [y:0..20], [x:20..40], [y:20..40], [x:40..60], [y:40..60], [x:60..80], [y:60..80], [x:80..100], [y:80..100]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", - ]; - - let mut config = ConfigOptions::new(); - config.execution.target_partitions = 10; - config.optimizer.enable_round_robin_repartition = true; - config.optimizer.prefer_existing_sort = false; - let dist_plan = EnforceDistribution::new().optimize(physical_plan, &config)?; - assert_plan_txt!(expected, dist_plan); - - Ok(()) - } - - #[test] - fn do_not_add_unnecessary_hash() -> Result<()> { - let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("c", &schema).unwrap(), - options: SortOptions::default(), - }]); - let alias = vec![("a".to_string(), "a".to_string())]; - let input = parquet_exec_with_sort(vec![sort_key]); - let physical_plan = aggregate_exec_with_alias(input, alias); - - let expected = &[ - "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", - "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", - ]; - // Make sure target partition number is 1. In this case hash repartition is unnecessary - assert_optimized!(expected, physical_plan.clone(), true, false, 1, false, 1024); - assert_optimized!(expected, physical_plan, false, false, 1, false, 1024); - - Ok(()) - } - - #[test] - fn do_not_add_unnecessary_hash2() -> Result<()> { - let schema = schema(); - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("c", &schema).unwrap(), - options: SortOptions::default(), - }]); - let alias = vec![("a".to_string(), "a".to_string())]; - let input = parquet_exec_multiple_sorted(vec![sort_key]); - let aggregate = aggregate_exec_with_alias(input, alias.clone()); - let physical_plan = aggregate_exec_with_alias(aggregate, alias); - - let expected = &[ - "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", - // Since hash requirements of this operator is satisfied. There shouldn't be - // a hash repartition here - "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", - "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", - "RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4", - "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[]", - "RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=2", - "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", - ]; - // Make sure target partition number is larger than 2 (e.g partition number at the source). - assert_optimized!(expected, physical_plan.clone(), true, false, 4, false, 1024); - assert_optimized!(expected, physical_plan, false, false, 4, false, 1024); - - Ok(()) - } - - #[test] - fn optimize_away_unnecessary_repartition() -> Result<()> { - let physical_plan = coalesce_partitions_exec(repartition_exec(parquet_exec())); - let expected = &[ - "CoalescePartitionsExec", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - plans_matches_expected!(expected, physical_plan.clone()); - - let expected = - &["ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]"]; - assert_optimized!(expected, physical_plan.clone(), true); - assert_optimized!(expected, physical_plan, false); - - Ok(()) - } - - #[test] - fn optimize_away_unnecessary_repartition2() -> Result<()> { - let physical_plan = filter_exec(repartition_exec(coalesce_partitions_exec( - filter_exec(repartition_exec(parquet_exec())), - ))); - let expected = &[ - "FilterExec: c@2 = 0", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " CoalescePartitionsExec", - " FilterExec: c@2 = 0", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - plans_matches_expected!(expected, physical_plan.clone()); - - let expected = &[ - "FilterExec: c@2 = 0", - "FilterExec: c@2 = 0", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e]", - ]; - assert_optimized!(expected, physical_plan.clone(), true); - assert_optimized!(expected, physical_plan, false); - - Ok(()) - } -} From 7cd4b3133c7e03201bc142f110ed4858d7d0d7b7 Mon Sep 17 00:00:00 2001 From: buraksenn Date: Tue, 21 Jan 2025 01:39:07 +0300 Subject: [PATCH 07/24] passes lint --- datafusion-cli/Cargo.lock | 1 + .../src/physical_optimizer/enforce_sorting.rs | 2585 ----------------- datafusion/core/src/physical_optimizer/mod.rs | 5 - .../replace_with_order_preserving_variants.rs | 1572 ---------- .../core/src/physical_optimizer/test_utils.rs | 54 - .../core/src/physical_optimizer/utils.rs | 25 - datafusion/core/src/test/mod.rs | 95 - .../physical_optimizer/enforce_sorting.rs | 1945 +++++++++++++ .../core/tests/physical_optimizer/mod.rs | 2 + .../replace_with_order_preserving_variants.rs | 1250 ++++++++ .../tests/physical_optimizer/test_util.rs | 122 +- datafusion/physical-optimizer/Cargo.toml | 1 + .../src/enforce_sorting/mod.rs | 656 +++++ .../replace_with_order_preserving_variants.rs | 280 ++ .../src/enforce_sorting}/sort_pushdown.rs | 29 +- .../src/enforce_sorting/utils.rs | 105 + datafusion/physical-optimizer/src/lib.rs | 1 + 17 files changed, 4377 insertions(+), 4351 deletions(-) delete mode 100644 datafusion/core/src/physical_optimizer/enforce_sorting.rs delete mode 100644 datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs delete mode 100644 datafusion/core/src/physical_optimizer/test_utils.rs create mode 100644 datafusion/core/tests/physical_optimizer/enforce_sorting.rs create mode 100644 datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs create mode 100644 datafusion/physical-optimizer/src/enforce_sorting/mod.rs create mode 100644 datafusion/physical-optimizer/src/enforce_sorting/replace_with_order_preserving_variants.rs rename datafusion/{core/src/physical_optimizer => physical-optimizer/src/enforce_sorting}/sort_pushdown.rs (96%) create mode 100644 datafusion/physical-optimizer/src/enforce_sorting/utils.rs diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index 8c7f2113eedb..8111442824db 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -1598,6 +1598,7 @@ dependencies = [ "itertools 0.14.0", "log", "recursive", + "url", ] [[package]] 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 167f9d6d45e7..000000000000 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ /dev/null @@ -1,2585 +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))) 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 plan = if let Some(ordering) = plan.output_ordering() { - Arc::new(SortPreservingMergeExec::new( - LexOrdering::new(ordering.to_vec()), - plan, - )) 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..22a20d7b9375 100644 --- a/datafusion/core/src/physical_optimizer/mod.rs +++ b/datafusion/core/src/physical_optimizer/mod.rs @@ -23,14 +23,9 @@ //! [`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 f32ffa8a5830..000000000000 --- a/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs +++ /dev/null @@ -1,1572 +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, -) -> 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) - } 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)); - 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, - )?; - - // 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/physical_optimizer/utils.rs b/datafusion/core/src/physical_optimizer/utils.rs index 9f2c28d564f0..ca52807d2f8b 100644 --- a/datafusion/core/src/physical_optimizer/utils.rs +++ b/datafusion/core/src/physical_optimizer/utils.rs @@ -23,13 +23,10 @@ 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_plan::limit::{GlobalLimitExec, LocalLimitExec}; use datafusion_physical_plan::tree_node::PlanContext; /// This utility function adds a `SortExec` above an operator according to the @@ -72,23 +69,6 @@ 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 window; -/// i.e. either a [`WindowAggExec`] or a [`BoundedWindowAggExec`]. -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 [`SortPreservingMergeExec`]. pub fn is_sort_preserving_merge(plan: &Arc) -> bool { plan.as_any().is::() @@ -99,11 +79,6 @@ 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::() 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..68ac2e8b54a0 --- /dev/null +++ b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs @@ -0,0 +1,1945 @@ +// 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..6def63feea5d --- /dev/null +++ b/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs @@ -0,0 +1,1250 @@ +use std::sync::Arc; + +use datafusion::prelude::{SessionConfig, SessionContext}; +use datafusion_execution::TaskContext; +use datafusion_physical_optimizer::test_utils::check_integrity; +use datafusion_physical_plan::filter::FilterExec; +use datafusion_physical_plan::joins::{HashJoinExec, PartitionMode}; +use datafusion_physical_plan::sorts::sort::SortExec; +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::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 object_store::memory::InMemory; +use object_store::ObjectStore; +use url::Url; + +use crate::physical_optimizer::test_util::stream_exec_ordered; +use datafusion_physical_plan::coalesce_batches::CoalesceBatchesExec; +use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; +use datafusion_physical_plan::repartition::RepartitionExec; +use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; +use rstest::rstest; +use datafusion_physical_optimizer::enforce_sorting::replace_with_order_preserving_variants::{replace_with_order_preserving_variants, OrderPreservationContext}; +use datafusion_common::tree_node::{TreeNode, TransformedResult}; + +/// 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(&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 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..df75bfb818bf 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,118 @@ 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(), + ) +} + +/// 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..f84911b3bb4b --- /dev/null +++ b/datafusion/physical-optimizer/src/enforce_sorting/mod.rs @@ -0,0 +1,656 @@ +// 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; +mod utils; + +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::enforce_sorting::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))) 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 plan = if let Some(ordering) = plan.output_ordering() { + Arc::new(SortPreservingMergeExec::new( + LexOrdering::new(ordering.to_vec()), + plan, + )) 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..f6d35086f5df --- /dev/null +++ b/datafusion/physical-optimizer/src/enforce_sorting/replace_with_order_preserving_variants.rs @@ -0,0 +1,280 @@ +// 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_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, +) -> 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) + } 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)); + 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, + )?; + + // 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 96% rename from datafusion/core/src/physical_optimizer/sort_pushdown.rs rename to datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs index 6c761f674b3b..8cb29f2688f5 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::enforce_sorting::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, @@ -68,7 +69,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/enforce_sorting/utils.rs b/datafusion/physical-optimizer/src/enforce_sorting/utils.rs new file mode 100644 index 000000000000..e0a59b32d6e9 --- /dev/null +++ b/datafusion/physical-optimizer/src/enforce_sorting/utils.rs @@ -0,0 +1,105 @@ +// 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_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}; +use std::sync::Arc; + +/// Checks whether the given operator is a [`SortExec`]. +pub fn is_sort(plan: &Arc) -> bool { + plan.as_any().is::() +} + +/// This utility function adds a `SortExec` above an operator according to the +/// given ordering requirements while preserving the original partitioning. +pub fn add_sort_above( + node: PlanContext, + sort_requirements: LexRequirement, + fetch: Option, +) -> PlanContext { + let mut sort_expr = LexOrdering::from(sort_requirements); + sort_expr.retain(|sort_expr| { + !node + .plan + .equivalence_properties() + .is_expr_constant(&sort_expr.expr) + }); + let mut new_sort = SortExec::new(sort_expr, Arc::clone(&node.plan)).with_fetch(fetch); + if node.plan.output_partitioning().partition_count() > 1 { + new_sort = new_sort.with_preserve_partitioning(true); + } + PlanContext::new(Arc::new(new_sort), T::default(), vec![node]) +} + +/// Checks whether the given operator is a window; +/// i.e. either a [`WindowAggExec`] or a [`BoundedWindowAggExec`]. +pub fn is_window(plan: &Arc) -> bool { + plan.as_any().is::() || 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`]. +pub fn is_sort_preserving_merge(plan: &Arc) -> bool { + plan.as_any().is::() +} + +/// Checks whether the given operator is a [`CoalescePartitionsExec`]. +pub fn is_coalesce_partitions(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::() +} + +/// This utility function adds a `SortExec` above an operator according to the +/// given ordering requirements while preserving the original partitioning. If +/// requirement is already satisfied no `SortExec` is added. +pub fn add_sort_above_with_check( + node: PlanContext, + sort_requirements: LexRequirement, + fetch: Option, +) -> PlanContext { + if !node + .plan + .equivalence_properties() + .ordering_satisfy_requirement(&sort_requirements) + { + add_sort_above(node, sort_requirements, fetch) + } else { + node + } +} + +/// 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::() +} diff --git a/datafusion/physical-optimizer/src/lib.rs b/datafusion/physical-optimizer/src/lib.rs index ccb18f679171..1c7575cd3145 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; From eb8ac39e38d5c0a22bd7265174929e7bee12a70c Mon Sep 17 00:00:00 2001 From: buraksenn Date: Tue, 21 Jan 2025 01:45:15 +0300 Subject: [PATCH 08/24] forgotten license header --- .../replace_with_order_preserving_variants.rs | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs b/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs index 6def63feea5d..0c33ec331532 100644 --- a/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs +++ b/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs @@ -1,3 +1,21 @@ +// 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}; From 8103f0042e294533af89094d4d606f8ab88740ba Mon Sep 17 00:00:00 2001 From: logan-keede Date: Tue, 21 Jan 2025 04:22:59 +0530 Subject: [PATCH 09/24] move enforce_sorting --- .../enforce_distribution.rs | 2 +- .../physical-optimizer/src/enforce_sorting.rs | 2585 +++++++++++++++++ datafusion/physical-optimizer/src/lib.rs | 3 + .../replace_with_order_preserving_variants.rs | 1572 ++++++++++ .../physical-optimizer/src/sort_pushdown.rs | 719 +++++ datafusion/physical-optimizer/src/utils.rs | 30 +- 6 files changed, 4895 insertions(+), 16 deletions(-) create mode 100644 datafusion/physical-optimizer/src/enforce_sorting.rs create mode 100644 datafusion/physical-optimizer/src/replace_with_order_preserving_variants.rs create mode 100644 datafusion/physical-optimizer/src/sort_pushdown.rs diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index 5575f50a38ec..9dca54fe30b7 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -5,7 +5,7 @@ use datafusion::datasource::file_format::file_compression_type::FileCompressionT use datafusion::datasource::listing::PartitionedFile; use datafusion::datasource::object_store::ObjectStoreUrl; use datafusion::datasource::physical_plan::{CsvExec, FileScanConfig, ParquetExec}; -use datafusion::physical_plan::enforce_sorting::EnforceSorting; +use datafusion_physical_optimizer::enforce_sorting::EnforceSorting; use datafusion_physical_plan::coalesce_batches::CoalesceBatchesExec; use datafusion_physical_plan::expressions::col; use datafusion_physical_plan::filter::FilterExec; diff --git a/datafusion/physical-optimizer/src/enforce_sorting.rs b/datafusion/physical-optimizer/src/enforce_sorting.rs new file mode 100644 index 000000000000..88ca34eb5bb3 --- /dev/null +++ b/datafusion/physical-optimizer/src/enforce_sorting.rs @@ -0,0 +1,2585 @@ +// 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::optimizer::PhysicalOptimizerRule; +use datafusion_common::config::ConfigOptions; +use datafusion_common::error::Result; +use crate::replace_with_order_preserving_variants::{ + replace_with_order_preserving_variants, OrderPreservationContext, +}; +use crate::sort_pushdown::{ + assign_initial_requirements, pushdown_sorts, SortPushDown, +}; +use crate::utils::{ + is_coalesce_partitions, is_limit, is_repartition, is_sort, is_sort_preserving_merge, + is_union, is_window, +}; +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 datafusion_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_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))) 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 plan = if let Some(ordering) = plan.output_ordering() { + Arc::new(SortPreservingMergeExec::new( + LexOrdering::new(ordering.to_vec()), + plan, + )) 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/physical-optimizer/src/lib.rs b/datafusion/physical-optimizer/src/lib.rs index a5db5fbf4fd2..6fb6602cbbcd 100644 --- a/datafusion/physical-optimizer/src/lib.rs +++ b/datafusion/physical-optimizer/src/lib.rs @@ -22,6 +22,8 @@ pub mod aggregate_statistics; pub mod coalesce_batches; pub mod combine_partial_final_agg; pub mod enforce_distribution; +pub mod enforce_sorting; +pub mod replace_with_order_preserving_variants; pub mod join_selection; pub mod limit_pushdown; pub mod limited_distinct_aggregation; @@ -34,3 +36,4 @@ pub mod topk_aggregation; pub mod update_aggr_exprs; pub use optimizer::PhysicalOptimizerRule; mod utils; +mod sort_pushdown; diff --git a/datafusion/physical-optimizer/src/replace_with_order_preserving_variants.rs b/datafusion/physical-optimizer/src/replace_with_order_preserving_variants.rs new file mode 100644 index 000000000000..c3be87d5cc8c --- /dev/null +++ b/datafusion/physical-optimizer/src/replace_with_order_preserving_variants.rs @@ -0,0 +1,1572 @@ +// 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 datafusion_common::error::Result; +use crate::utils::{is_coalesce_partitions, is_sort}; +use datafusion_physical_plan::repartition::RepartitionExec; +use datafusion_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, +) -> 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) + } 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)); + 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, + )?; + + // 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/physical-optimizer/src/sort_pushdown.rs b/datafusion/physical-optimizer/src/sort_pushdown.rs new file mode 100644 index 000000000000..a2a0bf7ad309 --- /dev/null +++ b/datafusion/physical-optimizer/src/sort_pushdown.rs @@ -0,0 +1,719 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +use std::fmt::Debug; +use std::sync::Arc; + +use super::utils::{add_sort_above, is_sort}; +use crate::utils::{is_sort_preserving_merge, is_union, is_window}; +use datafusion_physical_plan::filter::FilterExec; +use datafusion_physical_plan::joins::utils::calculate_join_output_ordering; +use datafusion_physical_plan::joins::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}; +use arrow_schema::SchemaRef; + +use datafusion_common::tree_node::{ + ConcreteTreeNode, Transformed, TreeNode, TreeNodeRecursion, +}; +use datafusion_common::{plan_err, HashSet, JoinSide, Result}; +use datafusion_expr::JoinType; +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; + +/// 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 +/// computational cost by pushing down `SortExec`s through some executors. The +/// 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 +#[derive(Default, Clone)] +pub struct ParentRequirements { + ordering_requirement: Option, + fetch: Option, +} + +pub type SortPushDown = PlanContext; + +/// Assigns the ordering requirement of the root node to the its children. +pub fn assign_initial_requirements(node: &mut SortPushDown) { + let reqs = node.plan.required_input_ordering(); + for (child, requirement) in node.children.iter_mut().zip(reqs) { + child.data = ParentRequirements { + ordering_requirement: requirement, + fetch: None, + }; + } +} + +pub(crate) 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)?; + } + let (new_node, children) = new_node.data.take_children(); + let new_children = children + .into_iter() + .map(pushdown_sorts) + .collect::>()?; + new_node.with_new_children(new_children) +} + +fn pushdown_sorts_helper( + mut requirements: SortPushDown, +) -> Result> { + let plan = &requirements.plan; + let parent_reqs = requirements + .data + .ordering_requirement + .clone() + .unwrap_or_default(); + let satisfy_parent = plan + .equivalence_properties() + .ordering_satisfy_requirement(&parent_reqs); + + if is_sort(plan) { + let required_ordering = plan + .output_ordering() + .cloned() + .map(LexRequirement::from) + .unwrap_or_default(); + if !satisfy_parent { + // Make sure this `SortExec` satisfies parent requirements: + let sort_reqs = requirements.data.ordering_requirement.unwrap_or_default(); + let fetch = requirements.data.fetch; + requirements = requirements.children.swap_remove(0); + requirements = add_sort_above(requirements, sort_reqs, fetch); + }; + + // We can safely get the 0th index as we are dealing with a `SortExec`. + let mut child = requirements.children.swap_remove(0); + if let Some(adjusted) = + pushdown_requirement_to_children(&child.plan, &required_ordering)? + { + let fetch = child.plan.fetch(); + for (grand_child, order) in child.children.iter_mut().zip(adjusted) { + grand_child.data = ParentRequirements { + ordering_requirement: order, + fetch, + }; + } + // Can push down requirements + child.data = ParentRequirements { + ordering_requirement: Some(required_ordering), + fetch, + }; + + return Ok(Transformed { + data: child, + transformed: true, + tnr: TreeNodeRecursion::Stop, + }); + } else { + // Can not push down requirements + requirements.children = vec![child]; + assign_initial_requirements(&mut requirements); + } + } else if satisfy_parent { + // For non-sort operators, immediately return if parent requirements are met: + let reqs = plan.required_input_ordering(); + for (child, order) in requirements.children.iter_mut().zip(reqs) { + child.data.ordering_requirement = order; + } + } else if let Some(adjusted) = pushdown_requirement_to_children(plan, &parent_reqs)? { + // Can not satisfy the parent requirements, check whether we can push + // requirements down: + for (child, order) in requirements.children.iter_mut().zip(adjusted) { + child.data.ordering_requirement = order; + } + requirements.data.ordering_requirement = None; + } else { + // Can not push down requirements, add new `SortExec`: + let sort_reqs = requirements + .data + .ordering_requirement + .clone() + .unwrap_or_default(); + let fetch = requirements.data.fetch; + requirements = add_sort_above(requirements, sort_reqs, fetch); + assign_initial_requirements(&mut requirements); + } + Ok(Transformed::yes(requirements)) +} + +fn pushdown_requirement_to_children( + plan: &Arc, + parent_required: &LexRequirement, +) -> Result>>> { + let maintains_input_order = plan.maintains_input_order(); + if is_window(plan) { + let required_input_ordering = plan.required_input_ordering(); + let request_child = required_input_ordering[0].clone().unwrap_or_default(); + let child_plan = plan.children().swap_remove(0); + + match determine_children_requirement(parent_required, &request_child, child_plan) + { + RequirementsCompatibility::Satisfy => { + let req = (!request_child.is_empty()) + .then(|| LexRequirement::new(request_child.to_vec())); + Ok(Some(vec![req])) + } + RequirementsCompatibility::Compatible(adjusted) => Ok(Some(vec![adjusted])), + RequirementsCompatibility::NonCompatible => Ok(None), + } + } else if let Some(sort_exec) = plan.as_any().downcast_ref::() { + let sort_req = LexRequirement::from( + sort_exec + .properties() + .output_ordering() + .cloned() + .unwrap_or(LexOrdering::default()), + ); + if sort_exec + .properties() + .eq_properties + .requirements_compatible(parent_required, &sort_req) + { + debug_assert!(!parent_required.is_empty()); + Ok(Some(vec![Some(LexRequirement::new( + parent_required.to_vec(), + ))])) + } else { + Ok(None) + } + } else if plan.fetch().is_some() + && plan.supports_limit_pushdown() + && plan + .maintains_input_order() + .iter() + .all(|maintain| *maintain) + { + let output_req = LexRequirement::from( + plan.properties() + .output_ordering() + .cloned() + .unwrap_or(LexOrdering::default()), + ); + // Push down through operator with fetch when: + // - requirement is aligned with output ordering + // - it preserves ordering during execution + if plan + .properties() + .eq_properties + .requirements_compatible(parent_required, &output_req) + { + let req = (!parent_required.is_empty()) + .then(|| LexRequirement::new(parent_required.to_vec())); + Ok(Some(vec![req])) + } else { + Ok(None) + } + } else if is_union(plan) { + // UnionExec does not have real sort requirements for its input. Here we change the adjusted_request_ordering to UnionExec's output ordering and + // propagate the sort requirements down to correct the unnecessary descendant SortExec under the UnionExec + let req = (!parent_required.is_empty()).then(|| parent_required.clone()); + Ok(Some(vec![req; plan.children().len()])) + } else if let Some(smj) = plan.as_any().downcast_ref::() { + // If the current plan is SortMergeJoinExec + let left_columns_len = smj.left().schema().fields().len(); + let parent_required_expr = LexOrdering::from(parent_required.clone()); + match expr_source_side( + parent_required_expr.as_ref(), + smj.join_type(), + left_columns_len, + ) { + Some(JoinSide::Left) => try_pushdown_requirements_to_join( + smj, + parent_required, + parent_required_expr.as_ref(), + JoinSide::Left, + ), + Some(JoinSide::Right) => { + let right_offset = + smj.schema().fields.len() - smj.right().schema().fields.len(); + let new_right_required = + shift_right_required(parent_required, right_offset)?; + let new_right_required_expr = LexOrdering::from(new_right_required); + try_pushdown_requirements_to_join( + smj, + parent_required, + new_right_required_expr.as_ref(), + JoinSide::Right, + ) + } + _ => { + // Can not decide the expr side for SortMergeJoinExec, can not push down + Ok(None) + } + } + } else if maintains_input_order.is_empty() + || !maintains_input_order.iter().any(|o| *o) + || plan.as_any().is::() + || plan.as_any().is::() + // TODO: Add support for Projection push down + || plan.as_any().is::() + || pushdown_would_violate_requirements(parent_required, plan.as_ref()) + { + // If the current plan is a leaf node or can not maintain any of the input ordering, can not pushed down requirements. + // For RepartitionExec, we always choose to not push down the sort requirements even the RepartitionExec(input_partition=1) could maintain input ordering. + // Pushing down is not beneficial + Ok(None) + } else if is_sort_preserving_merge(plan) { + let new_ordering = LexOrdering::from(parent_required.clone()); + let mut spm_eqs = plan.equivalence_properties().clone(); + // Sort preserving merge will have new ordering, one requirement above is pushed down to its below. + spm_eqs = spm_eqs.with_reorder(new_ordering); + // Do not push-down through SortPreservingMergeExec when + // ordering requirement invalidates requirement of sort preserving merge exec. + if !spm_eqs.ordering_satisfy(&plan.output_ordering().cloned().unwrap_or_default()) + { + Ok(None) + } else { + // Can push-down through SortPreservingMergeExec, because parent requirement is finer + // than SortPreservingMergeExec output ordering. + let req = (!parent_required.is_empty()) + .then(|| LexRequirement::new(parent_required.to_vec())); + Ok(Some(vec![req])) + } + } else if let Some(hash_join) = plan.as_any().downcast_ref::() { + handle_hash_join(hash_join, parent_required) + } else { + handle_custom_pushdown(plan, parent_required, maintains_input_order) + } + // TODO: Add support for Projection push down +} + +/// Return true if pushing the sort requirements through a node would violate +/// the input sorting requirements for the plan +fn pushdown_would_violate_requirements( + parent_required: &LexRequirement, + child: &dyn ExecutionPlan, +) -> bool { + child + .required_input_ordering() + .iter() + .any(|child_required| { + let Some(child_required) = child_required.as_ref() else { + // no requirements, so pushing down would not violate anything + return false; + }; + // check if the plan's requirements would still e satisfied if we pushed + // down the parent requirements + child_required + .iter() + .zip(parent_required.iter()) + .all(|(c, p)| !c.compatible(p)) + }) +} + +/// Determine children requirements: +/// - If children requirements are more specific, do not push down parent +/// requirements. +/// - If parent requirements are more specific, push down parent requirements. +/// - If they are not compatible, need to add a sort. +fn determine_children_requirement( + parent_required: &LexRequirement, + request_child: &LexRequirement, + child_plan: &Arc, +) -> RequirementsCompatibility { + if child_plan + .equivalence_properties() + .requirements_compatible(request_child, parent_required) + { + // Child requirements are more specific, no need to push down. + RequirementsCompatibility::Satisfy + } else if child_plan + .equivalence_properties() + .requirements_compatible(parent_required, request_child) + { + // Parent requirements are more specific, adjust child's requirements + // and push down the new requirements: + let adjusted = (!parent_required.is_empty()) + .then(|| LexRequirement::new(parent_required.to_vec())); + RequirementsCompatibility::Compatible(adjusted) + } else { + RequirementsCompatibility::NonCompatible + } +} + +fn try_pushdown_requirements_to_join( + smj: &SortMergeJoinExec, + parent_required: &LexRequirement, + sort_expr: &LexOrdering, + push_side: JoinSide, +) -> Result>>> { + let left_eq_properties = smj.left().equivalence_properties(); + let right_eq_properties = smj.right().equivalence_properties(); + let mut smj_required_orderings = smj.required_input_ordering(); + let right_requirement = smj_required_orderings.swap_remove(1); + let left_requirement = smj_required_orderings.swap_remove(0); + let left_ordering = &smj.left().output_ordering().cloned().unwrap_or_default(); + let right_ordering = &smj.right().output_ordering().cloned().unwrap_or_default(); + + let (new_left_ordering, new_right_ordering) = match push_side { + JoinSide::Left => { + let left_eq_properties = + left_eq_properties.clone().with_reorder(sort_expr.clone()); + if left_eq_properties + .ordering_satisfy_requirement(&left_requirement.unwrap_or_default()) + { + // After re-ordering requirement is still satisfied + (sort_expr, right_ordering) + } else { + return Ok(None); + } + } + JoinSide::Right => { + let right_eq_properties = + right_eq_properties.clone().with_reorder(sort_expr.clone()); + if right_eq_properties + .ordering_satisfy_requirement(&right_requirement.unwrap_or_default()) + { + // After re-ordering requirement is still satisfied + (left_ordering, sort_expr) + } else { + return Ok(None); + } + } + JoinSide::None => return Ok(None), + }; + let join_type = smj.join_type(); + let probe_side = SortMergeJoinExec::probe_side(&join_type); + let new_output_ordering = calculate_join_output_ordering( + new_left_ordering, + new_right_ordering, + join_type, + smj.on(), + smj.left().schema().fields.len(), + &smj.maintains_input_order(), + Some(probe_side), + ); + let mut smj_eqs = smj.properties().equivalence_properties().clone(); + // smj will have this ordering when its input changes. + smj_eqs = smj_eqs.with_reorder(new_output_ordering.unwrap_or_default()); + let should_pushdown = smj_eqs.ordering_satisfy_requirement(parent_required); + Ok(should_pushdown.then(|| { + let mut required_input_ordering = smj.required_input_ordering(); + let new_req = Some(LexRequirement::from(sort_expr.clone())); + match push_side { + JoinSide::Left => { + required_input_ordering[0] = new_req; + } + JoinSide::Right => { + required_input_ordering[1] = new_req; + } + JoinSide::None => unreachable!(), + } + required_input_ordering + })) +} + +fn expr_source_side( + required_exprs: &LexOrdering, + join_type: JoinType, + left_columns_len: usize, +) -> Option { + match join_type { + JoinType::Inner + | JoinType::Left + | JoinType::Right + | JoinType::Full + | JoinType::LeftMark => { + let all_column_sides = required_exprs + .iter() + .filter_map(|r| { + r.expr.as_any().downcast_ref::().map(|col| { + if col.index() < left_columns_len { + JoinSide::Left + } else { + JoinSide::Right + } + }) + }) + .collect::>(); + + // If the exprs are all coming from one side, the requirements can be pushed down + if all_column_sides.len() != required_exprs.len() { + None + } else if all_column_sides + .iter() + .all(|side| matches!(side, JoinSide::Left)) + { + Some(JoinSide::Left) + } else if all_column_sides + .iter() + .all(|side| matches!(side, JoinSide::Right)) + { + Some(JoinSide::Right) + } else { + None + } + } + JoinType::LeftSemi | JoinType::LeftAnti => required_exprs + .iter() + .all(|e| e.expr.as_any().downcast_ref::().is_some()) + .then_some(JoinSide::Left), + JoinType::RightSemi | JoinType::RightAnti => required_exprs + .iter() + .all(|e| e.expr.as_any().downcast_ref::().is_some()) + .then_some(JoinSide::Right), + } +} + +fn shift_right_required( + parent_required: &LexRequirement, + left_columns_len: usize, +) -> Result { + let new_right_required = parent_required + .iter() + .filter_map(|r| { + let col = r.expr.as_any().downcast_ref::()?; + col.index().checked_sub(left_columns_len).map(|offset| { + r.clone() + .with_expr(Arc::new(Column::new(col.name(), offset))) + }) + }) + .collect::>(); + if new_right_required.len() == parent_required.len() { + Ok(LexRequirement::new(new_right_required)) + } else { + plan_err!( + "Expect to shift all the parent required column indexes for SortMergeJoin" + ) + } +} + +/// Handles the custom pushdown of parent-required sorting requirements down to +/// the child execution plans, considering whether the input order is maintained. +/// +/// # Arguments +/// +/// * `plan` - A reference to an `ExecutionPlan` for which the pushdown will be applied. +/// * `parent_required` - The sorting requirements expected by the parent node. +/// * `maintains_input_order` - A vector of booleans indicating whether each child +/// maintains the input order. +/// +/// # Returns +/// +/// Returns `Ok(Some(Vec>))` if the sorting requirements can be +/// pushed down, `Ok(None)` if not. On error, returns a `Result::Err`. +fn handle_custom_pushdown( + plan: &Arc, + parent_required: &LexRequirement, + maintains_input_order: Vec, +) -> Result>>> { + // If there's no requirement from the parent or the plan has no children, return early + if parent_required.is_empty() || plan.children().is_empty() { + return Ok(None); + } + + // Collect all unique column indices used in the parent-required sorting expression + let all_indices: HashSet = parent_required + .iter() + .flat_map(|order| { + collect_columns(&order.expr) + .iter() + .map(|col| col.index()) + .collect::>() + }) + .collect(); + + // Get the number of fields in each child's schema + let len_of_child_schemas: Vec = plan + .children() + .iter() + .map(|c| c.schema().fields().len()) + .collect(); + + // Find the index of the child that maintains input order + let Some(maintained_child_idx) = maintains_input_order + .iter() + .enumerate() + .find(|(_, m)| **m) + .map(|pair| pair.0) + else { + return Ok(None); + }; + + // Check if all required columns come from the child that maintains input order + let start_idx = len_of_child_schemas[..maintained_child_idx] + .iter() + .sum::(); + let end_idx = start_idx + len_of_child_schemas[maintained_child_idx]; + let all_from_maintained_child = + all_indices.iter().all(|i| i >= &start_idx && i < &end_idx); + + // If all columns are from the maintained child, update the parent requirements + if all_from_maintained_child { + let sub_offset = len_of_child_schemas + .iter() + .take(maintained_child_idx) + .sum::(); + // Transform the parent-required expression for the child schema by adjusting columns + let updated_parent_req = parent_required + .iter() + .map(|req| { + let child_schema = plan.children()[maintained_child_idx].schema(); + let updated_columns = Arc::clone(&req.expr) + .transform_up(|expr| { + if let Some(col) = expr.as_any().downcast_ref::() { + let new_index = col.index() - sub_offset; + Ok(Transformed::yes(Arc::new(Column::new( + child_schema.field(new_index).name(), + new_index, + )))) + } else { + Ok(Transformed::no(expr)) + } + })? + .data; + Ok(PhysicalSortRequirement::new(updated_columns, req.options)) + }) + .collect::>>()?; + + // Prepare the result, populating with the updated requirements for children that maintain order + let result = maintains_input_order + .iter() + .map(|&maintains_order| { + if maintains_order { + Some(LexRequirement::new(updated_parent_req.clone())) + } else { + None + } + }) + .collect(); + + Ok(Some(result)) + } else { + Ok(None) + } +} + +// For hash join we only maintain the input order for the right child +// for join type: Inner, Right, RightSemi, RightAnti +fn handle_hash_join( + plan: &HashJoinExec, + parent_required: &LexRequirement, +) -> Result>>> { + // If there's no requirement from the parent or the plan has no children + // or the join type is not Inner, Right, RightSemi, RightAnti, return early + if parent_required.is_empty() || !plan.maintains_input_order()[1] { + return Ok(None); + } + + // Collect all unique column indices used in the parent-required sorting expression + let all_indices: HashSet = parent_required + .iter() + .flat_map(|order| { + collect_columns(&order.expr) + .into_iter() + .map(|col| col.index()) + .collect::>() + }) + .collect(); + + let column_indices = build_join_column_index(plan); + let projected_indices: Vec<_> = if let Some(projection) = &plan.projection { + projection.iter().map(|&i| &column_indices[i]).collect() + } else { + column_indices.iter().collect() + }; + let len_of_left_fields = projected_indices + .iter() + .filter(|ci| ci.side == JoinSide::Left) + .count(); + + let all_from_right_child = all_indices.iter().all(|i| *i >= len_of_left_fields); + + // If all columns are from the right child, update the parent requirements + if all_from_right_child { + // Transform the parent-required expression for the child schema by adjusting columns + let updated_parent_req = parent_required + .iter() + .map(|req| { + let child_schema = plan.children()[1].schema(); + let updated_columns = Arc::clone(&req.expr) + .transform_up(|expr| { + if let Some(col) = expr.as_any().downcast_ref::() { + let index = projected_indices[col.index()].index; + Ok(Transformed::yes(Arc::new(Column::new( + child_schema.field(index).name(), + index, + )))) + } else { + Ok(Transformed::no(expr)) + } + })? + .data; + Ok(PhysicalSortRequirement::new(updated_columns, req.options)) + }) + .collect::>>()?; + + // Populating with the updated requirements for children that maintain order + Ok(Some(vec![ + None, + Some(LexRequirement::new(updated_parent_req)), + ])) + } else { + Ok(None) + } +} + +// this function is used to build the column index for the hash join +// push down sort requirements to the right child +fn build_join_column_index(plan: &HashJoinExec) -> Vec { + let map_fields = |schema: SchemaRef, side: JoinSide| { + schema + .fields() + .iter() + .enumerate() + .map(|(index, _)| ColumnIndex { index, side }) + .collect::>() + }; + + match plan.join_type() { + JoinType::Inner | JoinType::Right => { + map_fields(plan.left().schema(), JoinSide::Left) + .into_iter() + .chain(map_fields(plan.right().schema(), JoinSide::Right)) + .collect::>() + } + JoinType::RightSemi | JoinType::RightAnti => { + map_fields(plan.right().schema(), JoinSide::Right) + } + _ => unreachable!("unexpected join type: {}", plan.join_type()), + } +} + +/// Define the Requirements Compatibility +#[derive(Debug)] +enum RequirementsCompatibility { + /// Requirements satisfy + Satisfy, + /// Requirements compatible + Compatible(Option), + /// Requirements not compatible + NonCompatible, +} diff --git a/datafusion/physical-optimizer/src/utils.rs b/datafusion/physical-optimizer/src/utils.rs index fa16b5422310..af03374fa785 100644 --- a/datafusion/physical-optimizer/src/utils.rs +++ b/datafusion/physical-optimizer/src/utils.rs @@ -23,13 +23,13 @@ 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::union::UnionExec; -// use datafusion_physical_plan::windows::{BoundedWindowAggExec, WindowAggExec}; +use datafusion_physical_plan::union::UnionExec; +use datafusion_physical_plan::windows::{BoundedWindowAggExec, WindowAggExec}; use datafusion_physical_plan::{ExecutionPlan, ExecutionPlanProperties}; use datafusion_physical_expr::LexRequirement; use datafusion_physical_expr_common::sort_expr::LexOrdering; -// use datafusion_physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; +use datafusion_physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; use datafusion_physical_plan::tree_node::PlanContext; /// This utility function adds a `SortExec` above an operator according to the @@ -74,20 +74,20 @@ 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::() -// } +pub fn is_limit(plan: &Arc) -> bool { + plan.as_any().is::() || plan.as_any().is::() +} /// Checks whether the given operator is a window; /// i.e. either a [`WindowAggExec`] or a [`BoundedWindowAggExec`]. -// pub fn is_window(plan: &Arc) -> bool { -// plan.as_any().is::() || plan.as_any().is::() -// } +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::() -// } +pub fn is_sort(plan: &Arc) -> bool { + plan.as_any().is::() +} /// Checks whether the given operator is a [`SortPreservingMergeExec`]. pub fn is_sort_preserving_merge(plan: &Arc) -> bool { @@ -100,9 +100,9 @@ pub fn is_coalesce_partitions(plan: &Arc) -> bool { } /// Checks whether the given operator is a [`UnionExec`]. -// pub fn is_union(plan: &Arc) -> bool { -// plan.as_any().is::() -// } +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 { From fef3c6ebcedd9fba037b88b945aef71472c88cf6 Mon Sep 17 00:00:00 2001 From: buraksenn Date: Tue, 21 Jan 2025 11:52:43 +0300 Subject: [PATCH 10/24] import order --- datafusion-testing | 2 +- .../replace_with_order_preserving_variants.rs | 27 ++++++++++--------- parquet-testing | 2 +- testing | 2 +- 4 files changed, 18 insertions(+), 15 deletions(-) diff --git a/datafusion-testing b/datafusion-testing index 36283d195c72..5b424aefd7f6 160000 --- a/datafusion-testing +++ b/datafusion-testing @@ -1 +1 @@ -Subproject commit 36283d195c728f26b16b517ba999fd62509b6649 +Subproject commit 5b424aefd7f6bf198220c37f59d39dbb25b47695 diff --git a/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs b/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs index 86d3879477cd..caced567df4c 100644 --- a/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs +++ b/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs @@ -19,10 +19,16 @@ use std::sync::Arc; use datafusion::prelude::{SessionConfig, SessionContext}; use datafusion_execution::TaskContext; -use datafusion_physical_optimizer::test_utils::check_integrity; + +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, }; @@ -31,24 +37,21 @@ 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 object_store::memory::InMemory; -use object_store::ObjectStore; -use url::Url; +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; -use datafusion_physical_plan::coalesce_batches::CoalesceBatchesExec; -use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; -use datafusion_physical_plan::repartition::RepartitionExec; -use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; + +use object_store::memory::InMemory; +use object_store::ObjectStore; use rstest::rstest; -use datafusion_physical_optimizer::enforce_sorting::replace_with_order_preserving_variants::{replace_with_order_preserving_variants, OrderPreservationContext}; -use datafusion_common::tree_node::{TreeNode, TransformedResult}; +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 diff --git a/parquet-testing b/parquet-testing index e45cd23f784a..f4d7ed772a62 160000 --- a/parquet-testing +++ b/parquet-testing @@ -1 +1 @@ -Subproject commit e45cd23f784aab3d6bf0701f8f4e621469ed3be7 +Subproject commit f4d7ed772a62a95111db50fbcad2460833e8c882 diff --git a/testing b/testing index 98fceecd024d..d2a137123034 160000 --- a/testing +++ b/testing @@ -1 +1 @@ -Subproject commit 98fceecd024dccd2f8a00e32fc144975f218acf4 +Subproject commit d2a13712303498963395318a4eb42872e66aead7 From dcbc0e1dcf0fafb63a34289b7fe8a3694ea697c0 Mon Sep 17 00:00:00 2001 From: logan-keede Date: Tue, 21 Jan 2025 15:17:25 +0530 Subject: [PATCH 11/24] merge fixes + formatting --- datafusion/core/src/physical_optimizer/mod.rs | 3 - .../core/src/physical_optimizer/utils.rs | 110 - .../replace_with_order_preserving_variants.rs | 1 - datafusion/physical-optimizer/Cargo.toml | 2 +- .../physical-optimizer/src/enforce_sorting.rs | 2585 ----------------- .../src/enforce_sorting/mod.rs | 4 +- .../replace_with_order_preserving_variants.rs | 2 +- .../src/enforce_sorting/sort_pushdown.rs | 2 +- datafusion/physical-optimizer/src/lib.rs | 2 - .../replace_with_order_preserving_variants.rs | 1572 ---------- .../physical-optimizer/src/sort_pushdown.rs | 719 ----- 11 files changed, 5 insertions(+), 4997 deletions(-) delete mode 100644 datafusion/core/src/physical_optimizer/utils.rs delete mode 100644 datafusion/physical-optimizer/src/enforce_sorting.rs delete mode 100644 datafusion/physical-optimizer/src/replace_with_order_preserving_variants.rs delete mode 100644 datafusion/physical-optimizer/src/sort_pushdown.rs diff --git a/datafusion/core/src/physical_optimizer/mod.rs b/datafusion/core/src/physical_optimizer/mod.rs index 22a20d7b9375..e6aa15a4c09d 100644 --- a/datafusion/core/src/physical_optimizer/mod.rs +++ b/datafusion/core/src/physical_optimizer/mod.rs @@ -22,10 +22,7 @@ //! //! [`ExecutionPlan`]: crate::physical_plan::ExecutionPlan -pub mod enforce_distribution; pub mod optimizer; pub mod projection_pushdown; -mod utils; - pub use datafusion_physical_optimizer::*; diff --git a/datafusion/core/src/physical_optimizer/utils.rs b/datafusion/core/src/physical_optimizer/utils.rs deleted file mode 100644 index 9f2c28d564f0..000000000000 --- a/datafusion/core/src/physical_optimizer/utils.rs +++ /dev/null @@ -1,110 +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 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_plan::limit::{GlobalLimitExec, LocalLimitExec}; -use datafusion_physical_plan::tree_node::PlanContext; - -/// This utility function adds a `SortExec` above an operator according to the -/// given ordering requirements while preserving the original partitioning. -pub fn add_sort_above( - node: PlanContext, - sort_requirements: LexRequirement, - fetch: Option, -) -> PlanContext { - let mut sort_expr = LexOrdering::from(sort_requirements); - sort_expr.retain(|sort_expr| { - !node - .plan - .equivalence_properties() - .is_expr_constant(&sort_expr.expr) - }); - let mut new_sort = SortExec::new(sort_expr, Arc::clone(&node.plan)).with_fetch(fetch); - if node.plan.output_partitioning().partition_count() > 1 { - new_sort = new_sort.with_preserve_partitioning(true); - } - PlanContext::new(Arc::new(new_sort), T::default(), vec![node]) -} - -/// This utility function adds a `SortExec` above an operator according to the -/// given ordering requirements while preserving the original partitioning. If -/// requirement is already satisfied no `SortExec` is added. -pub fn add_sort_above_with_check( - node: PlanContext, - sort_requirements: LexRequirement, - fetch: Option, -) -> PlanContext { - if !node - .plan - .equivalence_properties() - .ordering_satisfy_requirement(&sort_requirements) - { - add_sort_above(node, sort_requirements, fetch) - } else { - node - } -} - -/// 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 window; -/// i.e. either a [`WindowAggExec`] or a [`BoundedWindowAggExec`]. -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 [`SortPreservingMergeExec`]. -pub fn is_sort_preserving_merge(plan: &Arc) -> bool { - plan.as_any().is::() -} - -/// Checks whether the given operator is a [`CoalescePartitionsExec`]. -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::() -} diff --git a/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs b/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs index 0c33ec331532..86d3879477cd 100644 --- a/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs +++ b/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs @@ -15,7 +15,6 @@ // specific language governing permissions and limitations // under the License. - use std::sync::Arc; use datafusion::prelude::{SessionConfig, SessionContext}; diff --git a/datafusion/physical-optimizer/Cargo.toml b/datafusion/physical-optimizer/Cargo.toml index af687daabfad..2f6c89972bd3 100644 --- a/datafusion/physical-optimizer/Cargo.toml +++ b/datafusion/physical-optimizer/Cargo.toml @@ -33,7 +33,7 @@ workspace = true [features] recursive_protection = ["dep:recursive"] -parquet = ["dep:parquet"] + [dependencies] arrow = { workspace = true } diff --git a/datafusion/physical-optimizer/src/enforce_sorting.rs b/datafusion/physical-optimizer/src/enforce_sorting.rs deleted file mode 100644 index 88ca34eb5bb3..000000000000 --- a/datafusion/physical-optimizer/src/enforce_sorting.rs +++ /dev/null @@ -1,2585 +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::optimizer::PhysicalOptimizerRule; -use datafusion_common::config::ConfigOptions; -use datafusion_common::error::Result; -use crate::replace_with_order_preserving_variants::{ - replace_with_order_preserving_variants, OrderPreservationContext, -}; -use crate::sort_pushdown::{ - assign_initial_requirements, pushdown_sorts, SortPushDown, -}; -use crate::utils::{ - is_coalesce_partitions, is_limit, is_repartition, is_sort, is_sort_preserving_merge, - is_union, is_window, -}; -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 datafusion_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_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))) 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 plan = if let Some(ordering) = plan.output_ordering() { - Arc::new(SortPreservingMergeExec::new( - LexOrdering::new(ordering.to_vec()), - plan, - )) 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/physical-optimizer/src/enforce_sorting/mod.rs b/datafusion/physical-optimizer/src/enforce_sorting/mod.rs index f84911b3bb4b..8a31c5834fd5 100644 --- a/datafusion/physical-optimizer/src/enforce_sorting/mod.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/mod.rs @@ -37,7 +37,7 @@ pub mod replace_with_order_preserving_variants; pub mod sort_pushdown; -mod utils; +// mod utils; use std::sync::Arc; @@ -57,7 +57,7 @@ use crate::enforce_sorting::replace_with_order_preserving_variants::{ use crate::enforce_sorting::sort_pushdown::{ assign_initial_requirements, pushdown_sorts, SortPushDown, }; -use crate::enforce_sorting::utils::{ +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, }; diff --git a/datafusion/physical-optimizer/src/enforce_sorting/replace_with_order_preserving_variants.rs b/datafusion/physical-optimizer/src/enforce_sorting/replace_with_order_preserving_variants.rs index f6d35086f5df..486f2d0343bb 100644 --- a/datafusion/physical-optimizer/src/enforce_sorting/replace_with_order_preserving_variants.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/replace_with_order_preserving_variants.rs @@ -21,7 +21,7 @@ use std::sync::Arc; -use super::utils::{ +use crate::utils::{ is_coalesce_partitions, is_repartition, is_sort, is_sort_preserving_merge, }; diff --git a/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs b/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs index 8cb29f2688f5..537a1cdb024c 100644 --- a/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs @@ -20,7 +20,7 @@ use std::sync::Arc; use arrow_schema::SchemaRef; -use crate::enforce_sorting::utils::{ +use crate::utils::{ add_sort_above, is_sort, is_sort_preserving_merge, is_union, is_window, }; use datafusion_common::tree_node::{ diff --git a/datafusion/physical-optimizer/src/lib.rs b/datafusion/physical-optimizer/src/lib.rs index 6fb6602cbbcd..4fb26b950419 100644 --- a/datafusion/physical-optimizer/src/lib.rs +++ b/datafusion/physical-optimizer/src/lib.rs @@ -23,7 +23,6 @@ pub mod coalesce_batches; pub mod combine_partial_final_agg; pub mod enforce_distribution; pub mod enforce_sorting; -pub mod replace_with_order_preserving_variants; pub mod join_selection; pub mod limit_pushdown; pub mod limited_distinct_aggregation; @@ -36,4 +35,3 @@ pub mod topk_aggregation; pub mod update_aggr_exprs; pub use optimizer::PhysicalOptimizerRule; mod utils; -mod sort_pushdown; diff --git a/datafusion/physical-optimizer/src/replace_with_order_preserving_variants.rs b/datafusion/physical-optimizer/src/replace_with_order_preserving_variants.rs deleted file mode 100644 index c3be87d5cc8c..000000000000 --- a/datafusion/physical-optimizer/src/replace_with_order_preserving_variants.rs +++ /dev/null @@ -1,1572 +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 datafusion_common::error::Result; -use crate::utils::{is_coalesce_partitions, is_sort}; -use datafusion_physical_plan::repartition::RepartitionExec; -use datafusion_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, -) -> 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) - } 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)); - 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, - )?; - - // 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/physical-optimizer/src/sort_pushdown.rs b/datafusion/physical-optimizer/src/sort_pushdown.rs deleted file mode 100644 index a2a0bf7ad309..000000000000 --- a/datafusion/physical-optimizer/src/sort_pushdown.rs +++ /dev/null @@ -1,719 +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. - -use std::fmt::Debug; -use std::sync::Arc; - -use super::utils::{add_sort_above, is_sort}; -use crate::utils::{is_sort_preserving_merge, is_union, is_window}; -use datafusion_physical_plan::filter::FilterExec; -use datafusion_physical_plan::joins::utils::calculate_join_output_ordering; -use datafusion_physical_plan::joins::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}; -use arrow_schema::SchemaRef; - -use datafusion_common::tree_node::{ - ConcreteTreeNode, Transformed, TreeNode, TreeNodeRecursion, -}; -use datafusion_common::{plan_err, HashSet, JoinSide, Result}; -use datafusion_expr::JoinType; -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; - -/// 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 -/// computational cost by pushing down `SortExec`s through some executors. The -/// 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 -#[derive(Default, Clone)] -pub struct ParentRequirements { - ordering_requirement: Option, - fetch: Option, -} - -pub type SortPushDown = PlanContext; - -/// Assigns the ordering requirement of the root node to the its children. -pub fn assign_initial_requirements(node: &mut SortPushDown) { - let reqs = node.plan.required_input_ordering(); - for (child, requirement) in node.children.iter_mut().zip(reqs) { - child.data = ParentRequirements { - ordering_requirement: requirement, - fetch: None, - }; - } -} - -pub(crate) 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)?; - } - let (new_node, children) = new_node.data.take_children(); - let new_children = children - .into_iter() - .map(pushdown_sorts) - .collect::>()?; - new_node.with_new_children(new_children) -} - -fn pushdown_sorts_helper( - mut requirements: SortPushDown, -) -> Result> { - let plan = &requirements.plan; - let parent_reqs = requirements - .data - .ordering_requirement - .clone() - .unwrap_or_default(); - let satisfy_parent = plan - .equivalence_properties() - .ordering_satisfy_requirement(&parent_reqs); - - if is_sort(plan) { - let required_ordering = plan - .output_ordering() - .cloned() - .map(LexRequirement::from) - .unwrap_or_default(); - if !satisfy_parent { - // Make sure this `SortExec` satisfies parent requirements: - let sort_reqs = requirements.data.ordering_requirement.unwrap_or_default(); - let fetch = requirements.data.fetch; - requirements = requirements.children.swap_remove(0); - requirements = add_sort_above(requirements, sort_reqs, fetch); - }; - - // We can safely get the 0th index as we are dealing with a `SortExec`. - let mut child = requirements.children.swap_remove(0); - if let Some(adjusted) = - pushdown_requirement_to_children(&child.plan, &required_ordering)? - { - let fetch = child.plan.fetch(); - for (grand_child, order) in child.children.iter_mut().zip(adjusted) { - grand_child.data = ParentRequirements { - ordering_requirement: order, - fetch, - }; - } - // Can push down requirements - child.data = ParentRequirements { - ordering_requirement: Some(required_ordering), - fetch, - }; - - return Ok(Transformed { - data: child, - transformed: true, - tnr: TreeNodeRecursion::Stop, - }); - } else { - // Can not push down requirements - requirements.children = vec![child]; - assign_initial_requirements(&mut requirements); - } - } else if satisfy_parent { - // For non-sort operators, immediately return if parent requirements are met: - let reqs = plan.required_input_ordering(); - for (child, order) in requirements.children.iter_mut().zip(reqs) { - child.data.ordering_requirement = order; - } - } else if let Some(adjusted) = pushdown_requirement_to_children(plan, &parent_reqs)? { - // Can not satisfy the parent requirements, check whether we can push - // requirements down: - for (child, order) in requirements.children.iter_mut().zip(adjusted) { - child.data.ordering_requirement = order; - } - requirements.data.ordering_requirement = None; - } else { - // Can not push down requirements, add new `SortExec`: - let sort_reqs = requirements - .data - .ordering_requirement - .clone() - .unwrap_or_default(); - let fetch = requirements.data.fetch; - requirements = add_sort_above(requirements, sort_reqs, fetch); - assign_initial_requirements(&mut requirements); - } - Ok(Transformed::yes(requirements)) -} - -fn pushdown_requirement_to_children( - plan: &Arc, - parent_required: &LexRequirement, -) -> Result>>> { - let maintains_input_order = plan.maintains_input_order(); - if is_window(plan) { - let required_input_ordering = plan.required_input_ordering(); - let request_child = required_input_ordering[0].clone().unwrap_or_default(); - let child_plan = plan.children().swap_remove(0); - - match determine_children_requirement(parent_required, &request_child, child_plan) - { - RequirementsCompatibility::Satisfy => { - let req = (!request_child.is_empty()) - .then(|| LexRequirement::new(request_child.to_vec())); - Ok(Some(vec![req])) - } - RequirementsCompatibility::Compatible(adjusted) => Ok(Some(vec![adjusted])), - RequirementsCompatibility::NonCompatible => Ok(None), - } - } else if let Some(sort_exec) = plan.as_any().downcast_ref::() { - let sort_req = LexRequirement::from( - sort_exec - .properties() - .output_ordering() - .cloned() - .unwrap_or(LexOrdering::default()), - ); - if sort_exec - .properties() - .eq_properties - .requirements_compatible(parent_required, &sort_req) - { - debug_assert!(!parent_required.is_empty()); - Ok(Some(vec![Some(LexRequirement::new( - parent_required.to_vec(), - ))])) - } else { - Ok(None) - } - } else if plan.fetch().is_some() - && plan.supports_limit_pushdown() - && plan - .maintains_input_order() - .iter() - .all(|maintain| *maintain) - { - let output_req = LexRequirement::from( - plan.properties() - .output_ordering() - .cloned() - .unwrap_or(LexOrdering::default()), - ); - // Push down through operator with fetch when: - // - requirement is aligned with output ordering - // - it preserves ordering during execution - if plan - .properties() - .eq_properties - .requirements_compatible(parent_required, &output_req) - { - let req = (!parent_required.is_empty()) - .then(|| LexRequirement::new(parent_required.to_vec())); - Ok(Some(vec![req])) - } else { - Ok(None) - } - } else if is_union(plan) { - // UnionExec does not have real sort requirements for its input. Here we change the adjusted_request_ordering to UnionExec's output ordering and - // propagate the sort requirements down to correct the unnecessary descendant SortExec under the UnionExec - let req = (!parent_required.is_empty()).then(|| parent_required.clone()); - Ok(Some(vec![req; plan.children().len()])) - } else if let Some(smj) = plan.as_any().downcast_ref::() { - // If the current plan is SortMergeJoinExec - let left_columns_len = smj.left().schema().fields().len(); - let parent_required_expr = LexOrdering::from(parent_required.clone()); - match expr_source_side( - parent_required_expr.as_ref(), - smj.join_type(), - left_columns_len, - ) { - Some(JoinSide::Left) => try_pushdown_requirements_to_join( - smj, - parent_required, - parent_required_expr.as_ref(), - JoinSide::Left, - ), - Some(JoinSide::Right) => { - let right_offset = - smj.schema().fields.len() - smj.right().schema().fields.len(); - let new_right_required = - shift_right_required(parent_required, right_offset)?; - let new_right_required_expr = LexOrdering::from(new_right_required); - try_pushdown_requirements_to_join( - smj, - parent_required, - new_right_required_expr.as_ref(), - JoinSide::Right, - ) - } - _ => { - // Can not decide the expr side for SortMergeJoinExec, can not push down - Ok(None) - } - } - } else if maintains_input_order.is_empty() - || !maintains_input_order.iter().any(|o| *o) - || plan.as_any().is::() - || plan.as_any().is::() - // TODO: Add support for Projection push down - || plan.as_any().is::() - || pushdown_would_violate_requirements(parent_required, plan.as_ref()) - { - // If the current plan is a leaf node or can not maintain any of the input ordering, can not pushed down requirements. - // For RepartitionExec, we always choose to not push down the sort requirements even the RepartitionExec(input_partition=1) could maintain input ordering. - // Pushing down is not beneficial - Ok(None) - } else if is_sort_preserving_merge(plan) { - let new_ordering = LexOrdering::from(parent_required.clone()); - let mut spm_eqs = plan.equivalence_properties().clone(); - // Sort preserving merge will have new ordering, one requirement above is pushed down to its below. - spm_eqs = spm_eqs.with_reorder(new_ordering); - // Do not push-down through SortPreservingMergeExec when - // ordering requirement invalidates requirement of sort preserving merge exec. - if !spm_eqs.ordering_satisfy(&plan.output_ordering().cloned().unwrap_or_default()) - { - Ok(None) - } else { - // Can push-down through SortPreservingMergeExec, because parent requirement is finer - // than SortPreservingMergeExec output ordering. - let req = (!parent_required.is_empty()) - .then(|| LexRequirement::new(parent_required.to_vec())); - Ok(Some(vec![req])) - } - } else if let Some(hash_join) = plan.as_any().downcast_ref::() { - handle_hash_join(hash_join, parent_required) - } else { - handle_custom_pushdown(plan, parent_required, maintains_input_order) - } - // TODO: Add support for Projection push down -} - -/// Return true if pushing the sort requirements through a node would violate -/// the input sorting requirements for the plan -fn pushdown_would_violate_requirements( - parent_required: &LexRequirement, - child: &dyn ExecutionPlan, -) -> bool { - child - .required_input_ordering() - .iter() - .any(|child_required| { - let Some(child_required) = child_required.as_ref() else { - // no requirements, so pushing down would not violate anything - return false; - }; - // check if the plan's requirements would still e satisfied if we pushed - // down the parent requirements - child_required - .iter() - .zip(parent_required.iter()) - .all(|(c, p)| !c.compatible(p)) - }) -} - -/// Determine children requirements: -/// - If children requirements are more specific, do not push down parent -/// requirements. -/// - If parent requirements are more specific, push down parent requirements. -/// - If they are not compatible, need to add a sort. -fn determine_children_requirement( - parent_required: &LexRequirement, - request_child: &LexRequirement, - child_plan: &Arc, -) -> RequirementsCompatibility { - if child_plan - .equivalence_properties() - .requirements_compatible(request_child, parent_required) - { - // Child requirements are more specific, no need to push down. - RequirementsCompatibility::Satisfy - } else if child_plan - .equivalence_properties() - .requirements_compatible(parent_required, request_child) - { - // Parent requirements are more specific, adjust child's requirements - // and push down the new requirements: - let adjusted = (!parent_required.is_empty()) - .then(|| LexRequirement::new(parent_required.to_vec())); - RequirementsCompatibility::Compatible(adjusted) - } else { - RequirementsCompatibility::NonCompatible - } -} - -fn try_pushdown_requirements_to_join( - smj: &SortMergeJoinExec, - parent_required: &LexRequirement, - sort_expr: &LexOrdering, - push_side: JoinSide, -) -> Result>>> { - let left_eq_properties = smj.left().equivalence_properties(); - let right_eq_properties = smj.right().equivalence_properties(); - let mut smj_required_orderings = smj.required_input_ordering(); - let right_requirement = smj_required_orderings.swap_remove(1); - let left_requirement = smj_required_orderings.swap_remove(0); - let left_ordering = &smj.left().output_ordering().cloned().unwrap_or_default(); - let right_ordering = &smj.right().output_ordering().cloned().unwrap_or_default(); - - let (new_left_ordering, new_right_ordering) = match push_side { - JoinSide::Left => { - let left_eq_properties = - left_eq_properties.clone().with_reorder(sort_expr.clone()); - if left_eq_properties - .ordering_satisfy_requirement(&left_requirement.unwrap_or_default()) - { - // After re-ordering requirement is still satisfied - (sort_expr, right_ordering) - } else { - return Ok(None); - } - } - JoinSide::Right => { - let right_eq_properties = - right_eq_properties.clone().with_reorder(sort_expr.clone()); - if right_eq_properties - .ordering_satisfy_requirement(&right_requirement.unwrap_or_default()) - { - // After re-ordering requirement is still satisfied - (left_ordering, sort_expr) - } else { - return Ok(None); - } - } - JoinSide::None => return Ok(None), - }; - let join_type = smj.join_type(); - let probe_side = SortMergeJoinExec::probe_side(&join_type); - let new_output_ordering = calculate_join_output_ordering( - new_left_ordering, - new_right_ordering, - join_type, - smj.on(), - smj.left().schema().fields.len(), - &smj.maintains_input_order(), - Some(probe_side), - ); - let mut smj_eqs = smj.properties().equivalence_properties().clone(); - // smj will have this ordering when its input changes. - smj_eqs = smj_eqs.with_reorder(new_output_ordering.unwrap_or_default()); - let should_pushdown = smj_eqs.ordering_satisfy_requirement(parent_required); - Ok(should_pushdown.then(|| { - let mut required_input_ordering = smj.required_input_ordering(); - let new_req = Some(LexRequirement::from(sort_expr.clone())); - match push_side { - JoinSide::Left => { - required_input_ordering[0] = new_req; - } - JoinSide::Right => { - required_input_ordering[1] = new_req; - } - JoinSide::None => unreachable!(), - } - required_input_ordering - })) -} - -fn expr_source_side( - required_exprs: &LexOrdering, - join_type: JoinType, - left_columns_len: usize, -) -> Option { - match join_type { - JoinType::Inner - | JoinType::Left - | JoinType::Right - | JoinType::Full - | JoinType::LeftMark => { - let all_column_sides = required_exprs - .iter() - .filter_map(|r| { - r.expr.as_any().downcast_ref::().map(|col| { - if col.index() < left_columns_len { - JoinSide::Left - } else { - JoinSide::Right - } - }) - }) - .collect::>(); - - // If the exprs are all coming from one side, the requirements can be pushed down - if all_column_sides.len() != required_exprs.len() { - None - } else if all_column_sides - .iter() - .all(|side| matches!(side, JoinSide::Left)) - { - Some(JoinSide::Left) - } else if all_column_sides - .iter() - .all(|side| matches!(side, JoinSide::Right)) - { - Some(JoinSide::Right) - } else { - None - } - } - JoinType::LeftSemi | JoinType::LeftAnti => required_exprs - .iter() - .all(|e| e.expr.as_any().downcast_ref::().is_some()) - .then_some(JoinSide::Left), - JoinType::RightSemi | JoinType::RightAnti => required_exprs - .iter() - .all(|e| e.expr.as_any().downcast_ref::().is_some()) - .then_some(JoinSide::Right), - } -} - -fn shift_right_required( - parent_required: &LexRequirement, - left_columns_len: usize, -) -> Result { - let new_right_required = parent_required - .iter() - .filter_map(|r| { - let col = r.expr.as_any().downcast_ref::()?; - col.index().checked_sub(left_columns_len).map(|offset| { - r.clone() - .with_expr(Arc::new(Column::new(col.name(), offset))) - }) - }) - .collect::>(); - if new_right_required.len() == parent_required.len() { - Ok(LexRequirement::new(new_right_required)) - } else { - plan_err!( - "Expect to shift all the parent required column indexes for SortMergeJoin" - ) - } -} - -/// Handles the custom pushdown of parent-required sorting requirements down to -/// the child execution plans, considering whether the input order is maintained. -/// -/// # Arguments -/// -/// * `plan` - A reference to an `ExecutionPlan` for which the pushdown will be applied. -/// * `parent_required` - The sorting requirements expected by the parent node. -/// * `maintains_input_order` - A vector of booleans indicating whether each child -/// maintains the input order. -/// -/// # Returns -/// -/// Returns `Ok(Some(Vec>))` if the sorting requirements can be -/// pushed down, `Ok(None)` if not. On error, returns a `Result::Err`. -fn handle_custom_pushdown( - plan: &Arc, - parent_required: &LexRequirement, - maintains_input_order: Vec, -) -> Result>>> { - // If there's no requirement from the parent or the plan has no children, return early - if parent_required.is_empty() || plan.children().is_empty() { - return Ok(None); - } - - // Collect all unique column indices used in the parent-required sorting expression - let all_indices: HashSet = parent_required - .iter() - .flat_map(|order| { - collect_columns(&order.expr) - .iter() - .map(|col| col.index()) - .collect::>() - }) - .collect(); - - // Get the number of fields in each child's schema - let len_of_child_schemas: Vec = plan - .children() - .iter() - .map(|c| c.schema().fields().len()) - .collect(); - - // Find the index of the child that maintains input order - let Some(maintained_child_idx) = maintains_input_order - .iter() - .enumerate() - .find(|(_, m)| **m) - .map(|pair| pair.0) - else { - return Ok(None); - }; - - // Check if all required columns come from the child that maintains input order - let start_idx = len_of_child_schemas[..maintained_child_idx] - .iter() - .sum::(); - let end_idx = start_idx + len_of_child_schemas[maintained_child_idx]; - let all_from_maintained_child = - all_indices.iter().all(|i| i >= &start_idx && i < &end_idx); - - // If all columns are from the maintained child, update the parent requirements - if all_from_maintained_child { - let sub_offset = len_of_child_schemas - .iter() - .take(maintained_child_idx) - .sum::(); - // Transform the parent-required expression for the child schema by adjusting columns - let updated_parent_req = parent_required - .iter() - .map(|req| { - let child_schema = plan.children()[maintained_child_idx].schema(); - let updated_columns = Arc::clone(&req.expr) - .transform_up(|expr| { - if let Some(col) = expr.as_any().downcast_ref::() { - let new_index = col.index() - sub_offset; - Ok(Transformed::yes(Arc::new(Column::new( - child_schema.field(new_index).name(), - new_index, - )))) - } else { - Ok(Transformed::no(expr)) - } - })? - .data; - Ok(PhysicalSortRequirement::new(updated_columns, req.options)) - }) - .collect::>>()?; - - // Prepare the result, populating with the updated requirements for children that maintain order - let result = maintains_input_order - .iter() - .map(|&maintains_order| { - if maintains_order { - Some(LexRequirement::new(updated_parent_req.clone())) - } else { - None - } - }) - .collect(); - - Ok(Some(result)) - } else { - Ok(None) - } -} - -// For hash join we only maintain the input order for the right child -// for join type: Inner, Right, RightSemi, RightAnti -fn handle_hash_join( - plan: &HashJoinExec, - parent_required: &LexRequirement, -) -> Result>>> { - // If there's no requirement from the parent or the plan has no children - // or the join type is not Inner, Right, RightSemi, RightAnti, return early - if parent_required.is_empty() || !plan.maintains_input_order()[1] { - return Ok(None); - } - - // Collect all unique column indices used in the parent-required sorting expression - let all_indices: HashSet = parent_required - .iter() - .flat_map(|order| { - collect_columns(&order.expr) - .into_iter() - .map(|col| col.index()) - .collect::>() - }) - .collect(); - - let column_indices = build_join_column_index(plan); - let projected_indices: Vec<_> = if let Some(projection) = &plan.projection { - projection.iter().map(|&i| &column_indices[i]).collect() - } else { - column_indices.iter().collect() - }; - let len_of_left_fields = projected_indices - .iter() - .filter(|ci| ci.side == JoinSide::Left) - .count(); - - let all_from_right_child = all_indices.iter().all(|i| *i >= len_of_left_fields); - - // If all columns are from the right child, update the parent requirements - if all_from_right_child { - // Transform the parent-required expression for the child schema by adjusting columns - let updated_parent_req = parent_required - .iter() - .map(|req| { - let child_schema = plan.children()[1].schema(); - let updated_columns = Arc::clone(&req.expr) - .transform_up(|expr| { - if let Some(col) = expr.as_any().downcast_ref::() { - let index = projected_indices[col.index()].index; - Ok(Transformed::yes(Arc::new(Column::new( - child_schema.field(index).name(), - index, - )))) - } else { - Ok(Transformed::no(expr)) - } - })? - .data; - Ok(PhysicalSortRequirement::new(updated_columns, req.options)) - }) - .collect::>>()?; - - // Populating with the updated requirements for children that maintain order - Ok(Some(vec![ - None, - Some(LexRequirement::new(updated_parent_req)), - ])) - } else { - Ok(None) - } -} - -// this function is used to build the column index for the hash join -// push down sort requirements to the right child -fn build_join_column_index(plan: &HashJoinExec) -> Vec { - let map_fields = |schema: SchemaRef, side: JoinSide| { - schema - .fields() - .iter() - .enumerate() - .map(|(index, _)| ColumnIndex { index, side }) - .collect::>() - }; - - match plan.join_type() { - JoinType::Inner | JoinType::Right => { - map_fields(plan.left().schema(), JoinSide::Left) - .into_iter() - .chain(map_fields(plan.right().schema(), JoinSide::Right)) - .collect::>() - } - JoinType::RightSemi | JoinType::RightAnti => { - map_fields(plan.right().schema(), JoinSide::Right) - } - _ => unreachable!("unexpected join type: {}", plan.join_type()), - } -} - -/// Define the Requirements Compatibility -#[derive(Debug)] -enum RequirementsCompatibility { - /// Requirements satisfy - Satisfy, - /// Requirements compatible - Compatible(Option), - /// Requirements not compatible - NonCompatible, -} From 1f116e4a2862d7d84485c383cdd7cee190d85541 Mon Sep 17 00:00:00 2001 From: logan-keede Date: Tue, 21 Jan 2025 17:32:16 +0530 Subject: [PATCH 12/24] fix: forgotten license --- .../physical_optimizer/enforce_distribution.rs | 17 +++++++++++++++++ datafusion/core/tests/physical_optimizer/mod.rs | 1 + 2 files changed, 18 insertions(+) diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index 9dca54fe30b7..029ec3b8dac7 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -1,3 +1,20 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + use std::ops::Deref; use super::*; diff --git a/datafusion/core/tests/physical_optimizer/mod.rs b/datafusion/core/tests/physical_optimizer/mod.rs index b1f755fde4d1..3702c62ecf69 100644 --- a/datafusion/core/tests/physical_optimizer/mod.rs +++ b/datafusion/core/tests/physical_optimizer/mod.rs @@ -17,6 +17,7 @@ mod combine_partial_final_agg; mod enforce_sorting; +mod enforce_distribution; mod limited_distinct_aggregation; mod replace_with_order_preserving_variants; mod sanity_checker; From 5093e61c33e2a593afc04a006a9de7848d063364 Mon Sep 17 00:00:00 2001 From: logan-keede Date: Tue, 21 Jan 2025 17:34:46 +0530 Subject: [PATCH 13/24] fix: cargo fmt --- .../enforce_distribution.rs | 158 ++++++------------ .../core/tests/physical_optimizer/mod.rs | 2 +- 2 files changed, 55 insertions(+), 105 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index 029ec3b8dac7..5423af384a3b 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -23,6 +23,10 @@ use datafusion::datasource::listing::PartitionedFile; use datafusion::datasource::object_store::ObjectStoreUrl; use datafusion::datasource::physical_plan::{CsvExec, FileScanConfig, ParquetExec}; use datafusion_physical_optimizer::enforce_sorting::EnforceSorting; +use datafusion_physical_optimizer::output_requirements::OutputRequirements; +use datafusion_physical_optimizer::test_utils::{ + check_integrity, coalesce_partitions_exec, repartition_exec, +}; use datafusion_physical_plan::coalesce_batches::CoalesceBatchesExec; use datafusion_physical_plan::expressions::col; use datafusion_physical_plan::filter::FilterExec; @@ -30,10 +34,6 @@ use datafusion_physical_plan::joins::utils::JoinOn; use datafusion_physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; use datafusion_physical_plan::sorts::sort::SortExec; use datafusion_physical_plan::{displayable, DisplayAs, DisplayFormatType, Statistics}; -use datafusion_physical_optimizer::output_requirements::OutputRequirements; -use datafusion_physical_optimizer::test_utils::{ - check_integrity, coalesce_partitions_exec, repartition_exec, -}; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use datafusion_common::ScalarValue; @@ -174,9 +174,7 @@ fn parquet_exec_multiple() -> Arc { } /// Created a sorted parquet exec with multiple files -fn parquet_exec_multiple_sorted( - output_ordering: Vec, -) -> Arc { +fn parquet_exec_multiple_sorted(output_ordering: Vec) -> Arc { ParquetExec::builder( FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema()) .with_file_groups(vec![ @@ -254,8 +252,7 @@ fn aggregate_exec_with_alias( let schema = schema(); let mut group_by_expr: Vec<(Arc, String)> = vec![]; for (column, alias) in alias_pairs.iter() { - group_by_expr - .push((col(column, &input.schema()).unwrap(), alias.to_string())); + group_by_expr.push((col(column, &input.schema()).unwrap(), alias.to_string())); } let group_by = PhysicalGroupBy::new_single(group_by_expr.clone()); @@ -604,8 +601,7 @@ fn multi_hash_joins() -> Result<()> { | JoinType::LeftMark => { // Join on (a == c) let top_join_on = vec![( - Arc::new(Column::new_with_schema("a", &join.schema()).unwrap()) - as _, + Arc::new(Column::new_with_schema("a", &join.schema()).unwrap()) as _, Arc::new(Column::new_with_schema("c", &schema()).unwrap()) as _, )]; let top_join = hash_join_exec( @@ -666,8 +662,7 @@ fn multi_hash_joins() -> Result<()> { // This time we use (b1 == c) for top join // Join on (b1 == c) let top_join_on = vec![( - Arc::new(Column::new_with_schema("b1", &join.schema()).unwrap()) - as _, + Arc::new(Column::new_with_schema("b1", &join.schema()).unwrap()) as _, Arc::new(Column::new_with_schema("c", &schema()).unwrap()) as _, )]; @@ -818,13 +813,11 @@ fn multi_joins_after_multi_alias() -> Result<()> { let join = hash_join_exec(left, right.clone(), &join_on, &JoinType::Inner); // Projection(c as c1) - let alias_pairs: Vec<(String, String)> = - vec![("c".to_string(), "c1".to_string())]; + let alias_pairs: Vec<(String, String)> = vec![("c".to_string(), "c1".to_string())]; let projection = projection_exec_with_alias(join, alias_pairs); // Projection(c1 as a) - let alias_pairs: Vec<(String, String)> = - vec![("c1".to_string(), "a".to_string())]; + let alias_pairs: Vec<(String, String)> = vec![("c1".to_string(), "a".to_string())]; let projection2 = projection_exec_with_alias(projection, alias_pairs); // Join on (a == c) @@ -1013,30 +1006,24 @@ fn multi_hash_join_key_ordering() -> Result<()> { let top_join_on = vec![ ( Arc::new( - Column::new_with_schema("B", &bottom_left_projection.schema()) - .unwrap(), - ) as _, - Arc::new( - Column::new_with_schema("b1", &bottom_right_join.schema()).unwrap(), + Column::new_with_schema("B", &bottom_left_projection.schema()).unwrap(), ) as _, + Arc::new(Column::new_with_schema("b1", &bottom_right_join.schema()).unwrap()) + as _, ), ( Arc::new( - Column::new_with_schema("C", &bottom_left_projection.schema()) - .unwrap(), - ) as _, - Arc::new( - Column::new_with_schema("c", &bottom_right_join.schema()).unwrap(), + Column::new_with_schema("C", &bottom_left_projection.schema()).unwrap(), ) as _, + Arc::new(Column::new_with_schema("c", &bottom_right_join.schema()).unwrap()) + as _, ), ( Arc::new( - Column::new_with_schema("AA", &bottom_left_projection.schema()) - .unwrap(), - ) as _, - Arc::new( - Column::new_with_schema("a1", &bottom_right_join.schema()).unwrap(), + Column::new_with_schema("AA", &bottom_left_projection.schema()).unwrap(), ) as _, + Arc::new(Column::new_with_schema("a1", &bottom_right_join.schema()).unwrap()) + as _, ), ]; @@ -1152,30 +1139,24 @@ fn reorder_join_keys_to_left_input() -> Result<()> { let top_join_on = vec![ ( Arc::new( - Column::new_with_schema("B", &bottom_left_projection.schema()) - .unwrap(), - ) as _, - Arc::new( - Column::new_with_schema("b1", &bottom_right_join.schema()).unwrap(), + Column::new_with_schema("B", &bottom_left_projection.schema()).unwrap(), ) as _, + Arc::new(Column::new_with_schema("b1", &bottom_right_join.schema()).unwrap()) + as _, ), ( Arc::new( - Column::new_with_schema("C", &bottom_left_projection.schema()) - .unwrap(), - ) as _, - Arc::new( - Column::new_with_schema("c", &bottom_right_join.schema()).unwrap(), + Column::new_with_schema("C", &bottom_left_projection.schema()).unwrap(), ) as _, + Arc::new(Column::new_with_schema("c", &bottom_right_join.schema()).unwrap()) + as _, ), ( Arc::new( - Column::new_with_schema("AA", &bottom_left_projection.schema()) - .unwrap(), - ) as _, - Arc::new( - Column::new_with_schema("a1", &bottom_right_join.schema()).unwrap(), + Column::new_with_schema("AA", &bottom_left_projection.schema()).unwrap(), ) as _, + Arc::new(Column::new_with_schema("a1", &bottom_right_join.schema()).unwrap()) + as _, ), ]; @@ -1292,30 +1273,24 @@ fn reorder_join_keys_to_right_input() -> Result<()> { let top_join_on = vec![ ( Arc::new( - Column::new_with_schema("B", &bottom_left_projection.schema()) - .unwrap(), - ) as _, - Arc::new( - Column::new_with_schema("b1", &bottom_right_join.schema()).unwrap(), + Column::new_with_schema("B", &bottom_left_projection.schema()).unwrap(), ) as _, + Arc::new(Column::new_with_schema("b1", &bottom_right_join.schema()).unwrap()) + as _, ), ( Arc::new( - Column::new_with_schema("C", &bottom_left_projection.schema()) - .unwrap(), - ) as _, - Arc::new( - Column::new_with_schema("c", &bottom_right_join.schema()).unwrap(), + Column::new_with_schema("C", &bottom_left_projection.schema()).unwrap(), ) as _, + Arc::new(Column::new_with_schema("c", &bottom_right_join.schema()).unwrap()) + as _, ), ( Arc::new( - Column::new_with_schema("AA", &bottom_left_projection.schema()) - .unwrap(), - ) as _, - Arc::new( - Column::new_with_schema("a1", &bottom_right_join.schema()).unwrap(), + Column::new_with_schema("AA", &bottom_left_projection.schema()).unwrap(), ) as _, + Arc::new(Column::new_with_schema("a1", &bottom_right_join.schema()).unwrap()) + as _, ), ]; @@ -1401,20 +1376,15 @@ fn multi_smj_joins() -> Result<()> { for join_type in join_types { let join = sort_merge_join_exec(left.clone(), right.clone(), &join_on, &join_type); - let join_plan = - format!("SortMergeJoin: join_type={join_type}, on=[(a@0, b1@1)]"); + let join_plan = format!("SortMergeJoin: join_type={join_type}, on=[(a@0, b1@1)]"); // Top join on (a == c) let top_join_on = vec![( Arc::new(Column::new_with_schema("a", &join.schema()).unwrap()) as _, Arc::new(Column::new_with_schema("c", &schema()).unwrap()) as _, )]; - let top_join = sort_merge_join_exec( - join.clone(), - parquet_exec(), - &top_join_on, - &join_type, - ); + let top_join = + sort_merge_join_exec(join.clone(), parquet_exec(), &top_join_on, &join_type); let top_join_plan = format!("SortMergeJoin: join_type={join_type}, on=[(a@0, c@2)]"); @@ -1529,16 +1499,11 @@ fn multi_smj_joins() -> Result<()> { // This time we use (b1 == c) for top join // Join on (b1 == c) let top_join_on = vec![( - Arc::new(Column::new_with_schema("b1", &join.schema()).unwrap()) - as _, + Arc::new(Column::new_with_schema("b1", &join.schema()).unwrap()) as _, Arc::new(Column::new_with_schema("c", &schema()).unwrap()) as _, )]; - let top_join = sort_merge_join_exec( - join, - parquet_exec(), - &top_join_on, - &join_type, - ); + let top_join = + sort_merge_join_exec(join, parquet_exec(), &top_join_on, &join_type); let top_join_plan = format!("SortMergeJoin: join_type={join_type}, on=[(b1@6, c@2)]"); @@ -1630,12 +1595,7 @@ fn multi_smj_joins() -> Result<()> { // this match arm cannot be reached _ => unreachable!() }; - assert_optimized!( - expected_first_sort_enforcement, - top_join, - false, - true - ); + assert_optimized!(expected_first_sort_enforcement, top_join, false, true); } _ => {} } @@ -2525,8 +2485,7 @@ fn parallelization_compressed_csv() -> Result<()> { #[test] fn parallelization_two_partitions() -> Result<()> { let alias = vec![("a".to_string(), "a".to_string())]; - let plan_parquet = - aggregate_exec_with_alias(parquet_exec_multiple(), alias.clone()); + let plan_parquet = aggregate_exec_with_alias(parquet_exec_multiple(), alias.clone()); let plan_csv = aggregate_exec_with_alias(csv_exec_multiple(), alias); let expected_parquet = [ @@ -2551,8 +2510,7 @@ fn parallelization_two_partitions() -> Result<()> { #[test] fn parallelization_two_partitions_into_four() -> Result<()> { let alias = vec![("a".to_string(), "a".to_string())]; - let plan_parquet = - aggregate_exec_with_alias(parquet_exec_multiple(), alias.clone()); + let plan_parquet = aggregate_exec_with_alias(parquet_exec_multiple(), alias.clone()); let plan_csv = aggregate_exec_with_alias(csv_exec_multiple(), alias); let expected_parquet = [ @@ -2658,10 +2616,8 @@ fn parallelization_ignores_limit() -> Result<()> { limit_exec(filter_exec(limit_exec(parquet_exec()))), alias.clone(), ); - let plan_csv = aggregate_exec_with_alias( - limit_exec(filter_exec(limit_exec(csv_exec()))), - alias, - ); + let plan_csv = + aggregate_exec_with_alias(limit_exec(filter_exec(limit_exec(csv_exec()))), alias); let expected_parquet = &[ "AggregateExec: mode=FinalPartitioned, gby=[a@0 as a], aggr=[]", @@ -2742,10 +2698,8 @@ fn parallelization_prior_to_sort_preserving_merge() -> Result<()> { sort_key.clone(), parquet_exec_with_sort(vec![sort_key.clone()]), ); - let plan_csv = sort_preserving_merge_exec( - sort_key.clone(), - csv_exec_with_sort(vec![sort_key]), - ); + let plan_csv = + sort_preserving_merge_exec(sort_key.clone(), csv_exec_with_sort(vec![sort_key])); // parallelization is not beneficial for SortPreservingMerge let expected_parquet = &[ @@ -2807,10 +2761,8 @@ fn parallelization_does_not_benefit() -> Result<()> { parquet_exec_with_sort(vec![sort_key.clone()]), sort_key.clone(), ); - let plan_csv = sort_required_exec_with_req( - csv_exec_with_sort(vec![sort_key.clone()]), - sort_key, - ); + let plan_csv = + sort_required_exec_with_req(csv_exec_with_sort(vec![sort_key.clone()]), sort_key); // no parallelization, because SortRequiredExec doesn't benefit from increased parallelism let expected_parquet = &[ @@ -2841,10 +2793,8 @@ fn parallelization_ignores_transitively_with_projection_parquet() -> Result<()> ("a".to_string(), "a2".to_string()), ("c".to_string(), "c2".to_string()), ]; - let proj_parquet = projection_exec_with_alias( - parquet_exec_with_sort(vec![sort_key]), - alias_pairs, - ); + let proj_parquet = + projection_exec_with_alias(parquet_exec_with_sort(vec![sort_key]), alias_pairs); let sort_key_after_projection = LexOrdering::new(vec![PhysicalSortExpr { expr: col("c2", &proj_parquet.schema()).unwrap(), options: SortOptions::default(), diff --git a/datafusion/core/tests/physical_optimizer/mod.rs b/datafusion/core/tests/physical_optimizer/mod.rs index 3702c62ecf69..f62159ebb0da 100644 --- a/datafusion/core/tests/physical_optimizer/mod.rs +++ b/datafusion/core/tests/physical_optimizer/mod.rs @@ -16,8 +16,8 @@ // under the License. mod combine_partial_final_agg; -mod enforce_sorting; mod enforce_distribution; +mod enforce_sorting; mod limited_distinct_aggregation; mod replace_with_order_preserving_variants; mod sanity_checker; From 9c40e497a95707c4dd7ac6d0c5601227e154fee6 Mon Sep 17 00:00:00 2001 From: buraksenn Date: Tue, 21 Jan 2025 15:32:22 +0300 Subject: [PATCH 14/24] fix tests --- .../physical_optimizer/enforce_sorting.rs | 3 +- .../replace_with_order_preserving_variants.rs | 38 +++++++++---------- .../tests/physical_optimizer/test_util.rs | 23 +++++++++++ 3 files changed, 43 insertions(+), 21 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs index 68ac2e8b54a0..21dbe06d7ac2 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs @@ -36,8 +36,7 @@ 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, + 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; diff --git a/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs b/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs index caced567df4c..e102ab5041b7 100644 --- a/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs +++ b/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs @@ -39,14 +39,14 @@ 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 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; +use crate::physical_optimizer::test_util::stream_exec_ordered_with_projection; use object_store::memory::InMemory; use object_store::ObjectStore; @@ -190,7 +190,7 @@ async fn test_replace_multiple_input_repartition_1( let schema = create_test_schema()?; let sort_exprs = vec![sort_expr("a", &schema)]; let source = if source_unbounded { - stream_exec_ordered(&schema, sort_exprs) + stream_exec_ordered_with_projection(&schema, sort_exprs) } else { memory_exec_sorted(&schema, sort_exprs) }; @@ -259,7 +259,7 @@ async fn test_with_inter_children_change_only( 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) + stream_exec_ordered_with_projection(&schema, sort_exprs) } else { memory_exec_sorted(&schema, sort_exprs) }; @@ -362,7 +362,7 @@ async fn test_replace_multiple_input_repartition_2( let schema = create_test_schema()?; let sort_exprs = vec![sort_expr("a", &schema)]; let source = if source_unbounded { - stream_exec_ordered(&schema, sort_exprs) + stream_exec_ordered_with_projection(&schema, sort_exprs) } else { memory_exec_sorted(&schema, sort_exprs) }; @@ -438,7 +438,7 @@ async fn test_replace_multiple_input_repartition_with_extra_steps( let schema = create_test_schema()?; let sort_exprs = vec![sort_expr("a", &schema)]; let source = if source_unbounded { - stream_exec_ordered(&schema, sort_exprs) + stream_exec_ordered_with_projection(&schema, sort_exprs) } else { memory_exec_sorted(&schema, sort_exprs) }; @@ -520,7 +520,7 @@ async fn test_replace_multiple_input_repartition_with_extra_steps_2( let schema = create_test_schema()?; let sort_exprs = vec![sort_expr("a", &schema)]; let source = if source_unbounded { - stream_exec_ordered(&schema, sort_exprs) + stream_exec_ordered_with_projection(&schema, sort_exprs) } else { memory_exec_sorted(&schema, sort_exprs) }; @@ -608,7 +608,7 @@ async fn test_not_replacing_when_no_need_to_preserve_sorting( let schema = create_test_schema()?; let sort_exprs = vec![sort_expr("a", &schema)]; let source = if source_unbounded { - stream_exec_ordered(&schema, sort_exprs) + stream_exec_ordered_with_projection(&schema, sort_exprs) } else { memory_exec_sorted(&schema, sort_exprs) }; @@ -681,7 +681,7 @@ async fn test_with_multiple_replacable_repartitions( let schema = create_test_schema()?; let sort_exprs = vec![sort_expr("a", &schema)]; let source = if source_unbounded { - stream_exec_ordered(&schema, sort_exprs) + stream_exec_ordered_with_projection(&schema, sort_exprs) } else { memory_exec_sorted(&schema, sort_exprs) }; @@ -769,7 +769,7 @@ async fn test_not_replace_with_different_orderings( let schema = create_test_schema()?; let sort_exprs = vec![sort_expr("a", &schema)]; let source = if source_unbounded { - stream_exec_ordered(&schema, sort_exprs) + stream_exec_ordered_with_projection(&schema, sort_exprs) } else { memory_exec_sorted(&schema, sort_exprs) }; @@ -841,7 +841,7 @@ async fn test_with_lost_ordering( let schema = create_test_schema()?; let sort_exprs = vec![sort_expr("a", &schema)]; let source = if source_unbounded { - stream_exec_ordered(&schema, sort_exprs) + stream_exec_ordered_with_projection(&schema, sort_exprs) } else { memory_exec_sorted(&schema, sort_exprs) }; @@ -911,7 +911,7 @@ async fn test_with_lost_and_kept_ordering( let schema = create_test_schema()?; let sort_exprs = vec![sort_expr("a", &schema)]; let source = if source_unbounded { - stream_exec_ordered(&schema, sort_exprs) + stream_exec_ordered_with_projection(&schema, sort_exprs) } else { memory_exec_sorted(&schema, sort_exprs) }; @@ -1017,7 +1017,7 @@ async fn test_with_multiple_child_trees( let left_sort_exprs = vec![sort_expr("a", &schema)]; let left_source = if source_unbounded { - stream_exec_ordered(&schema, left_sort_exprs) + stream_exec_ordered_with_projection(&schema, left_sort_exprs) } else { memory_exec_sorted(&schema, left_sort_exprs) }; @@ -1028,7 +1028,7 @@ async fn test_with_multiple_child_trees( let right_sort_exprs = vec![sort_expr("a", &schema)]; let right_source = if source_unbounded { - stream_exec_ordered(&schema, right_sort_exprs) + stream_exec_ordered_with_projection(&schema, right_sort_exprs) } else { memory_exec_sorted(&schema, right_sort_exprs) }; diff --git a/datafusion/core/tests/physical_optimizer/test_util.rs b/datafusion/core/tests/physical_optimizer/test_util.rs index df75bfb818bf..3c3da36199b1 100644 --- a/datafusion/core/tests/physical_optimizer/test_util.rs +++ b/datafusion/core/tests/physical_optimizer/test_util.rs @@ -268,6 +268,29 @@ pub fn stream_exec_ordered( ) } +// 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, From 9d19863029d1dd29101c0fe9b257a981c146d8d4 Mon Sep 17 00:00:00 2001 From: logan-keede Date: Tue, 21 Jan 2025 21:29:37 +0530 Subject: [PATCH 15/24] fix: ci tests --- .../enforce_distribution.rs | 40 ++++++++++++++++--- .../src/enforce_distribution.rs | 10 ++--- datafusion/physical-optimizer/src/lib.rs | 2 +- 3 files changed, 40 insertions(+), 12 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index 5423af384a3b..6c9ae24816e7 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -16,12 +16,15 @@ // under the License. use std::ops::Deref; +use std::sync::Arc; -use super::*; +use datafusion::config::ConfigOptions; use datafusion::datasource::file_format::file_compression_type::FileCompressionType; use datafusion::datasource::listing::PartitionedFile; use datafusion::datasource::object_store::ObjectStoreUrl; use datafusion::datasource::physical_plan::{CsvExec, FileScanConfig, ParquetExec}; + +use datafusion_physical_expr::PhysicalExpr; use datafusion_physical_optimizer::enforce_sorting::EnforceSorting; use datafusion_physical_optimizer::output_requirements::OutputRequirements; use datafusion_physical_optimizer::test_utils::{ @@ -31,22 +34,48 @@ use datafusion_physical_plan::coalesce_batches::CoalesceBatchesExec; use datafusion_physical_plan::expressions::col; use datafusion_physical_plan::filter::FilterExec; use datafusion_physical_plan::joins::utils::JoinOn; +use datafusion_physical_plan::joins::{HashJoinExec, PartitionMode, SortMergeJoinExec}; use datafusion_physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; 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::{displayable, DisplayAs, DisplayFormatType, Statistics}; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use datafusion_common::ScalarValue; -use datafusion_expr::Operator; -use datafusion_physical_expr::expressions::{BinaryExpr, Literal}; +use datafusion_expr::{JoinType, Operator}; +use datafusion_physical_expr::expressions::{BinaryExpr, Column, Literal}; use datafusion_physical_expr::{ expressions::binary, expressions::lit, LexOrdering, PhysicalSortExpr, }; use datafusion_physical_expr_common::sort_expr::LexRequirement; +use datafusion_physical_optimizer::enforce_distribution::*; +use datafusion_physical_optimizer::PhysicalOptimizerRule; +use datafusion_physical_plan::aggregates::{ + AggregateExec, AggregateMode, PhysicalGroupBy, +}; +use datafusion_physical_plan::execution_plan::ExecutionPlan; +use datafusion_physical_plan::projection::ProjectionExec; use datafusion_physical_plan::PlanProperties; +use std::fmt::Debug; + +use datafusion_common::error::Result; + +use arrow::compute::SortOptions; +use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; + +use datafusion_physical_plan::ExecutionPlanProperties; + +type DistributionContext = PlanContext; + +/// Keeps track of parent required key orderings. +type PlanWithKeyRequirements = PlanContext>>; /// Models operators like BoundedWindowExec that require an input /// ordering but is easy to construct +/// #[derive(Debug)] struct SortRequiredExec { input: Arc, @@ -133,8 +162,8 @@ impl ExecutionPlan for SortRequiredExec { fn execute( &self, _partition: usize, - _context: Arc, - ) -> Result { + _context: Arc, + ) -> Result { unreachable!(); } @@ -340,7 +369,6 @@ fn filter_exec(input: Arc) -> Arc { )); Arc::new(FilterExec::try_new(predicate, input).unwrap()) } - fn sort_exec( sort_exprs: LexOrdering, input: Arc, diff --git a/datafusion/physical-optimizer/src/enforce_distribution.rs b/datafusion/physical-optimizer/src/enforce_distribution.rs index be4c6667df3f..611106941d59 100644 --- a/datafusion/physical-optimizer/src/enforce_distribution.rs +++ b/datafusion/physical-optimizer/src/enforce_distribution.rs @@ -278,7 +278,7 @@ impl PhysicalOptimizerRule for EnforceDistribution { /// 4) If the current plan is Projection, transform the requirements to the columns before the Projection and push down requirements /// 5) For other types of operators, by default, pushdown the parent requirements to children. /// -fn adjust_input_keys_ordering( +pub fn adjust_input_keys_ordering( mut requirements: PlanWithKeyRequirements, ) -> Result> { let plan = Arc::clone(&requirements.plan); @@ -427,7 +427,7 @@ fn adjust_input_keys_ordering( Ok(Transformed::yes(requirements)) } -fn reorder_partitioned_join_keys( +pub fn reorder_partitioned_join_keys( mut join_plan: PlanWithKeyRequirements, on: &[(PhysicalExprRef, PhysicalExprRef)], sort_options: &[SortOptions], @@ -465,7 +465,7 @@ where Ok(join_plan) } -fn reorder_aggregate_keys( +pub fn reorder_aggregate_keys( mut agg_node: PlanWithKeyRequirements, agg_exec: &AggregateExec, ) -> Result { @@ -600,7 +600,7 @@ fn shift_right_required( /// The Bottom-Up approach will be useful in future if we plan to support storage partition-wised Joins. /// In that case, the datasources/tables might be pre-partitioned and we can't adjust the key ordering of the datasources /// and then can't apply the Top-Down reordering process. -pub(crate) fn reorder_join_keys_to_inputs( +pub fn reorder_join_keys_to_inputs( plan: Arc, ) -> Result> { let plan_any = plan.as_any(); @@ -1150,7 +1150,7 @@ fn get_repartition_requirement_status( /// operators to satisfy distribution requirements. Since this function /// takes care of such requirements, we should avoid manually adding data /// exchange operators in other places. -fn ensure_distribution( +pub fn ensure_distribution( dist_context: DistributionContext, config: &ConfigOptions, ) -> Result> { diff --git a/datafusion/physical-optimizer/src/lib.rs b/datafusion/physical-optimizer/src/lib.rs index 4fb26b950419..fed7bf07b34f 100644 --- a/datafusion/physical-optimizer/src/lib.rs +++ b/datafusion/physical-optimizer/src/lib.rs @@ -34,4 +34,4 @@ pub mod test_utils; pub mod topk_aggregation; pub mod update_aggr_exprs; pub use optimizer::PhysicalOptimizerRule; -mod utils; +pub mod utils; From eba718a3765e8270ee7b824b1650fa401cc6f0db Mon Sep 17 00:00:00 2001 From: logan-keede Date: Tue, 21 Jan 2025 21:55:57 +0530 Subject: [PATCH 16/24] fix: Cargo.toml formatting --- datafusion/core/tests/physical_optimizer/mod.rs | 1 - datafusion/physical-optimizer/Cargo.toml | 1 - 2 files changed, 2 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/mod.rs b/datafusion/core/tests/physical_optimizer/mod.rs index f62159ebb0da..4f463a35ef4c 100644 --- a/datafusion/core/tests/physical_optimizer/mod.rs +++ b/datafusion/core/tests/physical_optimizer/mod.rs @@ -21,5 +21,4 @@ mod enforce_sorting; mod limited_distinct_aggregation; mod replace_with_order_preserving_variants; mod sanity_checker; -// mod enforce_distribution; pub(crate) mod test_util; diff --git a/datafusion/physical-optimizer/Cargo.toml b/datafusion/physical-optimizer/Cargo.toml index 2f6c89972bd3..c6c75a0e0939 100644 --- a/datafusion/physical-optimizer/Cargo.toml +++ b/datafusion/physical-optimizer/Cargo.toml @@ -34,7 +34,6 @@ workspace = true [features] recursive_protection = ["dep:recursive"] - [dependencies] arrow = { workspace = true } arrow-schema = { workspace = true } From afb20f87a1e35ff8a26fecb124025fe06808f220 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Wed, 22 Jan 2025 14:41:22 +0300 Subject: [PATCH 17/24] further removals --- datafusion/core/tests/core_integration.rs | 1 - .../combine_partial_final_agg.rs | 13 +- .../physical_optimizer/enforce_sorting.rs | 1508 +++-------------- .../core/tests/physical_optimizer/mod.rs | 89 +- .../physical_optimizer/sanity_checker.rs | 630 ++----- .../tests/physical_optimizer/test_util.rs | 319 ---- datafusion/physical-optimizer/src/lib.rs | 1 - .../tests/enforce_sorting.rs | 1154 +++++++++++++ .../tests}/limited_distinct_aggregation.rs | 10 +- datafusion/physical-optimizer/tests/mod.rs | 22 + .../replace_with_order_preserving_variants.rs | 21 +- .../tests/sanity_checker.rs | 536 ++++++ .../{src => tests}/test_utils.rs | 86 +- 13 files changed, 2192 insertions(+), 2198 deletions(-) delete mode 100644 datafusion/core/tests/physical_optimizer/test_util.rs create mode 100644 datafusion/physical-optimizer/tests/enforce_sorting.rs rename datafusion/{core/tests/physical_optimizer => physical-optimizer/tests}/limited_distinct_aggregation.rs (97%) create mode 100644 datafusion/physical-optimizer/tests/mod.rs rename datafusion/{core/tests/physical_optimizer => physical-optimizer/tests}/replace_with_order_preserving_variants.rs (99%) create mode 100644 datafusion/physical-optimizer/tests/sanity_checker.rs rename datafusion/physical-optimizer/{src => tests}/test_utils.rs (84%) diff --git a/datafusion/core/tests/core_integration.rs b/datafusion/core/tests/core_integration.rs index e0917e6cca19..93f66282333d 100644 --- a/datafusion/core/tests/core_integration.rs +++ b/datafusion/core/tests/core_integration.rs @@ -42,7 +42,6 @@ mod custom_sources_cases; /// Run all tests that are found in the `optimizer` directory mod optimizer; -/// Run all tests that are found in the `physical_optimizer` directory mod physical_optimizer; #[cfg(test)] diff --git a/datafusion/core/tests/physical_optimizer/combine_partial_final_agg.rs b/datafusion/core/tests/physical_optimizer/combine_partial_final_agg.rs index b8a96f0f5a22..4b161b584c85 100644 --- a/datafusion/core/tests/physical_optimizer/combine_partial_final_agg.rs +++ b/datafusion/core/tests/physical_optimizer/combine_partial_final_agg.rs @@ -19,12 +19,16 @@ //! //! Note these tests are not in the same module as the optimizer pass because //! they rely on `ParquetExec` which is in the core crate. + +//! Tests for [`CombinePartialFinalAggregate`] physical optimizer rule + +mod r#mod; + use std::sync::Arc; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use datafusion::datasource::listing::PartitionedFile; use datafusion::datasource::physical_plan::{FileScanConfig, ParquetExec}; -use datafusion::physical_optimizer::combine_partial_final_agg::CombinePartialFinalAggregate; use datafusion_common::config::ConfigOptions; use datafusion_execution::object_store::ObjectStoreUrl; use datafusion_functions_aggregate::count::count_udaf; @@ -62,13 +66,6 @@ macro_rules! assert_optimized { }; } -fn trim_plan_display(plan: &str) -> Vec<&str> { - plan.split('\n') - .map(|s| s.trim()) - .filter(|s| !s.is_empty()) - .collect() -} - fn schema() -> SchemaRef { Arc::new(Schema::new(vec![ Field::new("a", DataType::Int64, true), diff --git a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs index 21dbe06d7ac2..b8cc75f1c44a 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs @@ -15,14 +15,6 @@ // 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; @@ -34,11 +26,6 @@ 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}; @@ -50,389 +37,8 @@ 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(()) -} +use datafusion_common::config::ConfigOptions; +use datafusion_common::tree_node::{TreeNode, TransformedResult}; #[tokio::test] async fn test_remove_unnecessary_sort5() -> Result<()> { @@ -450,169 +56,13 @@ async fn test_remove_unnecessary_sort5() -> Result<()> { let physical_plan = sort_exec(vec![sort_expr("a", &join.schema())], join); let expected_input = ["SortExec: expr=[a@2 ASC], preserve_partitioning=[false]", - " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(col_a@0, c@2)]", - " MemoryExec: partitions=1, partition_sizes=[0]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]"]; + " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(col_a@0, c@2)]", + " MemoryExec: partitions=1, partition_sizes=[0]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]"]; let expected_optimized = ["HashJoinExec: mode=Partitioned, join_type=Inner, on=[(col_a@0, c@2)]", - " MemoryExec: partitions=1, partition_sizes=[0]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]"]; - assert_optimized!(expected_input, expected_optimized, physical_plan, true); - - Ok(()) -} - -#[tokio::test] -async fn test_remove_unnecessary_sort6() -> Result<()> { - let schema = create_test_schema()?; - let source = memory_exec(&schema); - let input = Arc::new( - SortExec::new( - LexOrdering::new(vec![sort_expr("non_nullable_col", &schema)]), - source, - ) - .with_fetch(Some(2)), - ); - let physical_plan = sort_exec( - vec![ - sort_expr("non_nullable_col", &schema), - sort_expr("nullable_col", &schema), - ], - input, - ); - - let expected_input = [ - "SortExec: expr=[non_nullable_col@1 ASC, nullable_col@0 ASC], preserve_partitioning=[false]", - " SortExec: TopK(fetch=2), expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", - ]; - let expected_optimized = [ - "SortExec: TopK(fetch=2), expr=[non_nullable_col@1 ASC, nullable_col@0 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", - ]; - assert_optimized!(expected_input, expected_optimized, physical_plan, true); - - Ok(()) -} - -#[tokio::test] -async fn test_remove_unnecessary_sort7() -> Result<()> { - let schema = create_test_schema()?; - let source = memory_exec(&schema); - let input = Arc::new(SortExec::new( - LexOrdering::new(vec![ - sort_expr("non_nullable_col", &schema), - sort_expr("nullable_col", &schema), - ]), - source, - )); - - let physical_plan = Arc::new( - SortExec::new( - LexOrdering::new(vec![sort_expr("non_nullable_col", &schema)]), - input, - ) - .with_fetch(Some(2)), - ) as Arc; - - let expected_input = [ - "SortExec: TopK(fetch=2), expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", - " SortExec: expr=[non_nullable_col@1 ASC, nullable_col@0 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", - ]; - let expected_optimized = [ - "GlobalLimitExec: skip=0, fetch=2", - " SortExec: expr=[non_nullable_col@1 ASC, nullable_col@0 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", - ]; - assert_optimized!(expected_input, expected_optimized, physical_plan, true); - - Ok(()) -} - -#[tokio::test] -async fn test_remove_unnecessary_sort8() -> Result<()> { - let schema = create_test_schema()?; - let source = memory_exec(&schema); - let input = Arc::new(SortExec::new( - LexOrdering::new(vec![sort_expr("non_nullable_col", &schema)]), - source, - )); - let limit = Arc::new(LocalLimitExec::new(input, 2)); - let physical_plan = sort_exec( - vec![ - sort_expr("non_nullable_col", &schema), - sort_expr("nullable_col", &schema), - ], - limit, - ); - - let expected_input = [ - "SortExec: expr=[non_nullable_col@1 ASC, nullable_col@0 ASC], preserve_partitioning=[false]", - " LocalLimitExec: fetch=2", - " SortExec: expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", - ]; - let expected_optimized = [ - "LocalLimitExec: fetch=2", - " SortExec: TopK(fetch=2), expr=[non_nullable_col@1 ASC, nullable_col@0 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", - ]; - assert_optimized!(expected_input, expected_optimized, physical_plan, true); - - Ok(()) -} - -#[tokio::test] -async fn test_do_not_pushdown_through_limit() -> Result<()> { - let schema = create_test_schema()?; - let source = memory_exec(&schema); - // let input = sort_exec(vec![sort_expr("non_nullable_col", &schema)], source); - let input = Arc::new(SortExec::new( - LexOrdering::new(vec![sort_expr("non_nullable_col", &schema)]), - source, - )); - let limit = Arc::new(GlobalLimitExec::new(input, 0, Some(5))) as _; - let physical_plan = sort_exec(vec![sort_expr("nullable_col", &schema)], limit); - - let expected_input = [ - "SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " GlobalLimitExec: skip=0, fetch=5", - " SortExec: expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", - ]; - let expected_optimized = [ - "SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " GlobalLimitExec: skip=0, fetch=5", - " SortExec: expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", - ]; - assert_optimized!(expected_input, expected_optimized, physical_plan, true); - - Ok(()) -} - -#[tokio::test] -async fn test_remove_unnecessary_spm1() -> Result<()> { - let schema = create_test_schema()?; - let source = memory_exec(&schema); - let input = - sort_preserving_merge_exec(vec![sort_expr("non_nullable_col", &schema)], source); - let input2 = - sort_preserving_merge_exec(vec![sort_expr("non_nullable_col", &schema)], input); - let physical_plan = - sort_preserving_merge_exec(vec![sort_expr("nullable_col", &schema)], input2); - - let expected_input = [ - "SortPreservingMergeExec: [nullable_col@0 ASC]", - " SortPreservingMergeExec: [non_nullable_col@1 ASC]", - " SortPreservingMergeExec: [non_nullable_col@1 ASC]", - " MemoryExec: partitions=1, partition_sizes=[0]", - ]; - let expected_optimized = [ - "SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", " MemoryExec: partitions=1, partition_sizes=[0]", - ]; + " ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -638,75 +88,24 @@ async fn test_do_not_remove_sort_with_limit() -> Result<()> { let physical_plan = sort_preserving_merge_exec(sort_exprs, repartition); let expected_input = ["SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", - " UnionExec", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", - " GlobalLimitExec: skip=0, fetch=100", - " LocalLimitExec: fetch=100", - " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", + " UnionExec", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", + " GlobalLimitExec: skip=0, fetch=100", + " LocalLimitExec: fetch=100", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; // We should keep the bottom `SortExec`. let expected_optimized = ["SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", - " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[true]", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", - " UnionExec", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", - " GlobalLimitExec: skip=0, fetch=100", - " LocalLimitExec: fetch=100", - " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; - assert_optimized!(expected_input, expected_optimized, physical_plan, true); - - Ok(()) -} - -#[tokio::test] -async fn test_change_wrong_sorting() -> Result<()> { - let schema = create_test_schema()?; - let source = memory_exec(&schema); - let sort_exprs = vec![ - sort_expr("nullable_col", &schema), - sort_expr("non_nullable_col", &schema), - ]; - let sort = sort_exec(vec![sort_exprs[0].clone()], source); - let physical_plan = sort_preserving_merge_exec(sort_exprs, sort); - let expected_input = [ - "SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", - " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", - ]; - let expected_optimized = [ - "SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", - ]; - assert_optimized!(expected_input, expected_optimized, physical_plan, true); - - Ok(()) -} - -#[tokio::test] -async fn test_change_wrong_sorting2() -> Result<()> { - let schema = create_test_schema()?; - let source = memory_exec(&schema); - let sort_exprs = vec![ - sort_expr("nullable_col", &schema), - sort_expr("non_nullable_col", &schema), - ]; - let spm1 = sort_preserving_merge_exec(sort_exprs.clone(), source); - let sort2 = sort_exec(vec![sort_exprs[0].clone()], spm1); - let physical_plan = sort_preserving_merge_exec(vec![sort_exprs[1].clone()], sort2); - - let expected_input = [ - "SortPreservingMergeExec: [non_nullable_col@1 ASC]", - " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", - " MemoryExec: partitions=1, partition_sizes=[0]", - ]; - let expected_optimized = [ - "SortExec: expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", - ]; + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[true]", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", + " UnionExec", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", + " GlobalLimitExec: skip=0, fetch=100", + " LocalLimitExec: fetch=100", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -727,12 +126,12 @@ async fn test_union_inputs_sorted() -> Result<()> { // one input to the union is already sorted, one is not. let expected_input = vec![ - "SortPreservingMergeExec: [nullable_col@0 ASC]", - " UnionExec", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", - " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - ]; + "SortPreservingMergeExec: [nullable_col@0 ASC]", + " UnionExec", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", + " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + ]; // should not add a sort at the output of the union, input plan should not be changed let expected_optimized = expected_input.clone(); assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -759,12 +158,12 @@ async fn test_union_inputs_different_sorted() -> Result<()> { // one input to the union is already sorted, one is not. let expected_input = vec![ - "SortPreservingMergeExec: [nullable_col@0 ASC]", - " UnionExec", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC, non_nullable_col@1 ASC]", - " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - ]; + "SortPreservingMergeExec: [nullable_col@0 ASC]", + " UnionExec", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC, non_nullable_col@1 ASC]", + " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + ]; // should not add a sort at the output of the union, input plan should not be changed let expected_optimized = expected_input.clone(); assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -793,17 +192,17 @@ async fn test_union_inputs_different_sorted2() -> Result<()> { // First ParquetExec has output ordering(nullable_col@0 ASC). However, it doesn't satisfy the // required ordering of SortPreservingMergeExec. let expected_input = ["SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", - " UnionExec", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", - " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; + " UnionExec", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; let expected_optimized = ["SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", - " UnionExec", - " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", - " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; + " UnionExec", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -832,20 +231,20 @@ async fn test_union_inputs_different_sorted3() -> Result<()> { // Second input to the union is already Sorted (matches with the required ordering by the SortPreservingMergeExec above). // Third input to the union is not Sorted (SortExec is matches required ordering by the SortPreservingMergeExec above). let expected_input = ["SortPreservingMergeExec: [nullable_col@0 ASC]", - " UnionExec", - " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", - " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; + " UnionExec", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", + " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; // should adjust sorting in the first input of the union such that it is not unnecessarily fine let expected_optimized = ["SortPreservingMergeExec: [nullable_col@0 ASC]", - " UnionExec", - " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", - " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; + " UnionExec", + " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", + " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -874,20 +273,20 @@ async fn test_union_inputs_different_sorted4() -> Result<()> { // `UnionExec` satisfy the ordering, OR add a single sort after // the `UnionExec` (both of which are equally good for this example). let expected_input = ["SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", - " UnionExec", - " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", - " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; + " UnionExec", + " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", + " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; let expected_optimized = ["SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", - " UnionExec", - " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", - " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; + " UnionExec", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -924,17 +323,17 @@ async fn test_union_inputs_different_sorted5() -> Result<()> { // example below. However, we should be able to change the unnecessarily // fine `SortExec`s below with required `SortExec`s that are absolutely necessary. let expected_input = ["SortPreservingMergeExec: [nullable_col@0 ASC]", - " UnionExec", - " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 DESC NULLS LAST], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; + " UnionExec", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 DESC NULLS LAST], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; let expected_optimized = ["SortPreservingMergeExec: [nullable_col@0 ASC]", - " UnionExec", - " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; + " UnionExec", + " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -967,23 +366,23 @@ async fn test_union_inputs_different_sorted6() -> Result<()> { // The final plan should be valid AND the ordering of the third child // shouldn't be finer than necessary. let expected_input = ["SortPreservingMergeExec: [nullable_col@0 ASC]", - " UnionExec", - " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", - " SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; + " UnionExec", + " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", + " SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; // Should adjust the requirement in the third input of the union so // that it is not unnecessarily fine. let expected_optimized = ["SortPreservingMergeExec: [nullable_col@0 ASC]", - " UnionExec", - " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", - " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[true]", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; + " UnionExec", + " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", + " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[true]", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -1007,18 +406,18 @@ async fn test_union_inputs_different_sorted7() -> Result<()> { // Union has unnecessarily fine ordering below it. We should be able to replace them with absolutely necessary ordering. let expected_input = ["SortPreservingMergeExec: [nullable_col@0 ASC]", - " UnionExec", - " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; + " UnionExec", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; // Union preserves the inputs ordering and we should not change any of the SortExecs under UnionExec let expected_output = ["SortPreservingMergeExec: [nullable_col@0 ASC]", - " UnionExec", - " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; + " UnionExec", + " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; assert_optimized!(expected_input, expected_output, physical_plan, true); Ok(()) @@ -1059,15 +458,15 @@ async fn test_union_inputs_different_sorted8() -> Result<()> { // The `UnionExec` doesn't preserve any of the inputs ordering in the // example below. let expected_input = ["UnionExec", - " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " SortExec: expr=[nullable_col@0 DESC NULLS LAST, non_nullable_col@1 DESC NULLS LAST], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " SortExec: expr=[nullable_col@0 DESC NULLS LAST, non_nullable_col@1 DESC NULLS LAST], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; // Since `UnionExec` doesn't preserve ordering in the plan above. // We shouldn't keep SortExecs in the plan. let expected_optimized = ["UnionExec", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -1105,19 +504,19 @@ async fn test_window_multi_path_sort() -> Result<()> { // corresponding SortExecs together. Also, the inputs of these `SortExec`s // are not necessarily the same to be able to remove them. let expected_input = [ - "BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", - " SortPreservingMergeExec: [nullable_col@0 DESC NULLS LAST]", - " UnionExec", - " SortExec: expr=[nullable_col@0 DESC NULLS LAST], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC, non_nullable_col@1 ASC]", - " SortExec: expr=[nullable_col@0 DESC NULLS LAST], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]"]; + "BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", + " SortPreservingMergeExec: [nullable_col@0 DESC NULLS LAST]", + " UnionExec", + " SortExec: expr=[nullable_col@0 DESC NULLS LAST], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC, non_nullable_col@1 ASC]", + " SortExec: expr=[nullable_col@0 DESC NULLS LAST], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]"]; let expected_optimized = [ - "WindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(NULL), is_causal: false }]", - " SortPreservingMergeExec: [nullable_col@0 ASC]", - " UnionExec", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC, non_nullable_col@1 ASC]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]"]; + "WindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(NULL), is_causal: false }]", + " SortPreservingMergeExec: [nullable_col@0 ASC]", + " UnionExec", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC, non_nullable_col@1 ASC]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -1144,17 +543,17 @@ async fn test_window_multi_path_sort2() -> Result<()> { // The `WindowAggExec` can get its required sorting from the leaf nodes directly. // The unnecessary SortExecs should be removed let expected_input = ["BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", - " SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", - " UnionExec", - " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", - " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]"]; + " SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", + " UnionExec", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]"]; let expected_optimized = ["BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", - " SortPreservingMergeExec: [nullable_col@0 ASC]", - " UnionExec", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]"]; + " SortPreservingMergeExec: [nullable_col@0 ASC]", + " UnionExec", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col], output_ordering=[nullable_col@0 ASC]"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -1192,21 +591,21 @@ async fn test_union_inputs_different_sorted_with_limit() -> Result<()> { // Should not change the unnecessarily fine `SortExec`s because there is `LimitExec` let expected_input = ["SortPreservingMergeExec: [nullable_col@0 ASC]", - " UnionExec", - " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " GlobalLimitExec: skip=0, fetch=100", - " LocalLimitExec: fetch=100", - " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 DESC NULLS LAST], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; + " UnionExec", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " GlobalLimitExec: skip=0, fetch=100", + " LocalLimitExec: fetch=100", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 DESC NULLS LAST], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; let expected_optimized = ["SortPreservingMergeExec: [nullable_col@0 ASC]", - " UnionExec", - " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " GlobalLimitExec: skip=0, fetch=100", - " LocalLimitExec: fetch=100", - " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 DESC NULLS LAST], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; + " UnionExec", + " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " GlobalLimitExec: skip=0, fetch=100", + " LocalLimitExec: fetch=100", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 DESC NULLS LAST], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); Ok(()) @@ -1250,9 +649,9 @@ async fn test_sort_merge_join_order_by_left() -> Result<()> { " SortMergeJoin: join_type={join_type}, on=[(nullable_col@0, col_a@0)]" ); let expected_input = ["SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", - join_plan2.as_str(), - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]"]; + join_plan2.as_str(), + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]"]; let expected_optimized = match join_type { JoinType::Inner | JoinType::Left @@ -1260,23 +659,23 @@ async fn test_sort_merge_join_order_by_left() -> Result<()> { | JoinType::LeftAnti => { // can push down the sort requirements and save 1 SortExec vec![ - join_plan.as_str(), - " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " SortExec: expr=[col_a@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]", - ] + join_plan.as_str(), + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " SortExec: expr=[col_a@0 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]", + ] } _ => { // can not push down the sort requirements vec![ - "SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - join_plan2.as_str(), - " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " SortExec: expr=[col_a@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]", - ] + "SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", + join_plan2.as_str(), + " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " SortExec: expr=[col_a@0 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]", + ] } }; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -1325,30 +724,30 @@ async fn test_sort_merge_join_order_by_right() -> Result<()> { " SortMergeJoin: join_type={join_type}, on=[(nullable_col@0, col_a@0)]" ); let expected_input = [spm_plan, - join_plan2.as_str(), - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]"]; + join_plan2.as_str(), + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]"]; let expected_optimized = match join_type { JoinType::Inner | JoinType::Right | JoinType::RightAnti => { // can push down the sort requirements and save 1 SortExec vec![ - join_plan.as_str(), - " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " SortExec: expr=[col_a@0 ASC, col_b@1 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]", - ] + join_plan.as_str(), + " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " SortExec: expr=[col_a@0 ASC, col_b@1 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]", + ] } _ => { // can not push down the sort requirements for Left and Full join. vec![ - "SortExec: expr=[col_a@2 ASC, col_b@3 ASC], preserve_partitioning=[false]", - join_plan2.as_str(), - " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " SortExec: expr=[col_a@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]", - ] + "SortExec: expr=[col_a@2 ASC, col_b@3 ASC], preserve_partitioning=[false]", + join_plan2.as_str(), + " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " SortExec: expr=[col_a@0 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]", + ] } }; assert_optimized!(expected_input, expected_optimized, physical_plan, true); @@ -1380,17 +779,17 @@ async fn test_sort_merge_join_complex_order_by() -> Result<()> { let physical_plan = sort_preserving_merge_exec(sort_exprs1, join.clone()); let expected_input = ["SortPreservingMergeExec: [col_b@3 ASC, col_a@2 ASC]", - " SortMergeJoin: join_type=Inner, on=[(nullable_col@0, col_a@0)]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]"]; + " SortMergeJoin: join_type=Inner, on=[(nullable_col@0, col_a@0)]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]"]; // can not push down the sort requirements, need to add SortExec let expected_optimized = ["SortExec: expr=[col_b@3 ASC, col_a@2 ASC], preserve_partitioning=[false]", - " SortMergeJoin: join_type=Inner, on=[(nullable_col@0, col_a@0)]", - " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " SortExec: expr=[col_a@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]"]; + " SortMergeJoin: join_type=Inner, on=[(nullable_col@0, col_a@0)]", + " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " SortExec: expr=[col_a@0 ASC], preserve_partitioning=[false]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]"]; assert_optimized!(expected_input, expected_optimized, physical_plan, true); // order by (nullable_col, col_b, col_a) @@ -1402,543 +801,18 @@ async fn test_sort_merge_join_complex_order_by() -> Result<()> { let physical_plan = sort_preserving_merge_exec(sort_exprs2, join); let expected_input = ["SortPreservingMergeExec: [nullable_col@0 ASC, col_b@3 ASC, col_a@2 ASC]", - " SortMergeJoin: join_type=Inner, on=[(nullable_col@0, col_a@0)]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]"]; + " SortMergeJoin: join_type=Inner, on=[(nullable_col@0, col_a@0)]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]"]; // can not push down the sort requirements, need to add SortExec let expected_optimized = ["SortExec: expr=[nullable_col@0 ASC, col_b@3 ASC, col_a@2 ASC], preserve_partitioning=[false]", - " SortMergeJoin: join_type=Inner, on=[(nullable_col@0, col_a@0)]", - " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " SortExec: expr=[col_a@0 ASC], preserve_partitioning=[false]", - " ParquetExec: file_groups={1 group: [[x]]}, projection=[col_a, col_b]"]; - assert_optimized!(expected_input, expected_optimized, physical_plan, true); - - Ok(()) -} - -#[tokio::test] -async fn test_multiple_sort_window_exec() -> Result<()> { - let schema = create_test_schema()?; - let source = memory_exec(&schema); - - let sort_exprs1 = vec![sort_expr("nullable_col", &schema)]; - let sort_exprs2 = vec![ - sort_expr("nullable_col", &schema), - sort_expr("non_nullable_col", &schema), - ]; - - let sort1 = sort_exec(sort_exprs1.clone(), source); - let window_agg1 = bounded_window_exec("non_nullable_col", sort_exprs1.clone(), sort1); - let window_agg2 = bounded_window_exec("non_nullable_col", sort_exprs2, window_agg1); - // let filter_exec = sort_exec; - let physical_plan = bounded_window_exec("non_nullable_col", sort_exprs1, window_agg2); - - let expected_input = ["BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", - " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", - " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", - " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]"]; - - let expected_optimized = ["BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", - " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", - " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", - " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]"]; - 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]", - ]; + " 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_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 4f463a35ef4c..992028eeaa65 100644 --- a/datafusion/core/tests/physical_optimizer/mod.rs +++ b/datafusion/core/tests/physical_optimizer/mod.rs @@ -15,10 +15,89 @@ // specific language governing permissions and limitations // under the License. -mod combine_partial_final_agg; -mod enforce_distribution; +//! Physical Optimizer integration tests + mod enforce_sorting; -mod limited_distinct_aggregation; -mod replace_with_order_preserving_variants; mod sanity_checker; -pub(crate) mod test_util; + +use datafusion::datasource::listing::PartitionedFile; +use datafusion::datasource::physical_plan::{FileScanConfig, ParquetExec}; +use datafusion_execution::object_store::ObjectStoreUrl; +use datafusion_physical_expr_common::sort_expr::LexOrdering; +use std::sync::Arc; + +/// create a single parquet file that is sorted +pub(crate) fn parquet_exec_with_sort( + output_ordering: Vec, +) -> Arc { + ParquetExec::builder( + FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema()) + .with_file(PartitionedFile::new("x".to_string(), 100)) + .with_output_ordering(output_ordering), + ) + .build_arc() +} + +fn schema() -> SchemaRef { + Arc::new(Schema::new(vec![ + Field::new("a", DataType::Int64, true), + Field::new("b", DataType::Int64, true), + Field::new("c", DataType::Int64, true), + Field::new("d", DataType::Int32, true), + Field::new("e", DataType::Boolean, true), + ])) +} + +/// 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(), + ) +} + +/// 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/core/tests/physical_optimizer/sanity_checker.rs b/datafusion/core/tests/physical_optimizer/sanity_checker.rs index 538f0e443ddb..6b1f92996326 100644 --- a/datafusion/core/tests/physical_optimizer/sanity_checker.rs +++ b/datafusion/core/tests/physical_optimizer/sanity_checker.rs @@ -15,522 +15,116 @@ // specific language governing permissions and limitations // under the License. -//! Tests for [`SanityCheckPlan`] physical optimizer rule -//! -//! Note these tests are not in the same module as the optimizer pass because -//! they rely on `ParquetExec` which is in the core crate. - -use crate::physical_optimizer::test_util::{ - BinaryTestCase, QueryCase, SourceType, UnaryTestCase, -}; -use arrow::compute::SortOptions; -use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; -use datafusion_common::config::ConfigOptions; -use datafusion_common::Result; -use datafusion_expr::JoinType; -use datafusion_physical_expr::expressions::col; -use datafusion_physical_expr::Partitioning; -use datafusion_physical_optimizer::test_utils::{ - bounded_window_exec, global_limit_exec, local_limit_exec, memory_exec, - repartition_exec, sort_exec, sort_expr_options, sort_merge_join_exec, -}; -use datafusion_physical_optimizer::{sanity_checker::*, PhysicalOptimizerRule}; -use datafusion_physical_plan::displayable; -use datafusion_physical_plan::repartition::RepartitionExec; -use datafusion_physical_plan::ExecutionPlan; -use std::sync::Arc; - -fn create_test_schema() -> SchemaRef { - Arc::new(Schema::new(vec![Field::new("c9", DataType::Int32, true)])) -} - -fn create_test_schema2() -> SchemaRef { - Arc::new(Schema::new(vec![ - Field::new("a", DataType::Int32, true), - Field::new("b", DataType::Int32, true), - ])) -} - -/// Check if sanity checker should accept or reject plans. -fn assert_sanity_check(plan: &Arc, is_sane: bool) { - let sanity_checker = SanityCheckPlan::new(); - let opts = ConfigOptions::default(); - assert_eq!( - sanity_checker.optimize(plan.clone(), &opts).is_ok(), - is_sane - ); -} - -/// Check if the plan we created is as expected by comparing the plan -/// formatted as a string. -fn assert_plan(plan: &dyn ExecutionPlan, expected_lines: Vec<&str>) { - let plan_str = displayable(plan).indent(true).to_string(); - let actual_lines: Vec<&str> = plan_str.trim().lines().collect(); - assert_eq!(actual_lines, expected_lines); -} - -#[tokio::test] -async fn test_hash_left_join_swap() -> Result<()> { - let test1 = BinaryTestCase { - source_types: (SourceType::Unbounded, SourceType::Bounded), - expect_fail: false, - }; - - let test2 = BinaryTestCase { - source_types: (SourceType::Bounded, SourceType::Unbounded), - // Left join for bounded build side and unbounded probe side can generate - // both incremental matched rows and final non-matched rows. - expect_fail: false, - }; - let test3 = BinaryTestCase { - source_types: (SourceType::Bounded, SourceType::Bounded), - expect_fail: false, - }; - let case = QueryCase { - sql: "SELECT t2.c1 FROM left as t1 LEFT JOIN right as t2 ON t1.c1 = t2.c1" - .to_string(), - cases: vec![Arc::new(test1), Arc::new(test2), Arc::new(test3)], - error_operator: "operator: HashJoinExec".to_string(), - }; - - case.run().await?; - Ok(()) -} - -#[tokio::test] -async fn test_hash_right_join_swap() -> Result<()> { - let test1 = BinaryTestCase { - source_types: (SourceType::Unbounded, SourceType::Bounded), - expect_fail: true, - }; - let test2 = BinaryTestCase { - source_types: (SourceType::Bounded, SourceType::Unbounded), - expect_fail: false, - }; - let test3 = BinaryTestCase { - source_types: (SourceType::Bounded, SourceType::Bounded), - expect_fail: false, - }; - let case = QueryCase { - sql: "SELECT t2.c1 FROM left as t1 RIGHT JOIN right as t2 ON t1.c1 = t2.c1" - .to_string(), - cases: vec![Arc::new(test1), Arc::new(test2), Arc::new(test3)], - error_operator: "operator: HashJoinExec".to_string(), - }; - - case.run().await?; - Ok(()) -} - -#[tokio::test] -async fn test_hash_inner_join_swap() -> Result<()> { - let test1 = BinaryTestCase { - source_types: (SourceType::Unbounded, SourceType::Bounded), - expect_fail: false, - }; - let test2 = BinaryTestCase { - source_types: (SourceType::Bounded, SourceType::Unbounded), - expect_fail: false, - }; - let test3 = BinaryTestCase { - source_types: (SourceType::Bounded, SourceType::Bounded), - expect_fail: false, - }; - let case = QueryCase { - sql: "SELECT t2.c1 FROM left as t1 JOIN right as t2 ON t1.c1 = t2.c1".to_string(), - cases: vec![Arc::new(test1), Arc::new(test2), Arc::new(test3)], - error_operator: "Join Error".to_string(), - }; - - case.run().await?; - Ok(()) -} - -#[tokio::test] -async fn test_hash_full_outer_join_swap() -> Result<()> { - let test1 = BinaryTestCase { - source_types: (SourceType::Unbounded, SourceType::Bounded), - expect_fail: true, - }; - let test2 = BinaryTestCase { - source_types: (SourceType::Bounded, SourceType::Unbounded), - // Full join for bounded build side and unbounded probe side can generate - // both incremental matched rows and final non-matched rows. - expect_fail: false, - }; - let test3 = BinaryTestCase { - source_types: (SourceType::Bounded, SourceType::Bounded), - expect_fail: false, - }; - let case = QueryCase { - sql: "SELECT t2.c1 FROM left as t1 FULL JOIN right as t2 ON t1.c1 = t2.c1" - .to_string(), - cases: vec![Arc::new(test1), Arc::new(test2), Arc::new(test3)], - error_operator: "operator: HashJoinExec".to_string(), - }; - - case.run().await?; - Ok(()) -} - -#[tokio::test] -async fn test_aggregate() -> Result<()> { - let test1 = UnaryTestCase { - source_type: SourceType::Bounded, - expect_fail: false, - }; - let test2 = UnaryTestCase { - source_type: SourceType::Unbounded, - expect_fail: true, - }; - let case = QueryCase { - sql: "SELECT c1, MIN(c4) FROM test GROUP BY c1".to_string(), - cases: vec![Arc::new(test1), Arc::new(test2)], - error_operator: "operator: AggregateExec".to_string(), - }; - - case.run().await?; - Ok(()) -} - -#[tokio::test] -async fn test_window_agg_hash_partition() -> Result<()> { - let test1 = UnaryTestCase { - source_type: SourceType::Bounded, - expect_fail: false, - }; - let test2 = UnaryTestCase { - source_type: SourceType::Unbounded, - expect_fail: true, - }; - let case = QueryCase { - sql: "SELECT - c9, - SUM(c9) OVER(PARTITION BY c1 ORDER BY c9 ASC ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING) as sum1 - FROM test - LIMIT 5".to_string(), - cases: vec![Arc::new(test1), Arc::new(test2)], - error_operator: "operator: SortExec".to_string() - }; - - case.run().await?; - Ok(()) -} - -#[tokio::test] -async fn test_window_agg_single_partition() -> Result<()> { - let test1 = UnaryTestCase { - source_type: SourceType::Bounded, - expect_fail: false, - }; - let test2 = UnaryTestCase { - source_type: SourceType::Unbounded, - expect_fail: true, - }; - let case = QueryCase { - sql: "SELECT - c9, - SUM(c9) OVER(ORDER BY c9 ASC ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING) as sum1 - FROM test".to_string(), - cases: vec![Arc::new(test1), Arc::new(test2)], - error_operator: "operator: SortExec".to_string() - }; - case.run().await?; - Ok(()) -} - -#[tokio::test] -async fn test_hash_cross_join() -> Result<()> { - let test1 = BinaryTestCase { - source_types: (SourceType::Unbounded, SourceType::Bounded), - expect_fail: true, - }; - let test2 = BinaryTestCase { - source_types: (SourceType::Unbounded, SourceType::Unbounded), - expect_fail: true, - }; - let test3 = BinaryTestCase { - source_types: (SourceType::Bounded, SourceType::Unbounded), - expect_fail: true, - }; - let test4 = BinaryTestCase { - source_types: (SourceType::Bounded, SourceType::Bounded), - expect_fail: false, - }; - let case = QueryCase { - sql: "SELECT t2.c1 FROM left as t1 CROSS JOIN right as t2".to_string(), - cases: vec![ - Arc::new(test1), - Arc::new(test2), - Arc::new(test3), - Arc::new(test4), - ], - error_operator: "operator: CrossJoinExec".to_string(), - }; - - case.run().await?; - Ok(()) -} - -#[tokio::test] -async fn test_analyzer() -> Result<()> { - let test1 = UnaryTestCase { - source_type: SourceType::Bounded, - expect_fail: false, - }; - let test2 = UnaryTestCase { - source_type: SourceType::Unbounded, - expect_fail: false, - }; - let case = QueryCase { - sql: "EXPLAIN ANALYZE SELECT * FROM test".to_string(), - cases: vec![Arc::new(test1), Arc::new(test2)], - error_operator: "Analyze Error".to_string(), - }; - - case.run().await?; - Ok(()) -} - -#[tokio::test] -/// Tests that plan is valid when the sort requirements are satisfied. -async fn test_bounded_window_agg_sort_requirement() -> Result<()> { - let schema = create_test_schema(); - let source = memory_exec(&schema); - let sort_exprs = vec![sort_expr_options( - "c9", - &source.schema(), - SortOptions { - descending: false, - nulls_first: false, - }, - )]; - let sort = sort_exec(sort_exprs.clone(), source); - let bw = bounded_window_exec("c9", sort_exprs, sort); - assert_plan(bw.as_ref(), vec![ - "BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", - " SortExec: expr=[c9@0 ASC NULLS LAST], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]" - ]); - assert_sanity_check(&bw, true); - Ok(()) -} - -#[tokio::test] -/// Tests that plan is invalid when the sort requirements are not satisfied. -async fn test_bounded_window_agg_no_sort_requirement() -> Result<()> { - let schema = create_test_schema(); - let source = memory_exec(&schema); - let sort_exprs = vec![sort_expr_options( - "c9", - &source.schema(), - SortOptions { - descending: false, - nulls_first: false, - }, - )]; - let bw = bounded_window_exec("c9", sort_exprs, source); - assert_plan(bw.as_ref(), vec![ - "BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", - " MemoryExec: partitions=1, partition_sizes=[0]" - ]); - // Order requirement of the `BoundedWindowAggExec` is not satisfied. We expect to receive error during sanity check. - assert_sanity_check(&bw, false); - Ok(()) -} - -#[tokio::test] -/// A valid when a single partition requirement -/// is satisfied. -async fn test_global_limit_single_partition() -> Result<()> { - let schema = create_test_schema(); - let source = memory_exec(&schema); - let limit = global_limit_exec(source); - - assert_plan( - limit.as_ref(), - vec![ - "GlobalLimitExec: skip=0, fetch=100", - " MemoryExec: partitions=1, partition_sizes=[0]", - ], - ); - assert_sanity_check(&limit, true); - Ok(()) -} - -#[tokio::test] -/// An invalid plan when a single partition requirement -/// is not satisfied. -async fn test_global_limit_multi_partition() -> Result<()> { - let schema = create_test_schema(); - let source = memory_exec(&schema); - let limit = global_limit_exec(repartition_exec(source)); - - assert_plan( - limit.as_ref(), - vec![ - "GlobalLimitExec: skip=0, fetch=100", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[0]", - ], - ); - // Distribution requirement of the `GlobalLimitExec` is not satisfied. We expect to receive error during sanity check. - assert_sanity_check(&limit, false); - Ok(()) -} - -#[tokio::test] -/// A plan with no requirements should satisfy. -async fn test_local_limit() -> Result<()> { - let schema = create_test_schema(); - let source = memory_exec(&schema); - let limit = local_limit_exec(source); - - assert_plan( - limit.as_ref(), - vec![ - "LocalLimitExec: fetch=100", - " MemoryExec: partitions=1, partition_sizes=[0]", - ], - ); - assert_sanity_check(&limit, true); - Ok(()) -} - -#[tokio::test] -/// Valid plan with multiple children satisfy both order and distribution. -async fn test_sort_merge_join_satisfied() -> Result<()> { - let schema1 = create_test_schema(); - let schema2 = create_test_schema2(); - let source1 = memory_exec(&schema1); - let source2 = memory_exec(&schema2); - let sort_opts = SortOptions::default(); - let sort_exprs1 = vec![sort_expr_options("c9", &source1.schema(), sort_opts)]; - let sort_exprs2 = vec![sort_expr_options("a", &source2.schema(), sort_opts)]; - let left = sort_exec(sort_exprs1, source1); - let right = sort_exec(sort_exprs2, source2); - let left_jcol = col("c9", &left.schema()).unwrap(); - let right_jcol = col("a", &right.schema()).unwrap(); - let left = Arc::new(RepartitionExec::try_new( - left, - Partitioning::Hash(vec![left_jcol.clone()], 10), - )?); - - let right = Arc::new(RepartitionExec::try_new( - right, - Partitioning::Hash(vec![right_jcol.clone()], 10), - )?); - - let join_on = vec![(left_jcol as _, right_jcol as _)]; - let join_ty = JoinType::Inner; - let smj = sort_merge_join_exec(left, right, &join_on, &join_ty); - - assert_plan( - smj.as_ref(), - vec![ - "SortMergeJoin: join_type=Inner, on=[(c9@0, a@0)]", - " RepartitionExec: partitioning=Hash([c9@0], 10), input_partitions=1", - " SortExec: expr=[c9@0 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", - " RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=1", - " SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", - ], - ); - assert_sanity_check(&smj, true); - Ok(()) -} - -#[tokio::test] -/// Invalid case when the order is not satisfied by the 2nd -/// child. -async fn test_sort_merge_join_order_missing() -> Result<()> { - let schema1 = create_test_schema(); - let schema2 = create_test_schema2(); - let source1 = memory_exec(&schema1); - let right = memory_exec(&schema2); - let sort_exprs1 = vec![sort_expr_options( - "c9", - &source1.schema(), - SortOptions::default(), - )]; - let left = sort_exec(sort_exprs1, source1); - // Missing sort of the right child here.. - let left_jcol = col("c9", &left.schema()).unwrap(); - let right_jcol = col("a", &right.schema()).unwrap(); - let left = Arc::new(RepartitionExec::try_new( - left, - Partitioning::Hash(vec![left_jcol.clone()], 10), - )?); - - let right = Arc::new(RepartitionExec::try_new( - right, - Partitioning::Hash(vec![right_jcol.clone()], 10), - )?); - - let join_on = vec![(left_jcol as _, right_jcol as _)]; - let join_ty = JoinType::Inner; - let smj = sort_merge_join_exec(left, right, &join_on, &join_ty); - - assert_plan( - smj.as_ref(), - vec![ - "SortMergeJoin: join_type=Inner, on=[(c9@0, a@0)]", - " RepartitionExec: partitioning=Hash([c9@0], 10), input_partitions=1", - " SortExec: expr=[c9@0 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", - " RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[0]", - ], - ); - // Order requirement for the `SortMergeJoin` is not satisfied for right child. We expect to receive error during sanity check. - assert_sanity_check(&smj, false); - Ok(()) -} - -#[tokio::test] -/// Invalid case when the distribution is not satisfied by the 2nd -/// child. -async fn test_sort_merge_join_dist_missing() -> Result<()> { - let schema1 = create_test_schema(); - let schema2 = create_test_schema2(); - let source1 = memory_exec(&schema1); - let source2 = memory_exec(&schema2); - let sort_opts = SortOptions::default(); - let sort_exprs1 = vec![sort_expr_options("c9", &source1.schema(), sort_opts)]; - let sort_exprs2 = vec![sort_expr_options("a", &source2.schema(), sort_opts)]; - let left = sort_exec(sort_exprs1, source1); - let right = sort_exec(sort_exprs2, source2); - let right = Arc::new(RepartitionExec::try_new( - right, - Partitioning::RoundRobinBatch(10), - )?); - let left_jcol = col("c9", &left.schema()).unwrap(); - let right_jcol = col("a", &right.schema()).unwrap(); - let left = Arc::new(RepartitionExec::try_new( - left, - Partitioning::Hash(vec![left_jcol.clone()], 10), - )?); - - // Missing hash partitioning on right child. - - let join_on = vec![(left_jcol as _, right_jcol as _)]; - let join_ty = JoinType::Inner; - let smj = sort_merge_join_exec(left, right, &join_on, &join_ty); - - assert_plan( - smj.as_ref(), - vec![ - "SortMergeJoin: join_type=Inner, on=[(c9@0, a@0)]", - " RepartitionExec: partitioning=Hash([c9@0], 10), input_partitions=1", - " SortExec: expr=[c9@0 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", - " MemoryExec: partitions=1, partition_sizes=[0]", - ], - ); - // Distribution requirement for the `SortMergeJoin` is not satisfied for right child (has round-robin partitioning). We expect to receive error during sanity check. - assert_sanity_check(&smj, false); - Ok(()) +async fn register_current_csv( + ctx: &SessionContext, + table_name: &str, + infinite: bool, +) -> Result<()> { + let testdata = datafusion::test_util::arrow_test_data(); + let schema = datafusion::test_util::aggr_test_schema(); + let path = format!("{testdata}/csv/aggregate_test_100.csv"); + + match infinite { + true => { + let source = FileStreamProvider::new_file(schema, path.into()); + let config = StreamConfig::new(Arc::new(source)); + ctx.register_table(table_name, Arc::new(StreamTable::new(Arc::new(config))))?; + } + false => { + ctx.register_csv(table_name, &path, CsvReadOptions::new().schema(&schema)) + .await?; + } + } + + Ok(()) +} + +#[derive(Eq, PartialEq, Debug)] +pub enum SourceType { + Unbounded, + Bounded, +} + +#[async_trait] +pub trait SqlTestCase { + async fn register_table(&self, ctx: &SessionContext) -> Result<()>; + fn expect_fail(&self) -> bool; +} + +/// [UnaryTestCase] is designed for single input [ExecutionPlan]s. +pub struct UnaryTestCase { + pub source_type: SourceType, + pub expect_fail: bool, +} + +#[async_trait] +impl SqlTestCase for UnaryTestCase { + async fn register_table(&self, ctx: &SessionContext) -> Result<()> { + let table_is_infinite = self.source_type == SourceType::Unbounded; + register_current_csv(ctx, "test", table_is_infinite).await?; + Ok(()) + } + + fn expect_fail(&self) -> bool { + self.expect_fail + } +} +/// [BinaryTestCase] is designed for binary input [ExecutionPlan]s. +pub struct BinaryTestCase { + pub source_types: (SourceType, SourceType), + pub expect_fail: bool, +} + +#[async_trait] +impl SqlTestCase for BinaryTestCase { + async fn register_table(&self, ctx: &SessionContext) -> Result<()> { + let left_table_is_infinite = self.source_types.0 == SourceType::Unbounded; + let right_table_is_infinite = self.source_types.1 == SourceType::Unbounded; + register_current_csv(ctx, "left", left_table_is_infinite).await?; + register_current_csv(ctx, "right", right_table_is_infinite).await?; + Ok(()) + } + + fn expect_fail(&self) -> bool { + self.expect_fail + } +} + +pub struct QueryCase { + pub sql: String, + pub cases: Vec>, + pub error_operator: String, +} + +impl QueryCase { + /// Run the test cases + pub async fn run(&self) -> Result<()> { + for case in &self.cases { + let ctx = SessionContext::new(); + case.register_table(&ctx).await?; + let error = if case.expect_fail() { + Some(&self.error_operator) + } else { + None + }; + self.run_case(ctx, error).await?; + } + Ok(()) + } + async fn run_case(&self, ctx: SessionContext, error: Option<&String>) -> Result<()> { + let dataframe = ctx.sql(self.sql.as_str()).await?; + let plan = dataframe.create_physical_plan().await; + if let Some(error) = error { + let plan_error = plan.unwrap_err(); + assert!( + plan_error.to_string().contains(error.as_str()), + "plan_error: {:?} doesn't contain message: {:?}", + plan_error, + error.as_str() + ); + } else { + assert!(plan.is_ok()) + } + Ok(()) + } } diff --git a/datafusion/core/tests/physical_optimizer/test_util.rs b/datafusion/core/tests/physical_optimizer/test_util.rs deleted file mode 100644 index 3c3da36199b1..000000000000 --- a/datafusion/core/tests/physical_optimizer/test_util.rs +++ /dev/null @@ -1,319 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -//! Test utilities for physical optimizer tests - -use std::sync::Arc; - -use async_trait::async_trait; -use datafusion::datasource::stream::{FileStreamProvider, StreamConfig, StreamTable}; -use datafusion::error::Result; -use datafusion::prelude::{CsvReadOptions, SessionContext}; - -use arrow_schema::{DataType, Field, Schema, SchemaRef}; -use datafusion::datasource::file_format::file_compression_type::FileCompressionType; -use datafusion::datasource::physical_plan::CsvExec; -use datafusion::datasource::{ - listing::PartitionedFile, - physical_plan::{FileScanConfig, ParquetExec}, -}; -use datafusion_execution::object_store::ObjectStoreUrl; -use datafusion_execution::{SendableRecordBatchStream, TaskContext}; -use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; -use datafusion_physical_plan::streaming::{PartitionStream, StreamingTableExec}; -use datafusion_physical_plan::ExecutionPlan; - -/// create a single parquet file that is sorted -pub(crate) fn parquet_exec_with_sort( - output_ordering: Vec, -) -> Arc { - ParquetExec::builder( - FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema()) - .with_file(PartitionedFile::new("x".to_string(), 100)) - .with_output_ordering(output_ordering), - ) - .build_arc() -} - -pub(crate) fn schema() -> SchemaRef { - Arc::new(Schema::new(vec![ - Field::new("a", DataType::Int64, true), - Field::new("b", DataType::Int64, true), - Field::new("c", DataType::Int64, true), - Field::new("d", DataType::Int32, true), - Field::new("e", DataType::Boolean, true), - ])) -} - -pub(crate) fn trim_plan_display(plan: &str) -> Vec<&str> { - plan.split('\n') - .map(|s| s.trim()) - .filter(|s| !s.is_empty()) - .collect() -} - -async fn register_current_csv( - ctx: &SessionContext, - table_name: &str, - infinite: bool, -) -> Result<()> { - let testdata = datafusion::test_util::arrow_test_data(); - let schema = datafusion::test_util::aggr_test_schema(); - let path = format!("{testdata}/csv/aggregate_test_100.csv"); - - match infinite { - true => { - let source = FileStreamProvider::new_file(schema, path.into()); - let config = StreamConfig::new(Arc::new(source)); - ctx.register_table(table_name, Arc::new(StreamTable::new(Arc::new(config))))?; - } - false => { - ctx.register_csv(table_name, &path, CsvReadOptions::new().schema(&schema)) - .await?; - } - } - - Ok(()) -} - -#[derive(Eq, PartialEq, Debug)] -pub enum SourceType { - Unbounded, - Bounded, -} - -#[async_trait] -pub trait SqlTestCase { - async fn register_table(&self, ctx: &SessionContext) -> Result<()>; - fn expect_fail(&self) -> bool; -} - -/// [UnaryTestCase] is designed for single input [ExecutionPlan]s. -pub struct UnaryTestCase { - pub source_type: SourceType, - pub expect_fail: bool, -} - -#[async_trait] -impl SqlTestCase for UnaryTestCase { - async fn register_table(&self, ctx: &SessionContext) -> Result<()> { - let table_is_infinite = self.source_type == SourceType::Unbounded; - register_current_csv(ctx, "test", table_is_infinite).await?; - Ok(()) - } - - fn expect_fail(&self) -> bool { - self.expect_fail - } -} -/// [BinaryTestCase] is designed for binary input [ExecutionPlan]s. -pub struct BinaryTestCase { - pub source_types: (SourceType, SourceType), - pub expect_fail: bool, -} - -#[async_trait] -impl SqlTestCase for BinaryTestCase { - async fn register_table(&self, ctx: &SessionContext) -> Result<()> { - let left_table_is_infinite = self.source_types.0 == SourceType::Unbounded; - let right_table_is_infinite = self.source_types.1 == SourceType::Unbounded; - register_current_csv(ctx, "left", left_table_is_infinite).await?; - register_current_csv(ctx, "right", right_table_is_infinite).await?; - Ok(()) - } - - fn expect_fail(&self) -> bool { - self.expect_fail - } -} - -pub struct QueryCase { - pub sql: String, - pub cases: Vec>, - pub error_operator: String, -} - -impl QueryCase { - /// Run the test cases - pub async fn run(&self) -> Result<()> { - for case in &self.cases { - let ctx = SessionContext::new(); - case.register_table(&ctx).await?; - let error = if case.expect_fail() { - Some(&self.error_operator) - } else { - None - }; - self.run_case(ctx, error).await?; - } - Ok(()) - } - async fn run_case(&self, ctx: SessionContext, error: Option<&String>) -> Result<()> { - let dataframe = ctx.sql(self.sql.as_str()).await?; - let plan = dataframe.create_physical_plan().await; - if let Some(error) = error { - let plan_error = plan.unwrap_err(); - assert!( - plan_error.to_string().contains(error.as_str()), - "plan_error: {:?} doesn't contain message: {:?}", - plan_error, - error.as_str() - ); - } else { - assert!(plan.is_ok()) - } - Ok(()) - } -} - -/// Create a non sorted parquet exec -pub fn parquet_exec(schema: &SchemaRef) -> Arc { - ParquetExec::builder( - FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema.clone()) - .with_file(PartitionedFile::new("x".to_string(), 100)), - ) - .build_arc() -} - -// Created a sorted parquet exec -pub fn parquet_exec_sorted( - schema: &SchemaRef, - sort_exprs: impl IntoIterator, -) -> Arc { - let sort_exprs = sort_exprs.into_iter().collect(); - - ParquetExec::builder( - FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema.clone()) - .with_file(PartitionedFile::new("x".to_string(), 100)) - .with_output_ordering(vec![sort_exprs]), - ) - .build_arc() -} - -/// Created a sorted Csv exec -pub fn csv_exec_sorted( - schema: &SchemaRef, - sort_exprs: impl IntoIterator, -) -> Arc { - let sort_exprs = sort_exprs.into_iter().collect(); - - Arc::new( - CsvExec::builder( - FileScanConfig::new( - ObjectStoreUrl::parse("test:///").unwrap(), - schema.clone(), - ) - .with_file(PartitionedFile::new("x".to_string(), 100)) - .with_output_ordering(vec![sort_exprs]), - ) - .with_has_header(false) - .with_delimeter(0) - .with_quote(0) - .with_escape(None) - .with_comment(None) - .with_newlines_in_values(false) - .with_file_compression_type(FileCompressionType::UNCOMPRESSED) - .build(), - ) -} - -// construct a stream partition for test purposes -#[derive(Debug)] -pub(crate) struct TestStreamPartition { - pub schema: SchemaRef, -} - -impl PartitionStream for TestStreamPartition { - fn schema(&self) -> &SchemaRef { - &self.schema - } - fn execute(&self, _ctx: Arc) -> SendableRecordBatchStream { - unreachable!() - } -} - -/// Create an unbounded stream exec -pub fn stream_exec_ordered( - schema: &SchemaRef, - sort_exprs: impl IntoIterator, -) -> Arc { - let sort_exprs = sort_exprs.into_iter().collect(); - - Arc::new( - StreamingTableExec::try_new( - schema.clone(), - vec![Arc::new(TestStreamPartition { - schema: schema.clone(), - }) as _], - None, - vec![sort_exprs], - true, - None, - ) - .unwrap(), - ) -} - -// Creates a stream exec source for the test purposes -pub fn stream_exec_ordered_with_projection( - schema: &SchemaRef, - sort_exprs: impl IntoIterator, -) -> Arc { - let sort_exprs = sort_exprs.into_iter().collect(); - let projection: Vec = vec![0, 2, 3]; - - Arc::new( - StreamingTableExec::try_new( - schema.clone(), - vec![Arc::new(TestStreamPartition { - schema: schema.clone(), - }) as _], - Some(&projection), - vec![sort_exprs], - true, - None, - ) - .unwrap(), - ) -} - -/// Create a csv exec for tests -pub fn csv_exec_ordered( - schema: &SchemaRef, - sort_exprs: impl IntoIterator, -) -> Arc { - let sort_exprs = sort_exprs.into_iter().collect(); - - Arc::new( - CsvExec::builder( - FileScanConfig::new( - ObjectStoreUrl::parse("test:///").unwrap(), - schema.clone(), - ) - .with_file(PartitionedFile::new("file_path".to_string(), 100)) - .with_output_ordering(vec![sort_exprs]), - ) - .with_has_header(true) - .with_delimeter(0) - .with_quote(b'"') - .with_escape(None) - .with_comment(None) - .with_newlines_in_values(false) - .with_file_compression_type(FileCompressionType::UNCOMPRESSED) - .build(), - ) -} diff --git a/datafusion/physical-optimizer/src/lib.rs b/datafusion/physical-optimizer/src/lib.rs index d807fb220956..68eb55228a19 100644 --- a/datafusion/physical-optimizer/src/lib.rs +++ b/datafusion/physical-optimizer/src/lib.rs @@ -30,7 +30,6 @@ mod optimizer; pub mod output_requirements; pub mod pruning; pub mod sanity_checker; -pub mod test_utils; pub mod topk_aggregation; pub mod update_aggr_exprs; pub mod utils; diff --git a/datafusion/physical-optimizer/tests/enforce_sorting.rs b/datafusion/physical-optimizer/tests/enforce_sorting.rs new file mode 100644 index 000000000000..0b6ed1398de7 --- /dev/null +++ b/datafusion/physical-optimizer/tests/enforce_sorting.rs @@ -0,0 +1,1154 @@ +// 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_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 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::config::ConfigOptions; +use datafusion_common::tree_node::{TreeNode, TransformedResult}; +use crate::test_utils::{aggregate_exec, bounded_window_exec, coalesce_batches_exec, filter_exec, hash_join_exec, memory_exec, repartition_exec, sort_exec, sort_expr, sort_expr_options, sort_preserving_merge_exec, union_exec}; + +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 mut config = ConfigOptions::new(); + config.optimizer.repartition_sorts = $REPARTITION_SORTS; + + // This file has 4 rules that use tree node, apply these rules as in the + // EnforceSorting::optimize implementation + // After these operations tree nodes should be in a consistent state. + // This code block makes sure that these rules doesn't violate tree node integrity. + { + let plan_requirements = PlanWithCorrespondingSort::new_default($PLAN.clone()); + let adjusted = plan_requirements + .transform_up(ensure_sorting) + .data() + .and_then(check_integrity)?; + // TODO: End state payloads will be checked here. + + let new_plan = if repartition_sorts.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, + repartition_sorts, + ) + }) + .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, repartition_sorts)?; + + // Get string representation of the plan + let actual = get_plan_string(&optimized_physical_plan); + assert_eq!( + expected_optimized_lines, actual, + "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected_optimized_lines:#?}\nactual:\n\n{actual:#?}\n\n" + ); + + }; +} + +#[tokio::test] +async fn test_remove_unnecessary_sort() -> Result<()> { + let schema = create_test_schema()?; + let source = memory_exec(&schema); + let input = sort_exec(vec![sort_expr("non_nullable_col", &schema)], source); + let physical_plan = sort_exec(vec![sort_expr("nullable_col", &schema)], input); + + let expected_input = [ + "SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", + " SortExec: expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + let expected_optimized = [ + "SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + + Ok(()) +} + +#[tokio::test] +async fn test_remove_unnecessary_sort_window_multilayer() -> Result<()> { + let schema = create_test_schema()?; + let source = memory_exec(&schema); + + let sort_exprs = vec![sort_expr_options( + "non_nullable_col", + &source.schema(), + SortOptions { + descending: true, + nulls_first: true, + }, + )]; + let sort = sort_exec(sort_exprs.clone(), source); + // Add dummy layer propagating Sort above, to test whether sort can be removed from multi layer before + let coalesce_batches = coalesce_batches_exec(sort); + + let window_agg = + bounded_window_exec("non_nullable_col", sort_exprs, coalesce_batches); + + let sort_exprs = vec![sort_expr_options( + "non_nullable_col", + &window_agg.schema(), + SortOptions { + descending: false, + nulls_first: false, + }, + )]; + + let sort = sort_exec(sort_exprs.clone(), window_agg); + + // Add dummy layer propagating Sort above, to test whether sort can be removed from multi layer before + let filter = filter_exec( + Arc::new(NotExpr::new( + col("non_nullable_col", schema.as_ref()).unwrap(), + )), + sort, + ); + + let physical_plan = bounded_window_exec("non_nullable_col", sort_exprs, filter); + + let expected_input = ["BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", + " FilterExec: NOT non_nullable_col@1", + " SortExec: expr=[non_nullable_col@1 ASC NULLS LAST], preserve_partitioning=[false]", + " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", + " CoalesceBatchesExec: target_batch_size=128", + " SortExec: expr=[non_nullable_col@1 DESC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]"]; + + let expected_optimized = ["WindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(NULL), is_causal: false }]", + " FilterExec: NOT non_nullable_col@1", + " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", + " CoalesceBatchesExec: target_batch_size=128", + " SortExec: expr=[non_nullable_col@1 DESC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]"]; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + + Ok(()) +} + +#[tokio::test] +async fn test_add_required_sort() -> Result<()> { + let schema = create_test_schema()?; + let source = memory_exec(&schema); + + let sort_exprs = vec![sort_expr("nullable_col", &schema)]; + + let physical_plan = sort_preserving_merge_exec(sort_exprs, source); + + let expected_input = [ + "SortPreservingMergeExec: [nullable_col@0 ASC]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + let expected_optimized = [ + "SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + + Ok(()) +} + +#[tokio::test] +async fn test_remove_unnecessary_sort1() -> Result<()> { + let schema = create_test_schema()?; + let source = memory_exec(&schema); + let sort_exprs = vec![sort_expr("nullable_col", &schema)]; + let sort = sort_exec(sort_exprs.clone(), source); + let spm = sort_preserving_merge_exec(sort_exprs, sort); + + let sort_exprs = vec![sort_expr("nullable_col", &schema)]; + let sort = sort_exec(sort_exprs.clone(), spm); + let physical_plan = sort_preserving_merge_exec(sort_exprs, sort); + let expected_input = [ + "SortPreservingMergeExec: [nullable_col@0 ASC]", + " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", + " SortPreservingMergeExec: [nullable_col@0 ASC]", + " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + let expected_optimized = [ + "SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + + Ok(()) +} + +#[tokio::test] +async fn test_remove_unnecessary_sort2() -> Result<()> { + let schema = create_test_schema()?; + let source = memory_exec(&schema); + let sort_exprs = vec![sort_expr("non_nullable_col", &schema)]; + let sort = sort_exec(sort_exprs.clone(), source); + let spm = sort_preserving_merge_exec(sort_exprs, sort); + + let sort_exprs = vec![ + sort_expr("nullable_col", &schema), + sort_expr("non_nullable_col", &schema), + ]; + let sort2 = sort_exec(sort_exprs.clone(), spm); + let spm2 = sort_preserving_merge_exec(sort_exprs, sort2); + + let sort_exprs = vec![sort_expr("nullable_col", &schema)]; + let sort3 = sort_exec(sort_exprs, spm2); + let physical_plan = repartition_exec(repartition_exec(sort3)); + + let expected_input = [ + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", + " SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", + " SortPreservingMergeExec: [non_nullable_col@1 ASC]", + " SortExec: expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + + let expected_optimized = [ + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + + Ok(()) +} + +#[tokio::test] +async fn test_remove_unnecessary_sort3() -> Result<()> { + let schema = create_test_schema()?; + let source = memory_exec(&schema); + let sort_exprs = vec![sort_expr("non_nullable_col", &schema)]; + let sort = sort_exec(sort_exprs.clone(), source); + let spm = sort_preserving_merge_exec(sort_exprs, sort); + + let sort_exprs = LexOrdering::new(vec![ + sort_expr("nullable_col", &schema), + sort_expr("non_nullable_col", &schema), + ]); + let repartition_exec = repartition_exec(spm); + let sort2 = Arc::new( + SortExec::new(sort_exprs.clone(), repartition_exec) + .with_preserve_partitioning(true), + ) as _; + let spm2 = sort_preserving_merge_exec(sort_exprs, sort2); + + let physical_plan = aggregate_exec(spm2); + + // When removing a `SortPreservingMergeExec`, make sure that partitioning + // requirements are not violated. In some cases, we may need to replace + // it with a `CoalescePartitionsExec` instead of directly removing it. + let expected_input = [ + "AggregateExec: mode=Final, gby=[], aggr=[]", + " SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[true]", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " SortPreservingMergeExec: [non_nullable_col@1 ASC]", + " SortExec: expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + + let expected_optimized = [ + "AggregateExec: mode=Final, gby=[], aggr=[]", + " CoalescePartitionsExec", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + + Ok(()) +} + +#[tokio::test] +async fn test_remove_unnecessary_sort4() -> Result<()> { + let schema = create_test_schema()?; + let source1 = repartition_exec(memory_exec(&schema)); + + let source2 = repartition_exec(memory_exec(&schema)); + let union = union_exec(vec![source1, source2]); + + let sort_exprs = LexOrdering::new(vec![sort_expr("non_nullable_col", &schema)]); + // let sort = sort_exec(sort_exprs.clone(), union); + let sort = Arc::new( + SortExec::new(sort_exprs.clone(), union).with_preserve_partitioning(true), + ) as _; + let spm = sort_preserving_merge_exec(sort_exprs, sort); + + let filter = filter_exec( + Arc::new(NotExpr::new( + col("non_nullable_col", schema.as_ref()).unwrap(), + )), + spm, + ); + + let sort_exprs = vec![ + sort_expr("nullable_col", &schema), + sort_expr("non_nullable_col", &schema), + ]; + let physical_plan = sort_exec(sort_exprs, filter); + + // When removing a `SortPreservingMergeExec`, make sure that partitioning + // requirements are not violated. In some cases, we may need to replace + // it with a `CoalescePartitionsExec` instead of directly removing it. + let expected_input = ["SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", + " FilterExec: NOT non_nullable_col@1", + " SortPreservingMergeExec: [non_nullable_col@1 ASC]", + " SortExec: expr=[non_nullable_col@1 ASC], preserve_partitioning=[true]", + " UnionExec", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " MemoryExec: partitions=1, partition_sizes=[0]", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " MemoryExec: partitions=1, partition_sizes=[0]"]; + + let expected_optimized = ["SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[true]", + " FilterExec: NOT non_nullable_col@1", + " UnionExec", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " MemoryExec: partitions=1, partition_sizes=[0]", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " MemoryExec: partitions=1, partition_sizes=[0]"]; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + + Ok(()) +} + +#[tokio::test] +async fn test_remove_unnecessary_sort6() -> Result<()> { + let schema = create_test_schema()?; + let source = memory_exec(&schema); + let input = Arc::new( + SortExec::new( + LexOrdering::new(vec![sort_expr("non_nullable_col", &schema)]), + source, + ) + .with_fetch(Some(2)), + ); + let physical_plan = sort_exec( + vec![ + sort_expr("non_nullable_col", &schema), + sort_expr("nullable_col", &schema), + ], + input, + ); + + let expected_input = [ + "SortExec: expr=[non_nullable_col@1 ASC, nullable_col@0 ASC], preserve_partitioning=[false]", + " SortExec: TopK(fetch=2), expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + let expected_optimized = [ + "SortExec: TopK(fetch=2), expr=[non_nullable_col@1 ASC, nullable_col@0 ASC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + + Ok(()) +} + +#[tokio::test] +async fn test_remove_unnecessary_sort7() -> Result<()> { + let schema = create_test_schema()?; + let source = memory_exec(&schema); + let input = Arc::new(SortExec::new( + LexOrdering::new(vec![ + sort_expr("non_nullable_col", &schema), + sort_expr("nullable_col", &schema), + ]), + source, + )); + + let physical_plan = Arc::new( + SortExec::new( + LexOrdering::new(vec![sort_expr("non_nullable_col", &schema)]), + input, + ) + .with_fetch(Some(2)), + ) as Arc; + + let expected_input = [ + "SortExec: TopK(fetch=2), expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", + " SortExec: expr=[non_nullable_col@1 ASC, nullable_col@0 ASC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + let expected_optimized = [ + "GlobalLimitExec: skip=0, fetch=2", + " SortExec: expr=[non_nullable_col@1 ASC, nullable_col@0 ASC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + + Ok(()) +} + +#[tokio::test] +async fn test_remove_unnecessary_sort8() -> Result<()> { + let schema = create_test_schema()?; + let source = memory_exec(&schema); + let input = Arc::new(SortExec::new( + LexOrdering::new(vec![sort_expr("non_nullable_col", &schema)]), + source, + )); + let limit = Arc::new(LocalLimitExec::new(input, 2)); + let physical_plan = sort_exec( + vec![ + sort_expr("non_nullable_col", &schema), + sort_expr("nullable_col", &schema), + ], + limit, + ); + + let expected_input = [ + "SortExec: expr=[non_nullable_col@1 ASC, nullable_col@0 ASC], preserve_partitioning=[false]", + " LocalLimitExec: fetch=2", + " SortExec: expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + let expected_optimized = [ + "LocalLimitExec: fetch=2", + " SortExec: TopK(fetch=2), expr=[non_nullable_col@1 ASC, nullable_col@0 ASC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + + Ok(()) +} + +#[tokio::test] +async fn test_do_not_pushdown_through_limit() -> Result<()> { + let schema = create_test_schema()?; + let source = memory_exec(&schema); + // let input = sort_exec(vec![sort_expr("non_nullable_col", &schema)], source); + let input = Arc::new(SortExec::new( + LexOrdering::new(vec![sort_expr("non_nullable_col", &schema)]), + source, + )); + let limit = Arc::new(GlobalLimitExec::new(input, 0, Some(5))) as _; + let physical_plan = sort_exec(vec![sort_expr("nullable_col", &schema)], limit); + + let expected_input = [ + "SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", + " GlobalLimitExec: skip=0, fetch=5", + " SortExec: expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + let expected_optimized = [ + "SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", + " GlobalLimitExec: skip=0, fetch=5", + " SortExec: expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + + Ok(()) +} + +#[tokio::test] +async fn test_remove_unnecessary_spm1() -> Result<()> { + let schema = create_test_schema()?; + let source = memory_exec(&schema); + let input = + sort_preserving_merge_exec(vec![sort_expr("non_nullable_col", &schema)], source); + let input2 = + sort_preserving_merge_exec(vec![sort_expr("non_nullable_col", &schema)], input); + let physical_plan = + sort_preserving_merge_exec(vec![sort_expr("nullable_col", &schema)], input2); + + let expected_input = [ + "SortPreservingMergeExec: [nullable_col@0 ASC]", + " SortPreservingMergeExec: [non_nullable_col@1 ASC]", + " SortPreservingMergeExec: [non_nullable_col@1 ASC]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + let expected_optimized = [ + "SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + + Ok(()) +} + +#[tokio::test] +async fn test_change_wrong_sorting() -> Result<()> { + let schema = create_test_schema()?; + let source = memory_exec(&schema); + let sort_exprs = vec![ + sort_expr("nullable_col", &schema), + sort_expr("non_nullable_col", &schema), + ]; + let sort = sort_exec(vec![sort_exprs[0].clone()], source); + let physical_plan = sort_preserving_merge_exec(sort_exprs, sort); + let expected_input = [ + "SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", + " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + let expected_optimized = [ + "SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + + Ok(()) +} + +#[tokio::test] +async fn test_change_wrong_sorting2() -> Result<()> { + let schema = create_test_schema()?; + let source = memory_exec(&schema); + let sort_exprs = vec![ + sort_expr("nullable_col", &schema), + sort_expr("non_nullable_col", &schema), + ]; + let spm1 = sort_preserving_merge_exec(sort_exprs.clone(), source); + let sort2 = sort_exec(vec![sort_exprs[0].clone()], spm1); + let physical_plan = sort_preserving_merge_exec(vec![sort_exprs[1].clone()], sort2); + + let expected_input = [ + "SortPreservingMergeExec: [non_nullable_col@1 ASC]", + " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", + " SortPreservingMergeExec: [nullable_col@0 ASC, non_nullable_col@1 ASC]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + let expected_optimized = [ + "SortExec: expr=[non_nullable_col@1 ASC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + + Ok(()) +} + +#[tokio::test] +async fn test_multiple_sort_window_exec() -> Result<()> { + let schema = create_test_schema()?; + let source = memory_exec(&schema); + + let sort_exprs1 = vec![sort_expr("nullable_col", &schema)]; + let sort_exprs2 = vec![ + sort_expr("nullable_col", &schema), + sort_expr("non_nullable_col", &schema), + ]; + + let sort1 = sort_exec(sort_exprs1.clone(), source); + let window_agg1 = bounded_window_exec("non_nullable_col", sort_exprs1.clone(), sort1); + let window_agg2 = bounded_window_exec("non_nullable_col", sort_exprs2, window_agg1); + // let filter_exec = sort_exec; + let physical_plan = bounded_window_exec("non_nullable_col", sort_exprs1, window_agg2); + + let expected_input = ["BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", + " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", + " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", + " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]"]; + + let expected_optimized = ["BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", + " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", + " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", + " SortExec: expr=[nullable_col@0 ASC, non_nullable_col@1 ASC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]"]; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + + Ok(()) +} + +#[tokio::test] +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 config = ConfigOptions::new(); + + let memory_exec = memory_exec(&schema); + let sort_exprs = LexOrdering::new(vec![sort_expr("nullable_col", &schema)]); + let window = bounded_window_exec("nullable_col", sort_exprs.clone(), memory_exec); + let repartition = repartition_exec(window); + + let orig_plan = + Arc::new(SortExec::new(sort_exprs, repartition)) as Arc; + let actual = get_plan_string(&orig_plan); + let expected_input = vec![ + "SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + assert_eq!( + expected_input, actual, + "\n**Original Plan Mismatch\n\nexpected:\n\n{expected_input:#?}\nactual:\n\n{actual:#?}\n\n" + ); + + let mut plan = orig_plan.clone(); + let rules = vec![ + Arc::new(EnforceDistribution::new()) as Arc, + Arc::new(EnforceSorting::new()) as Arc, + ]; + for rule in rules { + plan = rule.optimize(plan, config)?; + } + let first_plan = plan.clone(); + + let mut plan = orig_plan.clone(); + let rules = vec![ + Arc::new(EnforceSorting::new()) as Arc, + Arc::new(EnforceDistribution::new()) as Arc, + Arc::new(EnforceSorting::new()) as Arc, + ]; + for rule in rules { + plan = rule.optimize(plan, config)?; + } + let second_plan = plan.clone(); + + assert_eq!(get_plan_string(&first_plan), get_plan_string(&second_plan)); + Ok(()) +} + +#[tokio::test] +async fn test_coalesce_propagate() -> Result<()> { + let schema = create_test_schema()?; + let source = memory_exec(&schema); + let repartition = repartition_exec(source); + let coalesce_partitions = Arc::new(CoalescePartitionsExec::new(repartition)); + let repartition = repartition_exec(coalesce_partitions); + let sort_exprs = LexOrdering::new(vec![sort_expr("nullable_col", &schema)]); + // Add local sort + let sort = Arc::new( + SortExec::new(sort_exprs.clone(), repartition).with_preserve_partitioning(true), + ) as _; + let spm = sort_preserving_merge_exec(sort_exprs.clone(), sort); + let sort = sort_exec(sort_exprs, spm); + + let physical_plan = sort.clone(); + // Sort Parallelize rule should end Coalesce + Sort linkage when Sort is Global Sort + // Also input plan is not valid as it is. We need to add SortExec before SortPreservingMergeExec. + let expected_input = [ + "SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[false]", + " SortPreservingMergeExec: [nullable_col@0 ASC]", + " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[true]", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " CoalescePartitionsExec", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + let expected_optimized = [ + "SortPreservingMergeExec: [nullable_col@0 ASC]", + " SortExec: expr=[nullable_col@0 ASC], preserve_partitioning=[true]", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + + Ok(()) +} + +#[tokio::test] +async fn test_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/limited_distinct_aggregation.rs b/datafusion/physical-optimizer/tests/limited_distinct_aggregation.rs similarity index 97% rename from datafusion/core/tests/physical_optimizer/limited_distinct_aggregation.rs rename to datafusion/physical-optimizer/tests/limited_distinct_aggregation.rs index 46a56fe1fb75..576edb199b7e 100644 --- a/datafusion/core/tests/physical_optimizer/limited_distinct_aggregation.rs +++ b/datafusion/physical-optimizer/tests/limited_distinct_aggregation.rs @@ -19,7 +19,6 @@ //! //! Note these tests are not in the same module as the optimizer pass because //! they rely on `ParquetExec` which is in the core crate. -use super::test_util::{parquet_exec_with_sort, schema, trim_plan_display}; use std::sync::Arc; @@ -31,7 +30,7 @@ use arrow::{ util::pretty::pretty_format_batches, }; use arrow_schema::SchemaRef; -use datafusion::{prelude::SessionContext, test_util::TestAggregate}; +use datafusion_common::config::ConfigOptions; use datafusion_common::Result; use datafusion_execution::config::SessionConfig; use datafusion_expr::Operator; @@ -91,11 +90,10 @@ fn assert_plan_matches_expected( expected: &[&str], ) -> Result<()> { let expected_lines: Vec<&str> = expected.to_vec(); - let session_ctx = SessionContext::new(); - let state = session_ctx.state(); + let config = ConfigOptions::new(); - let optimized = LimitedDistinctAggregation::new() - .optimize(Arc::clone(plan), state.config_options())?; + let optimized = + LimitedDistinctAggregation::new().optimize(Arc::clone(plan), config)?; let optimized_result = displayable(optimized.as_ref()).indent(true).to_string(); let actual_lines = trim_plan_display(&optimized_result); diff --git a/datafusion/physical-optimizer/tests/mod.rs b/datafusion/physical-optimizer/tests/mod.rs new file mode 100644 index 000000000000..e2e89f5d438b --- /dev/null +++ b/datafusion/physical-optimizer/tests/mod.rs @@ -0,0 +1,22 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +mod enforce_sorting; +mod limited_distinct_aggregation; +mod replace_with_order_preserving_variants; +mod sanity_checker; +mod test_utils; diff --git a/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs b/datafusion/physical-optimizer/tests/replace_with_order_preserving_variants.rs similarity index 99% rename from datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs rename to datafusion/physical-optimizer/tests/replace_with_order_preserving_variants.rs index e102ab5041b7..b83c661d07d7 100644 --- a/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs +++ b/datafusion/physical-optimizer/tests/replace_with_order_preserving_variants.rs @@ -17,9 +17,11 @@ use std::sync::Arc; -use datafusion::prelude::{SessionConfig, SessionContext}; +use arrow::array::{ArrayRef, Int32Array}; +use arrow::compute::SortOptions; +use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; +use arrow::record_batch::RecordBatch; use datafusion_execution::TaskContext; - use datafusion_physical_plan::coalesce_batches::CoalesceBatchesExec; use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; use datafusion_physical_plan::collect; @@ -32,12 +34,6 @@ use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeE use datafusion_physical_plan::{ displayable, get_plan_string, ExecutionPlan, Partitioning, }; - -use arrow::array::{ArrayRef, Int32Array}; -use arrow::compute::SortOptions; -use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; -use arrow::record_batch::RecordBatch; - use datafusion_common::tree_node::{TransformedResult, TreeNode}; use datafusion_common::Result; use datafusion_expr::{JoinType, Operator}; @@ -46,10 +42,8 @@ use datafusion_common::tree_node::{TransformedResult, TreeNode}; 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 crate::test_util::stream_exec_ordered_with_projection; +use datafusion_common::config::ConfigOptions; use rstest::rstest; use url::Url; @@ -153,7 +147,8 @@ macro_rules! assert_optimized { let expected_optimized_lines: Vec<&str> = $EXPECTED_OPTIMIZED_PLAN_LINES.iter().map(|s| *s).collect(); // Run the rule top-down - let config = SessionConfig::new().with_prefer_existing_sort($PREFER_EXISTING_SORT); + let mut config = ConfigOptions::new(); + config.optimizer.prefer_existing_sort=$PREFER_EXISTING_SORT; let plan_with_pipeline_fixer = OrderPreservationContext::new_default(physical_plan); let parallel = plan_with_pipeline_fixer.transform_up(|plan_with_pipeline_fixer| replace_with_order_preserving_variants(plan_with_pipeline_fixer, false, false, config.options())).data().and_then(check_integrity)?; let optimized_physical_plan = parallel.plan; diff --git a/datafusion/physical-optimizer/tests/sanity_checker.rs b/datafusion/physical-optimizer/tests/sanity_checker.rs new file mode 100644 index 000000000000..538f0e443ddb --- /dev/null +++ b/datafusion/physical-optimizer/tests/sanity_checker.rs @@ -0,0 +1,536 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +//! Tests for [`SanityCheckPlan`] physical optimizer rule +//! +//! Note these tests are not in the same module as the optimizer pass because +//! they rely on `ParquetExec` which is in the core crate. + +use crate::physical_optimizer::test_util::{ + BinaryTestCase, QueryCase, SourceType, UnaryTestCase, +}; +use arrow::compute::SortOptions; +use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; +use datafusion_common::config::ConfigOptions; +use datafusion_common::Result; +use datafusion_expr::JoinType; +use datafusion_physical_expr::expressions::col; +use datafusion_physical_expr::Partitioning; +use datafusion_physical_optimizer::test_utils::{ + bounded_window_exec, global_limit_exec, local_limit_exec, memory_exec, + repartition_exec, sort_exec, sort_expr_options, sort_merge_join_exec, +}; +use datafusion_physical_optimizer::{sanity_checker::*, PhysicalOptimizerRule}; +use datafusion_physical_plan::displayable; +use datafusion_physical_plan::repartition::RepartitionExec; +use datafusion_physical_plan::ExecutionPlan; +use std::sync::Arc; + +fn create_test_schema() -> SchemaRef { + Arc::new(Schema::new(vec![Field::new("c9", DataType::Int32, true)])) +} + +fn create_test_schema2() -> SchemaRef { + Arc::new(Schema::new(vec![ + Field::new("a", DataType::Int32, true), + Field::new("b", DataType::Int32, true), + ])) +} + +/// Check if sanity checker should accept or reject plans. +fn assert_sanity_check(plan: &Arc, is_sane: bool) { + let sanity_checker = SanityCheckPlan::new(); + let opts = ConfigOptions::default(); + assert_eq!( + sanity_checker.optimize(plan.clone(), &opts).is_ok(), + is_sane + ); +} + +/// Check if the plan we created is as expected by comparing the plan +/// formatted as a string. +fn assert_plan(plan: &dyn ExecutionPlan, expected_lines: Vec<&str>) { + let plan_str = displayable(plan).indent(true).to_string(); + let actual_lines: Vec<&str> = plan_str.trim().lines().collect(); + assert_eq!(actual_lines, expected_lines); +} + +#[tokio::test] +async fn test_hash_left_join_swap() -> Result<()> { + let test1 = BinaryTestCase { + source_types: (SourceType::Unbounded, SourceType::Bounded), + expect_fail: false, + }; + + let test2 = BinaryTestCase { + source_types: (SourceType::Bounded, SourceType::Unbounded), + // Left join for bounded build side and unbounded probe side can generate + // both incremental matched rows and final non-matched rows. + expect_fail: false, + }; + let test3 = BinaryTestCase { + source_types: (SourceType::Bounded, SourceType::Bounded), + expect_fail: false, + }; + let case = QueryCase { + sql: "SELECT t2.c1 FROM left as t1 LEFT JOIN right as t2 ON t1.c1 = t2.c1" + .to_string(), + cases: vec![Arc::new(test1), Arc::new(test2), Arc::new(test3)], + error_operator: "operator: HashJoinExec".to_string(), + }; + + case.run().await?; + Ok(()) +} + +#[tokio::test] +async fn test_hash_right_join_swap() -> Result<()> { + let test1 = BinaryTestCase { + source_types: (SourceType::Unbounded, SourceType::Bounded), + expect_fail: true, + }; + let test2 = BinaryTestCase { + source_types: (SourceType::Bounded, SourceType::Unbounded), + expect_fail: false, + }; + let test3 = BinaryTestCase { + source_types: (SourceType::Bounded, SourceType::Bounded), + expect_fail: false, + }; + let case = QueryCase { + sql: "SELECT t2.c1 FROM left as t1 RIGHT JOIN right as t2 ON t1.c1 = t2.c1" + .to_string(), + cases: vec![Arc::new(test1), Arc::new(test2), Arc::new(test3)], + error_operator: "operator: HashJoinExec".to_string(), + }; + + case.run().await?; + Ok(()) +} + +#[tokio::test] +async fn test_hash_inner_join_swap() -> Result<()> { + let test1 = BinaryTestCase { + source_types: (SourceType::Unbounded, SourceType::Bounded), + expect_fail: false, + }; + let test2 = BinaryTestCase { + source_types: (SourceType::Bounded, SourceType::Unbounded), + expect_fail: false, + }; + let test3 = BinaryTestCase { + source_types: (SourceType::Bounded, SourceType::Bounded), + expect_fail: false, + }; + let case = QueryCase { + sql: "SELECT t2.c1 FROM left as t1 JOIN right as t2 ON t1.c1 = t2.c1".to_string(), + cases: vec![Arc::new(test1), Arc::new(test2), Arc::new(test3)], + error_operator: "Join Error".to_string(), + }; + + case.run().await?; + Ok(()) +} + +#[tokio::test] +async fn test_hash_full_outer_join_swap() -> Result<()> { + let test1 = BinaryTestCase { + source_types: (SourceType::Unbounded, SourceType::Bounded), + expect_fail: true, + }; + let test2 = BinaryTestCase { + source_types: (SourceType::Bounded, SourceType::Unbounded), + // Full join for bounded build side and unbounded probe side can generate + // both incremental matched rows and final non-matched rows. + expect_fail: false, + }; + let test3 = BinaryTestCase { + source_types: (SourceType::Bounded, SourceType::Bounded), + expect_fail: false, + }; + let case = QueryCase { + sql: "SELECT t2.c1 FROM left as t1 FULL JOIN right as t2 ON t1.c1 = t2.c1" + .to_string(), + cases: vec![Arc::new(test1), Arc::new(test2), Arc::new(test3)], + error_operator: "operator: HashJoinExec".to_string(), + }; + + case.run().await?; + Ok(()) +} + +#[tokio::test] +async fn test_aggregate() -> Result<()> { + let test1 = UnaryTestCase { + source_type: SourceType::Bounded, + expect_fail: false, + }; + let test2 = UnaryTestCase { + source_type: SourceType::Unbounded, + expect_fail: true, + }; + let case = QueryCase { + sql: "SELECT c1, MIN(c4) FROM test GROUP BY c1".to_string(), + cases: vec![Arc::new(test1), Arc::new(test2)], + error_operator: "operator: AggregateExec".to_string(), + }; + + case.run().await?; + Ok(()) +} + +#[tokio::test] +async fn test_window_agg_hash_partition() -> Result<()> { + let test1 = UnaryTestCase { + source_type: SourceType::Bounded, + expect_fail: false, + }; + let test2 = UnaryTestCase { + source_type: SourceType::Unbounded, + expect_fail: true, + }; + let case = QueryCase { + sql: "SELECT + c9, + SUM(c9) OVER(PARTITION BY c1 ORDER BY c9 ASC ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING) as sum1 + FROM test + LIMIT 5".to_string(), + cases: vec![Arc::new(test1), Arc::new(test2)], + error_operator: "operator: SortExec".to_string() + }; + + case.run().await?; + Ok(()) +} + +#[tokio::test] +async fn test_window_agg_single_partition() -> Result<()> { + let test1 = UnaryTestCase { + source_type: SourceType::Bounded, + expect_fail: false, + }; + let test2 = UnaryTestCase { + source_type: SourceType::Unbounded, + expect_fail: true, + }; + let case = QueryCase { + sql: "SELECT + c9, + SUM(c9) OVER(ORDER BY c9 ASC ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING) as sum1 + FROM test".to_string(), + cases: vec![Arc::new(test1), Arc::new(test2)], + error_operator: "operator: SortExec".to_string() + }; + case.run().await?; + Ok(()) +} + +#[tokio::test] +async fn test_hash_cross_join() -> Result<()> { + let test1 = BinaryTestCase { + source_types: (SourceType::Unbounded, SourceType::Bounded), + expect_fail: true, + }; + let test2 = BinaryTestCase { + source_types: (SourceType::Unbounded, SourceType::Unbounded), + expect_fail: true, + }; + let test3 = BinaryTestCase { + source_types: (SourceType::Bounded, SourceType::Unbounded), + expect_fail: true, + }; + let test4 = BinaryTestCase { + source_types: (SourceType::Bounded, SourceType::Bounded), + expect_fail: false, + }; + let case = QueryCase { + sql: "SELECT t2.c1 FROM left as t1 CROSS JOIN right as t2".to_string(), + cases: vec![ + Arc::new(test1), + Arc::new(test2), + Arc::new(test3), + Arc::new(test4), + ], + error_operator: "operator: CrossJoinExec".to_string(), + }; + + case.run().await?; + Ok(()) +} + +#[tokio::test] +async fn test_analyzer() -> Result<()> { + let test1 = UnaryTestCase { + source_type: SourceType::Bounded, + expect_fail: false, + }; + let test2 = UnaryTestCase { + source_type: SourceType::Unbounded, + expect_fail: false, + }; + let case = QueryCase { + sql: "EXPLAIN ANALYZE SELECT * FROM test".to_string(), + cases: vec![Arc::new(test1), Arc::new(test2)], + error_operator: "Analyze Error".to_string(), + }; + + case.run().await?; + Ok(()) +} + +#[tokio::test] +/// Tests that plan is valid when the sort requirements are satisfied. +async fn test_bounded_window_agg_sort_requirement() -> Result<()> { + let schema = create_test_schema(); + let source = memory_exec(&schema); + let sort_exprs = vec![sort_expr_options( + "c9", + &source.schema(), + SortOptions { + descending: false, + nulls_first: false, + }, + )]; + let sort = sort_exec(sort_exprs.clone(), source); + let bw = bounded_window_exec("c9", sort_exprs, sort); + assert_plan(bw.as_ref(), vec![ + "BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", + " SortExec: expr=[c9@0 ASC NULLS LAST], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]" + ]); + assert_sanity_check(&bw, true); + Ok(()) +} + +#[tokio::test] +/// Tests that plan is invalid when the sort requirements are not satisfied. +async fn test_bounded_window_agg_no_sort_requirement() -> Result<()> { + let schema = create_test_schema(); + let source = memory_exec(&schema); + let sort_exprs = vec![sort_expr_options( + "c9", + &source.schema(), + SortOptions { + descending: false, + nulls_first: false, + }, + )]; + let bw = bounded_window_exec("c9", sort_exprs, source); + assert_plan(bw.as_ref(), vec![ + "BoundedWindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow, is_causal: false }], mode=[Sorted]", + " MemoryExec: partitions=1, partition_sizes=[0]" + ]); + // Order requirement of the `BoundedWindowAggExec` is not satisfied. We expect to receive error during sanity check. + assert_sanity_check(&bw, false); + Ok(()) +} + +#[tokio::test] +/// A valid when a single partition requirement +/// is satisfied. +async fn test_global_limit_single_partition() -> Result<()> { + let schema = create_test_schema(); + let source = memory_exec(&schema); + let limit = global_limit_exec(source); + + assert_plan( + limit.as_ref(), + vec![ + "GlobalLimitExec: skip=0, fetch=100", + " MemoryExec: partitions=1, partition_sizes=[0]", + ], + ); + assert_sanity_check(&limit, true); + Ok(()) +} + +#[tokio::test] +/// An invalid plan when a single partition requirement +/// is not satisfied. +async fn test_global_limit_multi_partition() -> Result<()> { + let schema = create_test_schema(); + let source = memory_exec(&schema); + let limit = global_limit_exec(repartition_exec(source)); + + assert_plan( + limit.as_ref(), + vec![ + "GlobalLimitExec: skip=0, fetch=100", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " MemoryExec: partitions=1, partition_sizes=[0]", + ], + ); + // Distribution requirement of the `GlobalLimitExec` is not satisfied. We expect to receive error during sanity check. + assert_sanity_check(&limit, false); + Ok(()) +} + +#[tokio::test] +/// A plan with no requirements should satisfy. +async fn test_local_limit() -> Result<()> { + let schema = create_test_schema(); + let source = memory_exec(&schema); + let limit = local_limit_exec(source); + + assert_plan( + limit.as_ref(), + vec![ + "LocalLimitExec: fetch=100", + " MemoryExec: partitions=1, partition_sizes=[0]", + ], + ); + assert_sanity_check(&limit, true); + Ok(()) +} + +#[tokio::test] +/// Valid plan with multiple children satisfy both order and distribution. +async fn test_sort_merge_join_satisfied() -> Result<()> { + let schema1 = create_test_schema(); + let schema2 = create_test_schema2(); + let source1 = memory_exec(&schema1); + let source2 = memory_exec(&schema2); + let sort_opts = SortOptions::default(); + let sort_exprs1 = vec![sort_expr_options("c9", &source1.schema(), sort_opts)]; + let sort_exprs2 = vec![sort_expr_options("a", &source2.schema(), sort_opts)]; + let left = sort_exec(sort_exprs1, source1); + let right = sort_exec(sort_exprs2, source2); + let left_jcol = col("c9", &left.schema()).unwrap(); + let right_jcol = col("a", &right.schema()).unwrap(); + let left = Arc::new(RepartitionExec::try_new( + left, + Partitioning::Hash(vec![left_jcol.clone()], 10), + )?); + + let right = Arc::new(RepartitionExec::try_new( + right, + Partitioning::Hash(vec![right_jcol.clone()], 10), + )?); + + let join_on = vec![(left_jcol as _, right_jcol as _)]; + let join_ty = JoinType::Inner; + let smj = sort_merge_join_exec(left, right, &join_on, &join_ty); + + assert_plan( + smj.as_ref(), + vec![ + "SortMergeJoin: join_type=Inner, on=[(c9@0, a@0)]", + " RepartitionExec: partitioning=Hash([c9@0], 10), input_partitions=1", + " SortExec: expr=[c9@0 ASC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]", + " RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=1", + " SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ], + ); + assert_sanity_check(&smj, true); + Ok(()) +} + +#[tokio::test] +/// Invalid case when the order is not satisfied by the 2nd +/// child. +async fn test_sort_merge_join_order_missing() -> Result<()> { + let schema1 = create_test_schema(); + let schema2 = create_test_schema2(); + let source1 = memory_exec(&schema1); + let right = memory_exec(&schema2); + let sort_exprs1 = vec![sort_expr_options( + "c9", + &source1.schema(), + SortOptions::default(), + )]; + let left = sort_exec(sort_exprs1, source1); + // Missing sort of the right child here.. + let left_jcol = col("c9", &left.schema()).unwrap(); + let right_jcol = col("a", &right.schema()).unwrap(); + let left = Arc::new(RepartitionExec::try_new( + left, + Partitioning::Hash(vec![left_jcol.clone()], 10), + )?); + + let right = Arc::new(RepartitionExec::try_new( + right, + Partitioning::Hash(vec![right_jcol.clone()], 10), + )?); + + let join_on = vec![(left_jcol as _, right_jcol as _)]; + let join_ty = JoinType::Inner; + let smj = sort_merge_join_exec(left, right, &join_on, &join_ty); + + assert_plan( + smj.as_ref(), + vec![ + "SortMergeJoin: join_type=Inner, on=[(c9@0, a@0)]", + " RepartitionExec: partitioning=Hash([c9@0], 10), input_partitions=1", + " SortExec: expr=[c9@0 ASC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]", + " RepartitionExec: partitioning=Hash([a@0], 10), input_partitions=1", + " MemoryExec: partitions=1, partition_sizes=[0]", + ], + ); + // Order requirement for the `SortMergeJoin` is not satisfied for right child. We expect to receive error during sanity check. + assert_sanity_check(&smj, false); + Ok(()) +} + +#[tokio::test] +/// Invalid case when the distribution is not satisfied by the 2nd +/// child. +async fn test_sort_merge_join_dist_missing() -> Result<()> { + let schema1 = create_test_schema(); + let schema2 = create_test_schema2(); + let source1 = memory_exec(&schema1); + let source2 = memory_exec(&schema2); + let sort_opts = SortOptions::default(); + let sort_exprs1 = vec![sort_expr_options("c9", &source1.schema(), sort_opts)]; + let sort_exprs2 = vec![sort_expr_options("a", &source2.schema(), sort_opts)]; + let left = sort_exec(sort_exprs1, source1); + let right = sort_exec(sort_exprs2, source2); + let right = Arc::new(RepartitionExec::try_new( + right, + Partitioning::RoundRobinBatch(10), + )?); + let left_jcol = col("c9", &left.schema()).unwrap(); + let right_jcol = col("a", &right.schema()).unwrap(); + let left = Arc::new(RepartitionExec::try_new( + left, + Partitioning::Hash(vec![left_jcol.clone()], 10), + )?); + + // Missing hash partitioning on right child. + + let join_on = vec![(left_jcol as _, right_jcol as _)]; + let join_ty = JoinType::Inner; + let smj = sort_merge_join_exec(left, right, &join_on, &join_ty); + + assert_plan( + smj.as_ref(), + vec![ + "SortMergeJoin: join_type=Inner, on=[(c9@0, a@0)]", + " RepartitionExec: partitioning=Hash([c9@0], 10), input_partitions=1", + " SortExec: expr=[c9@0 ASC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " SortExec: expr=[a@0 ASC], preserve_partitioning=[false]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ], + ); + // Distribution requirement for the `SortMergeJoin` is not satisfied for right child (has round-robin partitioning). We expect to receive error during sanity check. + assert_sanity_check(&smj, false); + Ok(()) +} diff --git a/datafusion/physical-optimizer/src/test_utils.rs b/datafusion/physical-optimizer/tests/test_utils.rs similarity index 84% rename from datafusion/physical-optimizer/src/test_utils.rs rename to datafusion/physical-optimizer/tests/test_utils.rs index 9f0b5ddf6f40..8007df41774c 100644 --- a/datafusion/physical-optimizer/src/test_utils.rs +++ b/datafusion/physical-optimizer/tests/test_utils.rs @@ -15,21 +15,23 @@ // specific language governing permissions and limitations // under the License. -//! Collection of testing utility functions that are leveraged by the query optimizer rules - -use std::sync::Arc; +//! Test utilities for physical optimizer tests use std::any::Any; use std::fmt::Formatter; +use std::sync::Arc; use arrow_schema::{Schema, SchemaRef, SortOptions}; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_common::{JoinType, Result}; +use datafusion_execution::object_store::ObjectStoreUrl; +use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use datafusion_expr::test::function_stub::count_udaf; use datafusion_expr::{WindowFrame, WindowFunctionDefinition}; use datafusion_physical_expr::expressions::col; use datafusion_physical_expr::PhysicalExpr; -use datafusion_physical_expr::PhysicalSortExpr; +use datafusion_physical_expr_common::sort_expr::LexRequirement; +use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; use datafusion_physical_plan::aggregates::{ AggregateExec, AggregateMode, PhysicalGroupBy, }; @@ -43,18 +45,15 @@ use datafusion_physical_plan::memory::MemoryExec; use datafusion_physical_plan::repartition::RepartitionExec; use datafusion_physical_plan::sorts::sort::SortExec; use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; +use datafusion_physical_plan::streaming::{PartitionStream, StreamingTableExec}; +use datafusion_physical_plan::tree_node::PlanContext; use datafusion_physical_plan::union::UnionExec; use datafusion_physical_plan::windows::{create_window_expr, BoundedWindowAggExec}; -use datafusion_physical_plan::{InputOrderMode, Partitioning}; - -use datafusion_execution::{SendableRecordBatchStream, TaskContext}; -use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; use datafusion_physical_plan::ExecutionPlan; - -use datafusion_physical_plan::tree_node::PlanContext; use datafusion_physical_plan::{ displayable, DisplayAs, DisplayFormatType, PlanProperties, }; +use datafusion_physical_plan::{InputOrderMode, Partitioning}; pub fn sort_merge_join_exec( left: Arc, @@ -334,3 +333,70 @@ pub fn check_integrity(context: PlanContext) -> Result Vec<&str> { + plan.split('\n') + .map(|s| s.trim()) + .filter(|s| !s.is_empty()) + .collect() +} + +// construct a stream partition for test purposes +#[derive(Debug)] +pub(crate) struct TestStreamPartition { + pub schema: SchemaRef, +} + +impl PartitionStream for TestStreamPartition { + fn schema(&self) -> &SchemaRef { + &self.schema + } + fn execute(&self, _ctx: Arc) -> SendableRecordBatchStream { + unreachable!() + } +} + +/// Create an unbounded stream exec +pub fn stream_exec_ordered( + schema: &SchemaRef, + sort_exprs: impl IntoIterator, +) -> Arc { + let sort_exprs = sort_exprs.into_iter().collect(); + + Arc::new( + StreamingTableExec::try_new( + 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(), + ) +} From f04332d6cf70d5d60a258c61a373bd0314866979 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Wed, 22 Jan 2025 16:35:45 +0300 Subject: [PATCH 18/24] Further migrations and simplificaitons --- datafusion/core/src/test_util/mod.rs | 58 --- .../combine_partial_final_agg.rs | 10 - .../enforce_distribution.rs | 30 +- .../physical_optimizer/enforce_sorting.rs | 351 ++++++++++++++- .../limited_distinct_aggregation.rs | 265 ++++++++++++ .../core/tests/physical_optimizer/mod.rs | 43 +- .../replace_with_order_preserving_variants.rs | 69 ++- .../physical_optimizer/sanity_checker.rs | 229 ++++++++++ .../src/aggregate_statistics.rs | 60 +-- datafusion/physical-optimizer/src/lib.rs | 2 +- .../{tests => src}/test_utils.rs | 181 +++++++- datafusion/physical-optimizer/src/utils.rs | 6 - .../tests/enforce_sorting.rs | 402 +++--------------- .../tests/limited_distinct_aggregation.rs | 324 +------------- datafusion/physical-optimizer/tests/mod.rs | 2 - .../tests/sanity_checker.rs | 226 ---------- 16 files changed, 1144 insertions(+), 1114 deletions(-) create mode 100644 datafusion/core/tests/physical_optimizer/limited_distinct_aggregation.rs rename datafusion/{physical-optimizer/tests => core/tests/physical_optimizer}/replace_with_order_preserving_variants.rs (99%) rename datafusion/physical-optimizer/{tests => src}/test_utils.rs (70%) diff --git a/datafusion/core/src/test_util/mod.rs b/datafusion/core/src/test_util/mod.rs index d608db25fe98..50e33b27e1bb 100644 --- a/datafusion/core/src/test_util/mod.rs +++ b/datafusion/core/src/test_util/mod.rs @@ -44,11 +44,7 @@ use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use arrow::record_batch::RecordBatch; use datafusion_catalog::Session; use datafusion_common::TableReference; -use datafusion_expr::utils::COUNT_STAR_EXPANSION; use datafusion_expr::{CreateExternalTable, Expr, SortExpr, TableType}; -use datafusion_functions_aggregate::count::count_udaf; -use datafusion_physical_expr::aggregate::{AggregateExprBuilder, AggregateFunctionExpr}; -use datafusion_physical_expr::{expressions, PhysicalExpr}; use async_trait::async_trait; use futures::Stream; @@ -276,57 +272,3 @@ pub fn bounded_stream(batch: RecordBatch, limit: usize) -> SendableRecordBatchSt batch, }) } - -/// Describe the type of aggregate being tested -pub enum TestAggregate { - /// Testing COUNT(*) type aggregates - CountStar, - - /// Testing for COUNT(column) aggregate - ColumnA(Arc), -} - -impl TestAggregate { - /// Create a new COUNT(*) aggregate - pub fn new_count_star() -> Self { - Self::CountStar - } - - /// Create a new COUNT(column) aggregate - pub fn new_count_column(schema: &Arc) -> Self { - Self::ColumnA(Arc::clone(schema)) - } - - /// Return appropriate expr depending if COUNT is for col or table (*) - pub fn count_expr(&self, schema: &Schema) -> AggregateFunctionExpr { - AggregateExprBuilder::new(count_udaf(), vec![self.column()]) - .schema(Arc::new(schema.clone())) - .alias(self.column_name()) - .build() - .unwrap() - } - - /// what argument would this aggregate need in the plan? - fn column(&self) -> Arc { - match self { - Self::CountStar => expressions::lit(COUNT_STAR_EXPANSION), - Self::ColumnA(s) => expressions::col("a", s).unwrap(), - } - } - - /// What name would this aggregate produce in a plan? - pub fn column_name(&self) -> &'static str { - match self { - Self::CountStar => "COUNT(*)", - Self::ColumnA(_) => "COUNT(a)", - } - } - - /// What is the expected count? - pub fn expected_count(&self) -> i64 { - match self { - TestAggregate::CountStar => 3, - TestAggregate::ColumnA(_) => 2, - } - } -} diff --git a/datafusion/core/tests/physical_optimizer/combine_partial_final_agg.rs b/datafusion/core/tests/physical_optimizer/combine_partial_final_agg.rs index 4b161b584c85..28761fff891a 100644 --- a/datafusion/core/tests/physical_optimizer/combine_partial_final_agg.rs +++ b/datafusion/core/tests/physical_optimizer/combine_partial_final_agg.rs @@ -22,8 +22,6 @@ //! Tests for [`CombinePartialFinalAggregate`] physical optimizer rule -mod r#mod; - use std::sync::Arc; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; @@ -74,14 +72,6 @@ fn schema() -> SchemaRef { ])) } -fn parquet_exec(schema: &SchemaRef) -> Arc { - ParquetExec::builder( - FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema.clone()) - .with_file(PartitionedFile::new("x".to_string(), 100)), - ) - .build_arc() -} - fn partial_aggregate_exec( input: Arc, group_by: PhysicalGroupBy, diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index 6c9ae24816e7..e1b6ff8bf482 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use std::fmt::Debug; use std::ops::Deref; use std::sync::Arc; @@ -23,7 +24,6 @@ use datafusion::datasource::file_format::file_compression_type::FileCompressionT use datafusion::datasource::listing::PartitionedFile; use datafusion::datasource::object_store::ObjectStoreUrl; use datafusion::datasource::physical_plan::{CsvExec, FileScanConfig, ParquetExec}; - use datafusion_physical_expr::PhysicalExpr; use datafusion_physical_optimizer::enforce_sorting::EnforceSorting; use datafusion_physical_optimizer::output_requirements::OutputRequirements; @@ -40,9 +40,7 @@ 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::{displayable, DisplayAs, DisplayFormatType, Statistics}; - use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use datafusion_common::ScalarValue; use datafusion_expr::{JoinType, Operator}; @@ -59,13 +57,9 @@ use datafusion_physical_plan::aggregates::{ use datafusion_physical_plan::execution_plan::ExecutionPlan; use datafusion_physical_plan::projection::ProjectionExec; use datafusion_physical_plan::PlanProperties; -use std::fmt::Debug; - use datafusion_common::error::Result; - use arrow::compute::SortOptions; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; - use datafusion_physical_plan::ExecutionPlanProperties; type DistributionContext = PlanContext; @@ -172,32 +166,10 @@ impl ExecutionPlan for SortRequiredExec { } } -pub(crate) fn schema() -> SchemaRef { - Arc::new(Schema::new(vec![ - Field::new("a", DataType::Int64, true), - Field::new("b", DataType::Int64, true), - Field::new("c", DataType::Int64, true), - Field::new("d", DataType::Int32, true), - Field::new("e", DataType::Boolean, true), - ])) -} - fn parquet_exec() -> Arc { parquet_exec_with_sort(vec![]) } -/// create a single parquet file that is sorted -pub(crate) fn parquet_exec_with_sort( - output_ordering: Vec, -) -> Arc { - ParquetExec::builder( - FileScanConfig::new(ObjectStoreUrl::parse("test:///").unwrap(), schema()) - .with_file(PartitionedFile::new("x".to_string(), 100)) - .with_output_ordering(output_ordering), - ) - .build_arc() -} - fn parquet_exec_multiple() -> Arc { parquet_exec_multiple_sorted(vec![]) } diff --git a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs index b8cc75f1c44a..61d4a8abfa10 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs @@ -19,12 +19,10 @@ 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 datafusion_physical_expr::Partitioning; use datafusion_physical_expr_common::sort_expr::LexOrdering; @@ -33,12 +31,100 @@ use datafusion_physical_optimizer::enforce_sorting::replace_with_order_preservin 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::config::ConfigOptions; use datafusion_common::tree_node::{TreeNode, TransformedResult}; +use datafusion_physical_optimizer::test_utils::{check_integrity,bounded_window_exec, coalesce_partitions_exec, create_test_schema, create_test_schema2, create_test_schema3, filter_exec, global_limit_exec, hash_join_exec, limit_exec, local_limit_exec, memory_exec, repartition_exec, sort_exec, sort_expr, sort_expr_options, sort_merge_join_exec, sort_preserving_merge_exec, spr_repartition_exec, stream_exec_ordered, union_exec}; +use crate::physical_optimizer::{csv_exec_ordered, csv_exec_sorted, parquet_exec, parquet_exec_sorted}; + +/// Runs the sort enforcement optimizer and asserts the plan +/// against the original and expected plans +/// +/// `$EXPECTED_PLAN_LINES`: input plan +/// `$EXPECTED_OPTIMIZED_PLAN_LINES`: optimized plan +/// `$PLAN`: the plan to optimized +/// `REPARTITION_SORTS`: Flag to set `config.options.optimizer.repartition_sorts` option. +/// +macro_rules! assert_optimized { + ($EXPECTED_PLAN_LINES: expr, $EXPECTED_OPTIMIZED_PLAN_LINES: expr, $PLAN: expr, $REPARTITION_SORTS: expr) => { + let mut config = ConfigOptions::new(); + config.optimizer.repartition_sorts = $REPARTITION_SORTS; + + // This file has 4 rules that use tree node, apply these rules as in the + // EnforceSorting::optimize implementation + // After these operations tree nodes should be in a consistent state. + // This code block makes sure that these rules doesn't violate tree node integrity. + { + let plan_requirements = PlanWithCorrespondingSort::new_default($PLAN.clone()); + let adjusted = plan_requirements + .transform_up(ensure_sorting) + .data() + .and_then(check_integrity)?; + // TODO: End state payloads will be checked here. + + let new_plan = if config.optimizer.repartition_sorts { + let plan_with_coalesce_partitions = + PlanWithCorrespondingCoalescePartitions::new_default(adjusted.plan); + let parallel = plan_with_coalesce_partitions + .transform_up(parallelize_sorts) + .data() + .and_then(check_integrity)?; + // TODO: End state payloads will be checked here. + parallel.plan + } else { + adjusted.plan + }; + + let plan_with_pipeline_fixer = OrderPreservationContext::new_default(new_plan); + let updated_plan = plan_with_pipeline_fixer + .transform_up(|plan_with_pipeline_fixer| { + replace_with_order_preserving_variants( + plan_with_pipeline_fixer, + false, + true, + &config, + ) + }) + .data() + .and_then(check_integrity)?; + // TODO: End state payloads will be checked here. + + let mut sort_pushdown = SortPushDown::new_default(updated_plan.plan); + assign_initial_requirements(&mut sort_pushdown); + check_integrity(pushdown_sorts(sort_pushdown)?)?; + // TODO: End state payloads will be checked here. + } + + let physical_plan = $PLAN; + let formatted = displayable(physical_plan.as_ref()).indent(true).to_string(); + let actual: Vec<&str> = formatted.trim().lines().collect(); + + let expected_plan_lines: Vec<&str> = $EXPECTED_PLAN_LINES + .iter().map(|s| *s).collect(); + + assert_eq!( + expected_plan_lines, actual, + "\n**Original Plan Mismatch\n\nexpected:\n\n{expected_plan_lines:#?}\nactual:\n\n{actual:#?}\n\n" + ); + + let expected_optimized_lines: Vec<&str> = $EXPECTED_OPTIMIZED_PLAN_LINES + .iter().map(|s| *s).collect(); + + // Run the actual optimizer + let optimized_physical_plan = + EnforceSorting::new().optimize(physical_plan,&config)?; + + // Get string representation of the plan + let actual = get_plan_string(&optimized_physical_plan); + assert_eq!( + expected_optimized_lines, actual, + "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected_optimized_lines:#?}\nactual:\n\n{actual:#?}\n\n" + ); + + }; +} #[tokio::test] async fn test_remove_unnecessary_sort5() -> Result<()> { @@ -816,3 +902,262 @@ async fn test_sort_merge_join_complex_order_by() -> Result<()> { 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] +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_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(()) +} diff --git a/datafusion/core/tests/physical_optimizer/limited_distinct_aggregation.rs b/datafusion/core/tests/physical_optimizer/limited_distinct_aggregation.rs new file mode 100644 index 000000000000..774fdbc83349 --- /dev/null +++ b/datafusion/core/tests/physical_optimizer/limited_distinct_aggregation.rs @@ -0,0 +1,265 @@ +// 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. + +//! Integration tests for [`LimitedDistinctAggregation`] physical optimizer rule + +use std::sync::Arc; + +use crate::physical_optimizer::parquet_exec_with_sort; +use arrow::{compute::SortOptions, util::pretty::pretty_format_batches}; +use datafusion::prelude::SessionContext; +use datafusion_common::Result; +use datafusion_execution::config::SessionConfig; +use datafusion_physical_expr::{expressions::col, PhysicalSortExpr}; +use datafusion_physical_expr_common::sort_expr::LexOrdering; +use datafusion_physical_optimizer::test_utils::{ + assert_plan_matches_expected, build_group_by, mock_data, schema, +}; +use datafusion_physical_plan::{ + aggregates::{AggregateExec, AggregateMode}, + collect, + limit::{GlobalLimitExec, LocalLimitExec}, + ExecutionPlan, +}; + +async fn assert_results_match_expected( + plan: Arc, + expected: &str, +) -> Result<()> { + let cfg = SessionConfig::new().with_target_partitions(1); + let ctx = SessionContext::new_with_config(cfg); + let batches = collect(plan, ctx.task_ctx()).await?; + let actual = format!("{}", pretty_format_batches(&batches)?); + assert_eq!(actual, expected); + Ok(()) +} + +#[tokio::test] +async fn test_partial_final() -> Result<()> { + let source = mock_data()?; + let schema = source.schema(); + + // `SELECT a FROM MemoryExec GROUP BY a LIMIT 4;`, Partial/Final AggregateExec + let partial_agg = AggregateExec::try_new( + AggregateMode::Partial, + build_group_by(&schema.clone(), vec!["a".to_string()]), + vec![], /* aggr_expr */ + vec![], /* filter_expr */ + source, /* input */ + schema.clone(), /* input_schema */ + )?; + let final_agg = AggregateExec::try_new( + AggregateMode::Final, + build_group_by(&schema.clone(), vec!["a".to_string()]), + vec![], /* aggr_expr */ + vec![], /* filter_expr */ + Arc::new(partial_agg), /* input */ + schema.clone(), /* input_schema */ + )?; + let limit_exec = LocalLimitExec::new( + Arc::new(final_agg), + 4, // fetch + ); + // expected to push the limit to the Partial and Final AggregateExecs + let expected = [ + "LocalLimitExec: fetch=4", + "AggregateExec: mode=Final, gby=[a@0 as a], aggr=[], lim=[4]", + "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[], lim=[4]", + "MemoryExec: partitions=1, partition_sizes=[1]", + ]; + let plan: Arc = Arc::new(limit_exec); + assert_plan_matches_expected(&plan, &expected)?; + let expected = r#" ++---+ +| a | ++---+ +| 1 | +| 2 | +| | +| 4 | ++---+ +"# + .trim(); + assert_results_match_expected(plan, expected).await?; + Ok(()) +} + +#[tokio::test] +async fn test_single_local() -> Result<()> { + let source = mock_data()?; + let schema = source.schema(); + + // `SELECT a FROM MemoryExec GROUP BY a LIMIT 4;`, Single AggregateExec + let single_agg = AggregateExec::try_new( + AggregateMode::Single, + build_group_by(&schema.clone(), vec!["a".to_string()]), + vec![], /* aggr_expr */ + vec![], /* filter_expr */ + source, /* input */ + schema.clone(), /* input_schema */ + )?; + let limit_exec = LocalLimitExec::new( + Arc::new(single_agg), + 4, // fetch + ); + // expected to push the limit to the AggregateExec + let expected = [ + "LocalLimitExec: fetch=4", + "AggregateExec: mode=Single, gby=[a@0 as a], aggr=[], lim=[4]", + "MemoryExec: partitions=1, partition_sizes=[1]", + ]; + let plan: Arc = Arc::new(limit_exec); + assert_plan_matches_expected(&plan, &expected)?; + let expected = r#" ++---+ +| a | ++---+ +| 1 | +| 2 | +| | +| 4 | ++---+ +"# + .trim(); + assert_results_match_expected(plan, expected).await?; + Ok(()) +} + +#[tokio::test] +async fn test_single_global() -> Result<()> { + let source = mock_data()?; + let schema = source.schema(); + + // `SELECT a FROM MemoryExec GROUP BY a LIMIT 4;`, Single AggregateExec + let single_agg = AggregateExec::try_new( + AggregateMode::Single, + build_group_by(&schema.clone(), vec!["a".to_string()]), + vec![], /* aggr_expr */ + vec![], /* filter_expr */ + source, /* input */ + schema.clone(), /* input_schema */ + )?; + let limit_exec = GlobalLimitExec::new( + Arc::new(single_agg), + 1, // skip + Some(3), // fetch + ); + // expected to push the skip+fetch limit to the AggregateExec + let expected = [ + "GlobalLimitExec: skip=1, fetch=3", + "AggregateExec: mode=Single, gby=[a@0 as a], aggr=[], lim=[4]", + "MemoryExec: partitions=1, partition_sizes=[1]", + ]; + let plan: Arc = Arc::new(limit_exec); + assert_plan_matches_expected(&plan, &expected)?; + let expected = r#" ++---+ +| a | ++---+ +| 2 | +| | +| 4 | ++---+ +"# + .trim(); + assert_results_match_expected(plan, expected).await?; + Ok(()) +} + +#[tokio::test] +async fn test_distinct_cols_different_than_group_by_cols() -> Result<()> { + let source = mock_data()?; + let schema = source.schema(); + + // `SELECT distinct a FROM MemoryExec GROUP BY a, b LIMIT 4;`, Single/Single AggregateExec + let group_by_agg = AggregateExec::try_new( + AggregateMode::Single, + build_group_by(&schema.clone(), vec!["a".to_string(), "b".to_string()]), + vec![], /* aggr_expr */ + vec![], /* filter_expr */ + source, /* input */ + schema.clone(), /* input_schema */ + )?; + let distinct_agg = AggregateExec::try_new( + AggregateMode::Single, + build_group_by(&schema.clone(), vec!["a".to_string()]), + vec![], /* aggr_expr */ + vec![], /* filter_expr */ + Arc::new(group_by_agg), /* input */ + schema.clone(), /* input_schema */ + )?; + let limit_exec = LocalLimitExec::new( + Arc::new(distinct_agg), + 4, // fetch + ); + // expected to push the limit to the outer AggregateExec only + let expected = [ + "LocalLimitExec: fetch=4", + "AggregateExec: mode=Single, gby=[a@0 as a], aggr=[], lim=[4]", + "AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[]", + "MemoryExec: partitions=1, partition_sizes=[1]", + ]; + let plan: Arc = Arc::new(limit_exec); + assert_plan_matches_expected(&plan, &expected)?; + let expected = r#" ++---+ +| a | ++---+ +| 1 | +| 2 | +| | +| 4 | ++---+ +"# + .trim(); + assert_results_match_expected(plan, expected).await?; + Ok(()) +} + +#[test] +fn test_has_order_by() -> Result<()> { + let sort_key = LexOrdering::new(vec![PhysicalSortExpr { + expr: col("a", &schema()).unwrap(), + options: SortOptions::default(), + }]); + let source = parquet_exec_with_sort(vec![sort_key]); + let schema = source.schema(); + + // `SELECT a FROM MemoryExec WHERE a > 1 GROUP BY a LIMIT 10;`, Single AggregateExec + // the `a > 1` filter is applied in the AggregateExec + let single_agg = AggregateExec::try_new( + AggregateMode::Single, + build_group_by(&schema, vec!["a".to_string()]), + vec![], /* aggr_expr */ + vec![], /* filter_expr */ + source, /* input */ + schema, /* input_schema */ + )?; + let limit_exec = LocalLimitExec::new( + Arc::new(single_agg), + 10, // fetch + ); + // expected not to push the limit to the AggregateExec + let expected = [ + "LocalLimitExec: fetch=10", + "AggregateExec: mode=Single, gby=[a@0 as a], aggr=[], ordering_mode=Sorted", + "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", + ]; + let plan: Arc = Arc::new(limit_exec); + assert_plan_matches_expected(&plan, &expected)?; + Ok(()) +} diff --git a/datafusion/core/tests/physical_optimizer/mod.rs b/datafusion/core/tests/physical_optimizer/mod.rs index 992028eeaa65..4e2827d4750b 100644 --- a/datafusion/core/tests/physical_optimizer/mod.rs +++ b/datafusion/core/tests/physical_optimizer/mod.rs @@ -18,14 +18,43 @@ //! Physical Optimizer integration tests mod enforce_sorting; +mod limited_distinct_aggregation; mod sanity_checker; +use arrow_schema::SchemaRef; +use datafusion::datasource::file_format::file_compression_type::FileCompressionType; use datafusion::datasource::listing::PartitionedFile; -use datafusion::datasource::physical_plan::{FileScanConfig, ParquetExec}; +use datafusion::datasource::physical_plan::{CsvExec, FileScanConfig, ParquetExec}; use datafusion_execution::object_store::ObjectStoreUrl; -use datafusion_physical_expr_common::sort_expr::LexOrdering; +use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; +use datafusion_physical_optimizer::test_utils::schema; +use datafusion_physical_plan::ExecutionPlan; use std::sync::Arc; +/// 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() +} + /// create a single parquet file that is sorted pub(crate) fn parquet_exec_with_sort( output_ordering: Vec, @@ -38,16 +67,6 @@ pub(crate) fn parquet_exec_with_sort( .build_arc() } -fn schema() -> SchemaRef { - Arc::new(Schema::new(vec![ - Field::new("a", DataType::Int64, true), - Field::new("b", DataType::Int64, true), - Field::new("c", DataType::Int64, true), - Field::new("d", DataType::Int32, true), - Field::new("e", DataType::Boolean, true), - ])) -} - /// Created a sorted Csv exec pub fn csv_exec_sorted( schema: &SchemaRef, diff --git a/datafusion/physical-optimizer/tests/replace_with_order_preserving_variants.rs b/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs similarity index 99% rename from datafusion/physical-optimizer/tests/replace_with_order_preserving_variants.rs rename to datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs index b83c661d07d7..c0a202fcfd02 100644 --- a/datafusion/physical-optimizer/tests/replace_with_order_preserving_variants.rs +++ b/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs @@ -39,14 +39,46 @@ use datafusion_common::tree_node::{TransformedResult, TreeNode}; 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::test_utils::{check_integrity, stream_exec_ordered_with_projection}; use datafusion_physical_optimizer::enforce_sorting::replace_with_order_preserving_variants::{replace_with_order_preserving_variants, OrderPreservationContext}; -use crate::test_util::stream_exec_ordered_with_projection; use datafusion_common::config::ConfigOptions; use rstest::rstest; use url::Url; +/// Runs the `replace_with_order_preserving_variants` sub-rule and asserts +/// the plan against the original and expected plans. +/// +/// # Parameters +/// +/// * `$EXPECTED_PLAN_LINES`: Expected input plan. +/// * `EXPECTED_OPTIMIZED_PLAN_LINES`: Optimized plan when the flag +/// `prefer_existing_sort` is `false`. +/// * `EXPECTED_PREFER_SORT_ON_OPTIMIZED_PLAN_LINES`: Optimized plan when +/// the flag `prefer_existing_sort` is `true`. +/// * `$PLAN`: The plan to optimize. +macro_rules! assert_optimized_prefer_sort_on_off { + ($EXPECTED_PLAN_LINES: expr, $EXPECTED_OPTIMIZED_PLAN_LINES: expr, $EXPECTED_PREFER_SORT_ON_OPTIMIZED_PLAN_LINES: expr, $PLAN: expr, $PREFER_EXISTING_SORT: expr, $SOURCE_UNBOUNDED: expr) => { + if $PREFER_EXISTING_SORT { + assert_optimized!( + $EXPECTED_PLAN_LINES, + $EXPECTED_PREFER_SORT_ON_OPTIMIZED_PLAN_LINES, + $PLAN, + $PREFER_EXISTING_SORT, + $SOURCE_UNBOUNDED + ); + } else { + assert_optimized!( + $EXPECTED_PLAN_LINES, + $EXPECTED_OPTIMIZED_PLAN_LINES, + $PLAN, + $PREFER_EXISTING_SORT, + $SOURCE_UNBOUNDED + ); + } + }; +} + /// Runs the `replace_with_order_preserving_variants` sub-rule and asserts /// the plan against the original and expected plans for both bounded and /// unbounded cases. @@ -87,39 +119,6 @@ macro_rules! assert_optimized_in_all_boundedness_situations { }; } -/// Runs the `replace_with_order_preserving_variants` sub-rule and asserts -/// the plan against the original and expected plans. -/// -/// # Parameters -/// -/// * `$EXPECTED_PLAN_LINES`: Expected input plan. -/// * `EXPECTED_OPTIMIZED_PLAN_LINES`: Optimized plan when the flag -/// `prefer_existing_sort` is `false`. -/// * `EXPECTED_PREFER_SORT_ON_OPTIMIZED_PLAN_LINES`: Optimized plan when -/// the flag `prefer_existing_sort` is `true`. -/// * `$PLAN`: The plan to optimize. -macro_rules! assert_optimized_prefer_sort_on_off { - ($EXPECTED_PLAN_LINES: expr, $EXPECTED_OPTIMIZED_PLAN_LINES: expr, $EXPECTED_PREFER_SORT_ON_OPTIMIZED_PLAN_LINES: expr, $PLAN: expr, $PREFER_EXISTING_SORT: expr, $SOURCE_UNBOUNDED: expr) => { - if $PREFER_EXISTING_SORT { - assert_optimized!( - $EXPECTED_PLAN_LINES, - $EXPECTED_PREFER_SORT_ON_OPTIMIZED_PLAN_LINES, - $PLAN, - $PREFER_EXISTING_SORT, - $SOURCE_UNBOUNDED - ); - } else { - assert_optimized!( - $EXPECTED_PLAN_LINES, - $EXPECTED_OPTIMIZED_PLAN_LINES, - $PLAN, - $PREFER_EXISTING_SORT, - $SOURCE_UNBOUNDED - ); - } - }; -} - /// Runs the `replace_with_order_preserving_variants` sub-rule and asserts /// the plan against the original and expected plans. /// diff --git a/datafusion/core/tests/physical_optimizer/sanity_checker.rs b/datafusion/core/tests/physical_optimizer/sanity_checker.rs index 6b1f92996326..58a11e2807d0 100644 --- a/datafusion/core/tests/physical_optimizer/sanity_checker.rs +++ b/datafusion/core/tests/physical_optimizer/sanity_checker.rs @@ -15,6 +15,12 @@ // specific language governing permissions and limitations // under the License. +use async_trait::async_trait; +use datafusion::datasource::stream::{FileStreamProvider, StreamConfig, StreamTable}; +use datafusion::prelude::{CsvReadOptions, SessionContext}; +use datafusion_common::Result; +use std::sync::Arc; + async fn register_current_csv( ctx: &SessionContext, table_name: &str, @@ -128,3 +134,226 @@ impl QueryCase { Ok(()) } } + +#[tokio::test] +async fn test_hash_left_join_swap() -> Result<()> { + let test1 = BinaryTestCase { + source_types: (SourceType::Unbounded, SourceType::Bounded), + expect_fail: false, + }; + + let test2 = BinaryTestCase { + source_types: (SourceType::Bounded, SourceType::Unbounded), + // Left join for bounded build side and unbounded probe side can generate + // both incremental matched rows and final non-matched rows. + expect_fail: false, + }; + let test3 = BinaryTestCase { + source_types: (SourceType::Bounded, SourceType::Bounded), + expect_fail: false, + }; + let case = QueryCase { + sql: "SELECT t2.c1 FROM left as t1 LEFT JOIN right as t2 ON t1.c1 = t2.c1" + .to_string(), + cases: vec![Arc::new(test1), Arc::new(test2), Arc::new(test3)], + error_operator: "operator: HashJoinExec".to_string(), + }; + + case.run().await?; + Ok(()) +} + +#[tokio::test] +async fn test_hash_right_join_swap() -> Result<()> { + let test1 = BinaryTestCase { + source_types: (SourceType::Unbounded, SourceType::Bounded), + expect_fail: true, + }; + let test2 = BinaryTestCase { + source_types: (SourceType::Bounded, SourceType::Unbounded), + expect_fail: false, + }; + let test3 = BinaryTestCase { + source_types: (SourceType::Bounded, SourceType::Bounded), + expect_fail: false, + }; + let case = QueryCase { + sql: "SELECT t2.c1 FROM left as t1 RIGHT JOIN right as t2 ON t1.c1 = t2.c1" + .to_string(), + cases: vec![Arc::new(test1), Arc::new(test2), Arc::new(test3)], + error_operator: "operator: HashJoinExec".to_string(), + }; + + case.run().await?; + Ok(()) +} + +#[tokio::test] +async fn test_hash_inner_join_swap() -> Result<()> { + let test1 = BinaryTestCase { + source_types: (SourceType::Unbounded, SourceType::Bounded), + expect_fail: false, + }; + let test2 = BinaryTestCase { + source_types: (SourceType::Bounded, SourceType::Unbounded), + expect_fail: false, + }; + let test3 = BinaryTestCase { + source_types: (SourceType::Bounded, SourceType::Bounded), + expect_fail: false, + }; + let case = QueryCase { + sql: "SELECT t2.c1 FROM left as t1 JOIN right as t2 ON t1.c1 = t2.c1".to_string(), + cases: vec![Arc::new(test1), Arc::new(test2), Arc::new(test3)], + error_operator: "Join Error".to_string(), + }; + + case.run().await?; + Ok(()) +} + +#[tokio::test] +async fn test_hash_full_outer_join_swap() -> Result<()> { + let test1 = BinaryTestCase { + source_types: (SourceType::Unbounded, SourceType::Bounded), + expect_fail: true, + }; + let test2 = BinaryTestCase { + source_types: (SourceType::Bounded, SourceType::Unbounded), + // Full join for bounded build side and unbounded probe side can generate + // both incremental matched rows and final non-matched rows. + expect_fail: false, + }; + let test3 = BinaryTestCase { + source_types: (SourceType::Bounded, SourceType::Bounded), + expect_fail: false, + }; + let case = QueryCase { + sql: "SELECT t2.c1 FROM left as t1 FULL JOIN right as t2 ON t1.c1 = t2.c1" + .to_string(), + cases: vec![Arc::new(test1), Arc::new(test2), Arc::new(test3)], + error_operator: "operator: HashJoinExec".to_string(), + }; + + case.run().await?; + Ok(()) +} + +#[tokio::test] +async fn test_aggregate() -> Result<()> { + let test1 = UnaryTestCase { + source_type: SourceType::Bounded, + expect_fail: false, + }; + let test2 = UnaryTestCase { + source_type: SourceType::Unbounded, + expect_fail: true, + }; + let case = QueryCase { + sql: "SELECT c1, MIN(c4) FROM test GROUP BY c1".to_string(), + cases: vec![Arc::new(test1), Arc::new(test2)], + error_operator: "operator: AggregateExec".to_string(), + }; + + case.run().await?; + Ok(()) +} + +#[tokio::test] +async fn test_window_agg_hash_partition() -> Result<()> { + let test1 = UnaryTestCase { + source_type: SourceType::Bounded, + expect_fail: false, + }; + let test2 = UnaryTestCase { + source_type: SourceType::Unbounded, + expect_fail: true, + }; + let case = QueryCase { + sql: "SELECT + c9, + SUM(c9) OVER(PARTITION BY c1 ORDER BY c9 ASC ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING) as sum1 + FROM test + LIMIT 5".to_string(), + cases: vec![Arc::new(test1), Arc::new(test2)], + error_operator: "operator: SortExec".to_string() + }; + + case.run().await?; + Ok(()) +} + +#[tokio::test] +async fn test_window_agg_single_partition() -> Result<()> { + let test1 = UnaryTestCase { + source_type: SourceType::Bounded, + expect_fail: false, + }; + let test2 = UnaryTestCase { + source_type: SourceType::Unbounded, + expect_fail: true, + }; + let case = QueryCase { + sql: "SELECT + c9, + SUM(c9) OVER(ORDER BY c9 ASC ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING) as sum1 + FROM test".to_string(), + cases: vec![Arc::new(test1), Arc::new(test2)], + error_operator: "operator: SortExec".to_string() + }; + case.run().await?; + Ok(()) +} + +#[tokio::test] +async fn test_hash_cross_join() -> Result<()> { + let test1 = BinaryTestCase { + source_types: (SourceType::Unbounded, SourceType::Bounded), + expect_fail: true, + }; + let test2 = BinaryTestCase { + source_types: (SourceType::Unbounded, SourceType::Unbounded), + expect_fail: true, + }; + let test3 = BinaryTestCase { + source_types: (SourceType::Bounded, SourceType::Unbounded), + expect_fail: true, + }; + let test4 = BinaryTestCase { + source_types: (SourceType::Bounded, SourceType::Bounded), + expect_fail: false, + }; + let case = QueryCase { + sql: "SELECT t2.c1 FROM left as t1 CROSS JOIN right as t2".to_string(), + cases: vec![ + Arc::new(test1), + Arc::new(test2), + Arc::new(test3), + Arc::new(test4), + ], + error_operator: "operator: CrossJoinExec".to_string(), + }; + + case.run().await?; + Ok(()) +} + +#[tokio::test] +async fn test_analyzer() -> Result<()> { + let test1 = UnaryTestCase { + source_type: SourceType::Bounded, + expect_fail: false, + }; + let test2 = UnaryTestCase { + source_type: SourceType::Unbounded, + expect_fail: false, + }; + let case = QueryCase { + sql: "EXPLAIN ANALYZE SELECT * FROM test".to_string(), + cases: vec![Arc::new(test1), Arc::new(test2)], + error_operator: "Analyze Error".to_string(), + }; + + case.run().await?; + Ok(()) +} diff --git a/datafusion/physical-optimizer/src/aggregate_statistics.rs b/datafusion/physical-optimizer/src/aggregate_statistics.rs index a00bc4b1d571..68bfd63b9a19 100644 --- a/datafusion/physical-optimizer/src/aggregate_statistics.rs +++ b/datafusion/physical-optimizer/src/aggregate_statistics.rs @@ -152,14 +152,9 @@ mod tests { use crate::aggregate_statistics::AggregateStatistics; use crate::PhysicalOptimizerRule; use datafusion_common::config::ConfigOptions; - use datafusion_common::utils::expr::COUNT_STAR_EXPANSION; use datafusion_execution::TaskContext; - use datafusion_functions_aggregate::count::count_udaf; - use datafusion_physical_expr::aggregate::AggregateExprBuilder; - use datafusion_physical_expr::PhysicalExpr; use datafusion_physical_plan::aggregates::AggregateExec; use datafusion_physical_plan::projection::ProjectionExec; - use datafusion_physical_plan::udaf::AggregateFunctionExpr; use datafusion_physical_plan::ExecutionPlan; use std::sync::Arc; @@ -172,6 +167,7 @@ mod tests { use datafusion_physical_plan::filter::FilterExec; use datafusion_physical_plan::memory::MemoryExec; + use crate::test_utils::TestAggregate; use arrow::array::Int32Array; use arrow::datatypes::{DataType, Field, Schema}; use arrow::record_batch::RecordBatch; @@ -179,60 +175,6 @@ mod tests { use datafusion_physical_expr::expressions::{self, cast}; use datafusion_physical_plan::aggregates::AggregateMode; - /// Describe the type of aggregate being tested - pub enum TestAggregate { - /// Testing COUNT(*) type aggregates - CountStar, - - /// Testing for COUNT(column) aggregate - ColumnA(Arc), - } - - impl TestAggregate { - /// Create a new COUNT(*) aggregate - pub fn new_count_star() -> Self { - Self::CountStar - } - - /// Create a new COUNT(column) aggregate - pub fn new_count_column(schema: &Arc) -> Self { - Self::ColumnA(Arc::clone(schema)) - } - - /// Return appropriate expr depending if COUNT is for col or table (*) - pub fn count_expr(&self, schema: &Schema) -> AggregateFunctionExpr { - AggregateExprBuilder::new(count_udaf(), vec![self.column()]) - .schema(Arc::new(schema.clone())) - .alias(self.column_name()) - .build() - .unwrap() - } - - /// what argument would this aggregate need in the plan? - fn column(&self) -> Arc { - match self { - Self::CountStar => expressions::lit(COUNT_STAR_EXPANSION), - Self::ColumnA(s) => expressions::col("a", s).unwrap(), - } - } - - /// What name would this aggregate produce in a plan? - pub fn column_name(&self) -> &'static str { - match self { - Self::CountStar => "COUNT(*)", - Self::ColumnA(_) => "COUNT(a)", - } - } - - /// What is the expected count? - pub fn expected_count(&self) -> i64 { - match self { - TestAggregate::CountStar => 3, - TestAggregate::ColumnA(_) => 2, - } - } - } - /// Mock data using a MemoryExec which has an exact count statistic fn mock_data() -> Result> { let schema = Arc::new(Schema::new(vec![ diff --git a/datafusion/physical-optimizer/src/lib.rs b/datafusion/physical-optimizer/src/lib.rs index 68eb55228a19..507cc6c75ad5 100644 --- a/datafusion/physical-optimizer/src/lib.rs +++ b/datafusion/physical-optimizer/src/lib.rs @@ -32,7 +32,7 @@ pub mod pruning; pub mod sanity_checker; pub mod topk_aggregation; pub mod update_aggr_exprs; -pub mod utils; pub use optimizer::PhysicalOptimizerRule; +pub mod test_utils; pub mod utils; diff --git a/datafusion/physical-optimizer/tests/test_utils.rs b/datafusion/physical-optimizer/src/test_utils.rs similarity index 70% rename from datafusion/physical-optimizer/tests/test_utils.rs rename to datafusion/physical-optimizer/src/test_utils.rs index 8007df41774c..a4a88a576c8b 100644 --- a/datafusion/physical-optimizer/tests/test_utils.rs +++ b/datafusion/physical-optimizer/src/test_utils.rs @@ -17,19 +17,21 @@ //! Test utilities for physical optimizer tests -use std::any::Any; -use std::fmt::Formatter; -use std::sync::Arc; - -use arrow_schema::{Schema, SchemaRef, SortOptions}; +use crate::limited_distinct_aggregation::LimitedDistinctAggregation; +use crate::PhysicalOptimizerRule; +use arrow::array::Int32Array; +use arrow::record_batch::RecordBatch; +use arrow_schema::{DataType, Field, Schema, SchemaRef, SortOptions}; +use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; +use datafusion_common::utils::expr::COUNT_STAR_EXPANSION; use datafusion_common::{JoinType, Result}; -use datafusion_execution::object_store::ObjectStoreUrl; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; use datafusion_expr::test::function_stub::count_udaf; use datafusion_expr::{WindowFrame, WindowFunctionDefinition}; +use datafusion_physical_expr::aggregate::{AggregateExprBuilder, AggregateFunctionExpr}; use datafusion_physical_expr::expressions::col; -use datafusion_physical_expr::PhysicalExpr; +use datafusion_physical_expr::{expressions, PhysicalExpr}; use datafusion_physical_expr_common::sort_expr::LexRequirement; use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; use datafusion_physical_plan::aggregates::{ @@ -54,6 +56,44 @@ use datafusion_physical_plan::{ displayable, DisplayAs, DisplayFormatType, PlanProperties, }; use datafusion_physical_plan::{InputOrderMode, Partitioning}; +use std::any::Any; +use std::fmt::Formatter; +use std::sync::Arc; + +pub fn schema() -> SchemaRef { + Arc::new(Schema::new(vec![ + Field::new("a", DataType::Int64, true), + Field::new("b", DataType::Int64, true), + Field::new("c", DataType::Int64, true), + Field::new("d", DataType::Int32, true), + Field::new("e", DataType::Boolean, true), + ])) +} + +pub fn create_test_schema() -> Result { + let nullable_column = Field::new("nullable_col", DataType::Int32, true); + let non_nullable_column = Field::new("non_nullable_col", DataType::Int32, false); + let schema = Arc::new(Schema::new(vec![nullable_column, non_nullable_column])); + Ok(schema) +} + +pub fn create_test_schema2() -> Result { + let col_a = Field::new("col_a", DataType::Int32, true); + let col_b = Field::new("col_b", DataType::Int32, true); + let schema = Arc::new(Schema::new(vec![col_a, col_b])); + Ok(schema) +} + +// Generate a schema which consists of 5 columns (a, b, c, d, e) +pub fn create_test_schema3() -> Result { + let a = Field::new("a", DataType::Int32, true); + let b = Field::new("b", DataType::Int32, false); + let c = Field::new("c", DataType::Int32, true); + let d = Field::new("d", DataType::Int32, false); + let e = Field::new("e", DataType::Int32, false); + let schema = Arc::new(Schema::new(vec![a, b, c, d, e])); + Ok(schema) +} pub fn sort_merge_join_exec( left: Arc, @@ -365,9 +405,9 @@ pub fn stream_exec_ordered( Arc::new( StreamingTableExec::try_new( - schema.clone(), + Arc::clone(schema), vec![Arc::new(TestStreamPartition { - schema: schema.clone(), + schema: Arc::clone(schema), }) as _], None, vec![sort_exprs], @@ -388,9 +428,9 @@ pub fn stream_exec_ordered_with_projection( Arc::new( StreamingTableExec::try_new( - schema.clone(), + Arc::clone(schema), vec![Arc::new(TestStreamPartition { - schema: schema.clone(), + schema: Arc::clone(schema), }) as _], Some(&projection), vec![sort_exprs], @@ -400,3 +440,122 @@ pub fn stream_exec_ordered_with_projection( .unwrap(), ) } + +pub fn mock_data() -> Result> { + let schema = Arc::new(Schema::new(vec![ + Field::new("a", DataType::Int32, true), + Field::new("b", DataType::Int32, true), + ])); + + let batch = RecordBatch::try_new( + Arc::clone(&schema), + vec![ + Arc::new(Int32Array::from(vec![ + Some(1), + Some(2), + None, + Some(1), + Some(4), + Some(5), + ])), + Arc::new(Int32Array::from(vec![ + Some(1), + None, + Some(6), + Some(2), + Some(8), + Some(9), + ])), + ], + )?; + + Ok(Arc::new(MemoryExec::try_new( + &[vec![batch]], + Arc::clone(&schema), + None, + )?)) +} + +pub fn build_group_by(input_schema: &SchemaRef, columns: Vec) -> PhysicalGroupBy { + let mut group_by_expr: Vec<(Arc, String)> = vec![]; + for column in columns.iter() { + group_by_expr.push((col(column, input_schema).unwrap(), column.to_string())); + } + PhysicalGroupBy::new_single(group_by_expr.clone()) +} + +pub fn assert_plan_matches_expected( + plan: &Arc, + expected: &[&str], +) -> Result<()> { + let expected_lines: Vec<&str> = expected.to_vec(); + let config = ConfigOptions::new(); + + let optimized = + LimitedDistinctAggregation::new().optimize(Arc::clone(plan), &config)?; + + let optimized_result = displayable(optimized.as_ref()).indent(true).to_string(); + let actual_lines = trim_plan_display(&optimized_result); + + assert_eq!( + &expected_lines, &actual_lines, + "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", + expected_lines, actual_lines + ); + + Ok(()) +} + +/// Describe the type of aggregate being tested +pub enum TestAggregate { + /// Testing COUNT(*) type aggregates + CountStar, + + /// Testing for COUNT(column) aggregate + ColumnA(Arc), +} + +impl TestAggregate { + /// Create a new COUNT(*) aggregate + pub fn new_count_star() -> Self { + Self::CountStar + } + + /// Create a new COUNT(column) aggregate + pub fn new_count_column(schema: &Arc) -> Self { + Self::ColumnA(Arc::clone(schema)) + } + + /// Return appropriate expr depending if COUNT is for col or table (*) + pub fn count_expr(&self, schema: &Schema) -> AggregateFunctionExpr { + AggregateExprBuilder::new(count_udaf(), vec![self.column()]) + .schema(Arc::new(schema.clone())) + .alias(self.column_name()) + .build() + .unwrap() + } + + /// what argument would this aggregate need in the plan? + fn column(&self) -> Arc { + match self { + Self::CountStar => expressions::lit(COUNT_STAR_EXPANSION), + Self::ColumnA(s) => col("a", s).unwrap(), + } + } + + /// What name would this aggregate produce in a plan? + pub fn column_name(&self) -> &'static str { + match self { + Self::CountStar => "COUNT(*)", + Self::ColumnA(_) => "COUNT(a)", + } + } + + /// What is the expected count? + pub fn expected_count(&self) -> i64 { + match self { + TestAggregate::CountStar => 3, + TestAggregate::ColumnA(_) => 2, + } + } +} diff --git a/datafusion/physical-optimizer/src/utils.rs b/datafusion/physical-optimizer/src/utils.rs index c800cb9782d2..66596174cdf6 100644 --- a/datafusion/physical-optimizer/src/utils.rs +++ b/datafusion/physical-optimizer/src/utils.rs @@ -28,13 +28,7 @@ use datafusion_physical_plan::{ExecutionPlan, ExecutionPlanProperties}; use datafusion_physical_expr::LexRequirement; use datafusion_physical_expr_common::sort_expr::LexOrdering; 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. diff --git a/datafusion/physical-optimizer/tests/enforce_sorting.rs b/datafusion/physical-optimizer/tests/enforce_sorting.rs index 0b6ed1398de7..12b2a993d330 100644 --- a/datafusion/physical-optimizer/tests/enforce_sorting.rs +++ b/datafusion/physical-optimizer/tests/enforce_sorting.rs @@ -15,57 +15,27 @@ // specific language governing permissions and limitations // under the License. -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_expr::expressions::{col, NotExpr}; use datafusion_physical_optimizer::PhysicalOptimizerRule; +use datafusion_physical_plan::displayable; + use datafusion_physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; -use crate::assert_optimized; -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::config::ConfigOptions; use datafusion_common::tree_node::{TreeNode, TransformedResult}; -use crate::test_utils::{aggregate_exec, bounded_window_exec, coalesce_batches_exec, filter_exec, hash_join_exec, memory_exec, repartition_exec, sort_exec, sort_expr, sort_expr_options, sort_preserving_merge_exec, union_exec}; - -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) -} +use datafusion_physical_optimizer::enforce_distribution::EnforceDistribution; +use datafusion_physical_optimizer::test_utils::{check_integrity,aggregate_exec, bounded_window_exec, coalesce_batches_exec, create_test_schema, create_test_schema3, filter_exec, memory_exec, repartition_exec, sort_exec, sort_expr, sort_expr_options, sort_preserving_merge_exec, stream_exec_ordered, union_exec, RequirementsTestExec}; /// Runs the sort enforcement optimizer and asserts the plan /// against the original and expected plans @@ -92,7 +62,7 @@ macro_rules! assert_optimized { .and_then(check_integrity)?; // TODO: End state payloads will be checked here. - let new_plan = if repartition_sorts.optimizer.repartition_sorts { + let new_plan = if config.optimizer.repartition_sorts { let plan_with_coalesce_partitions = PlanWithCorrespondingCoalescePartitions::new_default(adjusted.plan); let parallel = plan_with_coalesce_partitions @@ -112,7 +82,7 @@ macro_rules! assert_optimized { plan_with_pipeline_fixer, false, true, - repartition_sorts, + &config, ) }) .data() @@ -142,7 +112,7 @@ macro_rules! assert_optimized { // Run the actual optimizer let optimized_physical_plan = - EnforceSorting::new().optimize(physical_plan, repartition_sorts)?; + EnforceSorting::new().optimize(physical_plan, &config)?; // Get string representation of the plan let actual = get_plan_string(&optimized_physical_plan); @@ -663,41 +633,6 @@ async fn test_multiple_sort_window_exec() -> Result<()> { 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 @@ -733,7 +668,7 @@ async fn test_commutativity() -> Result<()> { Arc::new(EnforceSorting::new()) as Arc, ]; for rule in rules { - plan = rule.optimize(plan, config)?; + plan = rule.optimize(plan, &config)?; } let first_plan = plan.clone(); @@ -744,7 +679,7 @@ async fn test_commutativity() -> Result<()> { Arc::new(EnforceSorting::new()) as Arc, ]; for rule in rules { - plan = rule.optimize(plan, config)?; + plan = rule.optimize(plan, &config)?; } let second_plan = plan.clone(); @@ -790,231 +725,6 @@ async fn test_coalesce_propagate() -> Result<()> { 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()?; @@ -1043,52 +753,6 @@ async fn test_replace_with_partial_sort2() -> Result<()> { 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 @@ -1152,3 +816,49 @@ async fn test_push_with_required_input_ordering_allowed() -> Result<()> { assert_optimized!(expected_input, expected, plan, true); Ok(()) } + +#[tokio::test] +async fn test_replace_with_partial_sort() -> Result<()> { + let schema = create_test_schema3()?; + let input_sort_exprs = vec![sort_expr("a", &schema)]; + let unbounded_input = stream_exec_ordered(&schema, input_sort_exprs); + + let physical_plan = sort_exec( + vec![sort_expr("a", &schema), sort_expr("c", &schema)], + unbounded_input, + ); + + let expected_input = [ + "SortExec: expr=[a@0 ASC, c@2 ASC], preserve_partitioning=[false]", + " StreamingTableExec: partition_sizes=1, projection=[a, b, c, d, e], infinite_source=true, output_ordering=[a@0 ASC]" + ]; + let expected_optimized = [ + "PartialSortExec: expr=[a@0 ASC, c@2 ASC], common_prefix_length=[1]", + " StreamingTableExec: partition_sizes=1, projection=[a, b, c, d, e], infinite_source=true, output_ordering=[a@0 ASC]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan, true); + Ok(()) +} + +#[tokio::test] +async fn test_not_replaced_with_partial_sort_for_unbounded_input() -> Result<()> { + let schema = create_test_schema3()?; + let input_sort_exprs = vec![sort_expr("b", &schema), sort_expr("c", &schema)]; + let unbounded_input = stream_exec_ordered(&schema, input_sort_exprs); + + let physical_plan = sort_exec( + vec![ + sort_expr("a", &schema), + sort_expr("b", &schema), + sort_expr("c", &schema), + ], + unbounded_input, + ); + let expected_input = [ + "SortExec: expr=[a@0 ASC, b@1 ASC, c@2 ASC], preserve_partitioning=[false]", + " StreamingTableExec: partition_sizes=1, projection=[a, b, c, d, e], infinite_source=true, output_ordering=[b@1 ASC, c@2 ASC]" + ]; + let expected_no_change = expected_input; + assert_optimized!(expected_input, expected_no_change, physical_plan, true); + Ok(()) +} diff --git a/datafusion/physical-optimizer/tests/limited_distinct_aggregation.rs b/datafusion/physical-optimizer/tests/limited_distinct_aggregation.rs index 576edb199b7e..23ed1d7fc617 100644 --- a/datafusion/physical-optimizer/tests/limited_distinct_aggregation.rs +++ b/datafusion/physical-optimizer/tests/limited_distinct_aggregation.rs @@ -20,294 +20,20 @@ //! Note these tests are not in the same module as the optimizer pass because //! they rely on `ParquetExec` which is in the core crate. -use std::sync::Arc; - -use arrow::{ - array::Int32Array, - compute::SortOptions, - datatypes::{DataType, Field, Schema}, - record_batch::RecordBatch, - util::pretty::pretty_format_batches, -}; -use arrow_schema::SchemaRef; -use datafusion_common::config::ConfigOptions; +use arrow_schema::DataType; use datafusion_common::Result; -use datafusion_execution::config::SessionConfig; use datafusion_expr::Operator; -use datafusion_physical_expr::{ - expressions::{cast, col}, - PhysicalExpr, PhysicalSortExpr, -}; -use datafusion_physical_expr_common::sort_expr::LexOrdering; -use datafusion_physical_optimizer::{ - limited_distinct_aggregation::LimitedDistinctAggregation, PhysicalOptimizerRule, +use datafusion_physical_expr::expressions::{cast, col}; +use datafusion_physical_optimizer::test_utils::{ + assert_plan_matches_expected, build_group_by, mock_data, TestAggregate, }; use datafusion_physical_plan::{ - aggregates::{AggregateExec, AggregateMode, PhysicalGroupBy}, - collect, displayable, expressions, - limit::{GlobalLimitExec, LocalLimitExec}, - memory::MemoryExec, + aggregates::{AggregateExec, AggregateMode}, + expressions, + limit::LocalLimitExec, ExecutionPlan, }; - -fn mock_data() -> Result> { - let schema = Arc::new(Schema::new(vec![ - Field::new("a", DataType::Int32, true), - Field::new("b", DataType::Int32, true), - ])); - - let batch = RecordBatch::try_new( - Arc::clone(&schema), - vec![ - Arc::new(Int32Array::from(vec![ - Some(1), - Some(2), - None, - Some(1), - Some(4), - Some(5), - ])), - Arc::new(Int32Array::from(vec![ - Some(1), - None, - Some(6), - Some(2), - Some(8), - Some(9), - ])), - ], - )?; - - Ok(Arc::new(MemoryExec::try_new( - &[vec![batch]], - Arc::clone(&schema), - None, - )?)) -} - -fn assert_plan_matches_expected( - plan: &Arc, - expected: &[&str], -) -> Result<()> { - let expected_lines: Vec<&str> = expected.to_vec(); - let config = ConfigOptions::new(); - - let optimized = - LimitedDistinctAggregation::new().optimize(Arc::clone(plan), config)?; - - let optimized_result = displayable(optimized.as_ref()).indent(true).to_string(); - let actual_lines = trim_plan_display(&optimized_result); - - assert_eq!( - &expected_lines, &actual_lines, - "\n\nexpected:\n\n{:#?}\nactual:\n\n{:#?}\n\n", - expected_lines, actual_lines - ); - - Ok(()) -} - -async fn assert_results_match_expected( - plan: Arc, - expected: &str, -) -> Result<()> { - let cfg = SessionConfig::new().with_target_partitions(1); - let ctx = SessionContext::new_with_config(cfg); - let batches = collect(plan, ctx.task_ctx()).await?; - let actual = format!("{}", pretty_format_batches(&batches)?); - assert_eq!(actual, expected); - Ok(()) -} - -pub fn build_group_by(input_schema: &SchemaRef, columns: Vec) -> PhysicalGroupBy { - let mut group_by_expr: Vec<(Arc, String)> = vec![]; - for column in columns.iter() { - group_by_expr.push((col(column, input_schema).unwrap(), column.to_string())); - } - PhysicalGroupBy::new_single(group_by_expr.clone()) -} - -#[tokio::test] -async fn test_partial_final() -> Result<()> { - let source = mock_data()?; - let schema = source.schema(); - - // `SELECT a FROM MemoryExec GROUP BY a LIMIT 4;`, Partial/Final AggregateExec - let partial_agg = AggregateExec::try_new( - AggregateMode::Partial, - build_group_by(&schema.clone(), vec!["a".to_string()]), - vec![], /* aggr_expr */ - vec![], /* filter_expr */ - source, /* input */ - schema.clone(), /* input_schema */ - )?; - let final_agg = AggregateExec::try_new( - AggregateMode::Final, - build_group_by(&schema.clone(), vec!["a".to_string()]), - vec![], /* aggr_expr */ - vec![], /* filter_expr */ - Arc::new(partial_agg), /* input */ - schema.clone(), /* input_schema */ - )?; - let limit_exec = LocalLimitExec::new( - Arc::new(final_agg), - 4, // fetch - ); - // expected to push the limit to the Partial and Final AggregateExecs - let expected = [ - "LocalLimitExec: fetch=4", - "AggregateExec: mode=Final, gby=[a@0 as a], aggr=[], lim=[4]", - "AggregateExec: mode=Partial, gby=[a@0 as a], aggr=[], lim=[4]", - "MemoryExec: partitions=1, partition_sizes=[1]", - ]; - let plan: Arc = Arc::new(limit_exec); - assert_plan_matches_expected(&plan, &expected)?; - let expected = r#" -+---+ -| a | -+---+ -| 1 | -| 2 | -| | -| 4 | -+---+ -"# - .trim(); - assert_results_match_expected(plan, expected).await?; - Ok(()) -} - -#[tokio::test] -async fn test_single_local() -> Result<()> { - let source = mock_data()?; - let schema = source.schema(); - - // `SELECT a FROM MemoryExec GROUP BY a LIMIT 4;`, Single AggregateExec - let single_agg = AggregateExec::try_new( - AggregateMode::Single, - build_group_by(&schema.clone(), vec!["a".to_string()]), - vec![], /* aggr_expr */ - vec![], /* filter_expr */ - source, /* input */ - schema.clone(), /* input_schema */ - )?; - let limit_exec = LocalLimitExec::new( - Arc::new(single_agg), - 4, // fetch - ); - // expected to push the limit to the AggregateExec - let expected = [ - "LocalLimitExec: fetch=4", - "AggregateExec: mode=Single, gby=[a@0 as a], aggr=[], lim=[4]", - "MemoryExec: partitions=1, partition_sizes=[1]", - ]; - let plan: Arc = Arc::new(limit_exec); - assert_plan_matches_expected(&plan, &expected)?; - let expected = r#" -+---+ -| a | -+---+ -| 1 | -| 2 | -| | -| 4 | -+---+ -"# - .trim(); - assert_results_match_expected(plan, expected).await?; - Ok(()) -} - -#[tokio::test] -async fn test_single_global() -> Result<()> { - let source = mock_data()?; - let schema = source.schema(); - - // `SELECT a FROM MemoryExec GROUP BY a LIMIT 4;`, Single AggregateExec - let single_agg = AggregateExec::try_new( - AggregateMode::Single, - build_group_by(&schema.clone(), vec!["a".to_string()]), - vec![], /* aggr_expr */ - vec![], /* filter_expr */ - source, /* input */ - schema.clone(), /* input_schema */ - )?; - let limit_exec = GlobalLimitExec::new( - Arc::new(single_agg), - 1, // skip - Some(3), // fetch - ); - // expected to push the skip+fetch limit to the AggregateExec - let expected = [ - "GlobalLimitExec: skip=1, fetch=3", - "AggregateExec: mode=Single, gby=[a@0 as a], aggr=[], lim=[4]", - "MemoryExec: partitions=1, partition_sizes=[1]", - ]; - let plan: Arc = Arc::new(limit_exec); - assert_plan_matches_expected(&plan, &expected)?; - let expected = r#" -+---+ -| a | -+---+ -| 2 | -| | -| 4 | -+---+ -"# - .trim(); - assert_results_match_expected(plan, expected).await?; - Ok(()) -} - -#[tokio::test] -async fn test_distinct_cols_different_than_group_by_cols() -> Result<()> { - let source = mock_data()?; - let schema = source.schema(); - - // `SELECT distinct a FROM MemoryExec GROUP BY a, b LIMIT 4;`, Single/Single AggregateExec - let group_by_agg = AggregateExec::try_new( - AggregateMode::Single, - build_group_by(&schema.clone(), vec!["a".to_string(), "b".to_string()]), - vec![], /* aggr_expr */ - vec![], /* filter_expr */ - source, /* input */ - schema.clone(), /* input_schema */ - )?; - let distinct_agg = AggregateExec::try_new( - AggregateMode::Single, - build_group_by(&schema.clone(), vec!["a".to_string()]), - vec![], /* aggr_expr */ - vec![], /* filter_expr */ - Arc::new(group_by_agg), /* input */ - schema.clone(), /* input_schema */ - )?; - let limit_exec = LocalLimitExec::new( - Arc::new(distinct_agg), - 4, // fetch - ); - // expected to push the limit to the outer AggregateExec only - let expected = [ - "LocalLimitExec: fetch=4", - "AggregateExec: mode=Single, gby=[a@0 as a], aggr=[], lim=[4]", - "AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[]", - "MemoryExec: partitions=1, partition_sizes=[1]", - ]; - let plan: Arc = Arc::new(limit_exec); - assert_plan_matches_expected(&plan, &expected)?; - let expected = r#" -+---+ -| a | -+---+ -| 1 | -| 2 | -| | -| 4 | -+---+ -"# - .trim(); - assert_results_match_expected(plan, expected).await?; - Ok(()) -} +use std::sync::Arc; #[test] fn test_no_group_by() -> Result<()> { @@ -405,37 +131,3 @@ fn test_has_filter() -> Result<()> { assert_plan_matches_expected(&plan, &expected)?; Ok(()) } - -#[test] -fn test_has_order_by() -> Result<()> { - let sort_key = LexOrdering::new(vec![PhysicalSortExpr { - expr: col("a", &schema()).unwrap(), - options: SortOptions::default(), - }]); - let source = parquet_exec_with_sort(vec![sort_key]); - let schema = source.schema(); - - // `SELECT a FROM MemoryExec WHERE a > 1 GROUP BY a LIMIT 10;`, Single AggregateExec - // the `a > 1` filter is applied in the AggregateExec - let single_agg = AggregateExec::try_new( - AggregateMode::Single, - build_group_by(&schema, vec!["a".to_string()]), - vec![], /* aggr_expr */ - vec![], /* filter_expr */ - source, /* input */ - schema, /* input_schema */ - )?; - let limit_exec = LocalLimitExec::new( - Arc::new(single_agg), - 10, // fetch - ); - // expected not to push the limit to the AggregateExec - let expected = [ - "LocalLimitExec: fetch=10", - "AggregateExec: mode=Single, gby=[a@0 as a], aggr=[], ordering_mode=Sorted", - "ParquetExec: file_groups={1 group: [[x]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", - ]; - let plan: Arc = Arc::new(limit_exec); - assert_plan_matches_expected(&plan, &expected)?; - Ok(()) -} diff --git a/datafusion/physical-optimizer/tests/mod.rs b/datafusion/physical-optimizer/tests/mod.rs index e2e89f5d438b..0e5eab515590 100644 --- a/datafusion/physical-optimizer/tests/mod.rs +++ b/datafusion/physical-optimizer/tests/mod.rs @@ -17,6 +17,4 @@ mod enforce_sorting; mod limited_distinct_aggregation; -mod replace_with_order_preserving_variants; mod sanity_checker; -mod test_utils; diff --git a/datafusion/physical-optimizer/tests/sanity_checker.rs b/datafusion/physical-optimizer/tests/sanity_checker.rs index 538f0e443ddb..b9bee41229e1 100644 --- a/datafusion/physical-optimizer/tests/sanity_checker.rs +++ b/datafusion/physical-optimizer/tests/sanity_checker.rs @@ -20,9 +20,6 @@ //! Note these tests are not in the same module as the optimizer pass because //! they rely on `ParquetExec` which is in the core crate. -use crate::physical_optimizer::test_util::{ - BinaryTestCase, QueryCase, SourceType, UnaryTestCase, -}; use arrow::compute::SortOptions; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use datafusion_common::config::ConfigOptions; @@ -69,229 +66,6 @@ fn assert_plan(plan: &dyn ExecutionPlan, expected_lines: Vec<&str>) { assert_eq!(actual_lines, expected_lines); } -#[tokio::test] -async fn test_hash_left_join_swap() -> Result<()> { - let test1 = BinaryTestCase { - source_types: (SourceType::Unbounded, SourceType::Bounded), - expect_fail: false, - }; - - let test2 = BinaryTestCase { - source_types: (SourceType::Bounded, SourceType::Unbounded), - // Left join for bounded build side and unbounded probe side can generate - // both incremental matched rows and final non-matched rows. - expect_fail: false, - }; - let test3 = BinaryTestCase { - source_types: (SourceType::Bounded, SourceType::Bounded), - expect_fail: false, - }; - let case = QueryCase { - sql: "SELECT t2.c1 FROM left as t1 LEFT JOIN right as t2 ON t1.c1 = t2.c1" - .to_string(), - cases: vec![Arc::new(test1), Arc::new(test2), Arc::new(test3)], - error_operator: "operator: HashJoinExec".to_string(), - }; - - case.run().await?; - Ok(()) -} - -#[tokio::test] -async fn test_hash_right_join_swap() -> Result<()> { - let test1 = BinaryTestCase { - source_types: (SourceType::Unbounded, SourceType::Bounded), - expect_fail: true, - }; - let test2 = BinaryTestCase { - source_types: (SourceType::Bounded, SourceType::Unbounded), - expect_fail: false, - }; - let test3 = BinaryTestCase { - source_types: (SourceType::Bounded, SourceType::Bounded), - expect_fail: false, - }; - let case = QueryCase { - sql: "SELECT t2.c1 FROM left as t1 RIGHT JOIN right as t2 ON t1.c1 = t2.c1" - .to_string(), - cases: vec![Arc::new(test1), Arc::new(test2), Arc::new(test3)], - error_operator: "operator: HashJoinExec".to_string(), - }; - - case.run().await?; - Ok(()) -} - -#[tokio::test] -async fn test_hash_inner_join_swap() -> Result<()> { - let test1 = BinaryTestCase { - source_types: (SourceType::Unbounded, SourceType::Bounded), - expect_fail: false, - }; - let test2 = BinaryTestCase { - source_types: (SourceType::Bounded, SourceType::Unbounded), - expect_fail: false, - }; - let test3 = BinaryTestCase { - source_types: (SourceType::Bounded, SourceType::Bounded), - expect_fail: false, - }; - let case = QueryCase { - sql: "SELECT t2.c1 FROM left as t1 JOIN right as t2 ON t1.c1 = t2.c1".to_string(), - cases: vec![Arc::new(test1), Arc::new(test2), Arc::new(test3)], - error_operator: "Join Error".to_string(), - }; - - case.run().await?; - Ok(()) -} - -#[tokio::test] -async fn test_hash_full_outer_join_swap() -> Result<()> { - let test1 = BinaryTestCase { - source_types: (SourceType::Unbounded, SourceType::Bounded), - expect_fail: true, - }; - let test2 = BinaryTestCase { - source_types: (SourceType::Bounded, SourceType::Unbounded), - // Full join for bounded build side and unbounded probe side can generate - // both incremental matched rows and final non-matched rows. - expect_fail: false, - }; - let test3 = BinaryTestCase { - source_types: (SourceType::Bounded, SourceType::Bounded), - expect_fail: false, - }; - let case = QueryCase { - sql: "SELECT t2.c1 FROM left as t1 FULL JOIN right as t2 ON t1.c1 = t2.c1" - .to_string(), - cases: vec![Arc::new(test1), Arc::new(test2), Arc::new(test3)], - error_operator: "operator: HashJoinExec".to_string(), - }; - - case.run().await?; - Ok(()) -} - -#[tokio::test] -async fn test_aggregate() -> Result<()> { - let test1 = UnaryTestCase { - source_type: SourceType::Bounded, - expect_fail: false, - }; - let test2 = UnaryTestCase { - source_type: SourceType::Unbounded, - expect_fail: true, - }; - let case = QueryCase { - sql: "SELECT c1, MIN(c4) FROM test GROUP BY c1".to_string(), - cases: vec![Arc::new(test1), Arc::new(test2)], - error_operator: "operator: AggregateExec".to_string(), - }; - - case.run().await?; - Ok(()) -} - -#[tokio::test] -async fn test_window_agg_hash_partition() -> Result<()> { - let test1 = UnaryTestCase { - source_type: SourceType::Bounded, - expect_fail: false, - }; - let test2 = UnaryTestCase { - source_type: SourceType::Unbounded, - expect_fail: true, - }; - let case = QueryCase { - sql: "SELECT - c9, - SUM(c9) OVER(PARTITION BY c1 ORDER BY c9 ASC ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING) as sum1 - FROM test - LIMIT 5".to_string(), - cases: vec![Arc::new(test1), Arc::new(test2)], - error_operator: "operator: SortExec".to_string() - }; - - case.run().await?; - Ok(()) -} - -#[tokio::test] -async fn test_window_agg_single_partition() -> Result<()> { - let test1 = UnaryTestCase { - source_type: SourceType::Bounded, - expect_fail: false, - }; - let test2 = UnaryTestCase { - source_type: SourceType::Unbounded, - expect_fail: true, - }; - let case = QueryCase { - sql: "SELECT - c9, - SUM(c9) OVER(ORDER BY c9 ASC ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING) as sum1 - FROM test".to_string(), - cases: vec![Arc::new(test1), Arc::new(test2)], - error_operator: "operator: SortExec".to_string() - }; - case.run().await?; - Ok(()) -} - -#[tokio::test] -async fn test_hash_cross_join() -> Result<()> { - let test1 = BinaryTestCase { - source_types: (SourceType::Unbounded, SourceType::Bounded), - expect_fail: true, - }; - let test2 = BinaryTestCase { - source_types: (SourceType::Unbounded, SourceType::Unbounded), - expect_fail: true, - }; - let test3 = BinaryTestCase { - source_types: (SourceType::Bounded, SourceType::Unbounded), - expect_fail: true, - }; - let test4 = BinaryTestCase { - source_types: (SourceType::Bounded, SourceType::Bounded), - expect_fail: false, - }; - let case = QueryCase { - sql: "SELECT t2.c1 FROM left as t1 CROSS JOIN right as t2".to_string(), - cases: vec![ - Arc::new(test1), - Arc::new(test2), - Arc::new(test3), - Arc::new(test4), - ], - error_operator: "operator: CrossJoinExec".to_string(), - }; - - case.run().await?; - Ok(()) -} - -#[tokio::test] -async fn test_analyzer() -> Result<()> { - let test1 = UnaryTestCase { - source_type: SourceType::Bounded, - expect_fail: false, - }; - let test2 = UnaryTestCase { - source_type: SourceType::Unbounded, - expect_fail: false, - }; - let case = QueryCase { - sql: "EXPLAIN ANALYZE SELECT * FROM test".to_string(), - cases: vec![Arc::new(test1), Arc::new(test2)], - error_operator: "Analyze Error".to_string(), - }; - - case.run().await?; - Ok(()) -} - #[tokio::test] /// Tests that plan is valid when the sort requirements are satisfied. async fn test_bounded_window_agg_sort_requirement() -> Result<()> { From 20d47d3d28b165ca68bd48db9610ea639aaab555 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Wed, 22 Jan 2025 17:05:09 +0300 Subject: [PATCH 19/24] Fix failing tests --- .../combine_partial_final_agg.rs | 6 +- .../enforce_distribution.rs | 95 ++++++------------- .../core/tests/physical_optimizer/mod.rs | 3 + .../replace_with_order_preserving_variants.rs | 5 +- .../src/enforce_distribution.rs | 20 ++-- 5 files changed, 49 insertions(+), 80 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/combine_partial_final_agg.rs b/datafusion/core/tests/physical_optimizer/combine_partial_final_agg.rs index 28761fff891a..0f832f44e92d 100644 --- a/datafusion/core/tests/physical_optimizer/combine_partial_final_agg.rs +++ b/datafusion/core/tests/physical_optimizer/combine_partial_final_agg.rs @@ -24,11 +24,11 @@ use std::sync::Arc; +use crate::physical_optimizer::parquet_exec; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; -use datafusion::datasource::listing::PartitionedFile; -use datafusion::datasource::physical_plan::{FileScanConfig, ParquetExec}; +use datafusion::physical_optimizer::combine_partial_final_agg::CombinePartialFinalAggregate; +use datafusion::physical_optimizer::test_utils::trim_plan_display; use datafusion_common::config::ConfigOptions; -use datafusion_execution::object_store::ObjectStoreUrl; use datafusion_functions_aggregate::count::count_udaf; use datafusion_functions_aggregate::sum::sum_udaf; use datafusion_physical_expr::aggregate::{AggregateExprBuilder, AggregateFunctionExpr}; diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index e1b6ff8bf482..57219fea26fb 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -19,53 +19,47 @@ use std::fmt::Debug; use std::ops::Deref; use std::sync::Arc; +use crate::physical_optimizer::parquet_exec_with_sort; +use arrow::compute::SortOptions; use datafusion::config::ConfigOptions; use datafusion::datasource::file_format::file_compression_type::FileCompressionType; use datafusion::datasource::listing::PartitionedFile; use datafusion::datasource::object_store::ObjectStoreUrl; use datafusion::datasource::physical_plan::{CsvExec, FileScanConfig, ParquetExec}; -use datafusion_physical_expr::PhysicalExpr; -use datafusion_physical_optimizer::enforce_sorting::EnforceSorting; -use datafusion_physical_optimizer::output_requirements::OutputRequirements; -use datafusion_physical_optimizer::test_utils::{ - check_integrity, coalesce_partitions_exec, repartition_exec, -}; -use datafusion_physical_plan::coalesce_batches::CoalesceBatchesExec; -use datafusion_physical_plan::expressions::col; -use datafusion_physical_plan::filter::FilterExec; -use datafusion_physical_plan::joins::utils::JoinOn; -use datafusion_physical_plan::joins::{HashJoinExec, PartitionMode, SortMergeJoinExec}; -use datafusion_physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; -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::{displayable, DisplayAs, DisplayFormatType, Statistics}; -use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; +use datafusion_common::error::Result; +use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_common::ScalarValue; use datafusion_expr::{JoinType, Operator}; use datafusion_physical_expr::expressions::{BinaryExpr, Column, Literal}; +use datafusion_physical_expr::PhysicalExpr; use datafusion_physical_expr::{ expressions::binary, expressions::lit, LexOrdering, PhysicalSortExpr, }; use datafusion_physical_expr_common::sort_expr::LexRequirement; use datafusion_physical_optimizer::enforce_distribution::*; +use datafusion_physical_optimizer::enforce_sorting::EnforceSorting; +use datafusion_physical_optimizer::output_requirements::OutputRequirements; +use datafusion_physical_optimizer::test_utils::{ + check_integrity, coalesce_partitions_exec, repartition_exec, schema, + sort_merge_join_exec, sort_preserving_merge_exec, +}; use datafusion_physical_optimizer::PhysicalOptimizerRule; use datafusion_physical_plan::aggregates::{ AggregateExec, AggregateMode, PhysicalGroupBy, }; +use datafusion_physical_plan::coalesce_batches::CoalesceBatchesExec; use datafusion_physical_plan::execution_plan::ExecutionPlan; +use datafusion_physical_plan::expressions::col; +use datafusion_physical_plan::filter::FilterExec; +use datafusion_physical_plan::joins::utils::JoinOn; +use datafusion_physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; use datafusion_physical_plan::projection::ProjectionExec; -use datafusion_physical_plan::PlanProperties; -use datafusion_common::error::Result; -use arrow::compute::SortOptions; -use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; +use datafusion_physical_plan::sorts::sort::SortExec; +use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; +use datafusion_physical_plan::union::UnionExec; use datafusion_physical_plan::ExecutionPlanProperties; - -type DistributionContext = PlanContext; - -/// Keeps track of parent required key orderings. -type PlanWithKeyRequirements = PlanContext>>; +use datafusion_physical_plan::PlanProperties; +use datafusion_physical_plan::{displayable, DisplayAs, DisplayFormatType, Statistics}; /// Models operators like BoundedWindowExec that require an input /// ordering but is easy to construct @@ -298,39 +292,14 @@ fn hash_join_exec( join_on: &JoinOn, join_type: &JoinType, ) -> Arc { - Arc::new( - HashJoinExec::try_new( - left, - right, - join_on.clone(), - None, - join_type, - None, - PartitionMode::Partitioned, - false, - ) - .unwrap(), - ) -} - -fn sort_merge_join_exec( - left: Arc, - right: Arc, - join_on: &JoinOn, - join_type: &JoinType, -) -> Arc { - Arc::new( - SortMergeJoinExec::try_new( - left, - right, - join_on.clone(), - None, - *join_type, - vec![SortOptions::default(); join_on.len()], - false, - ) - .unwrap(), + datafusion_physical_optimizer::test_utils::hash_join_exec( + left, + right, + join_on.clone(), + None, + join_type, ) + .unwrap() } fn filter_exec(input: Arc) -> Arc { @@ -341,6 +310,7 @@ fn filter_exec(input: Arc) -> Arc { )); Arc::new(FilterExec::try_new(predicate, input).unwrap()) } + fn sort_exec( sort_exprs: LexOrdering, input: Arc, @@ -351,13 +321,6 @@ fn sort_exec( Arc::new(new_sort) } -fn sort_preserving_merge_exec( - sort_exprs: LexOrdering, - input: Arc, -) -> Arc { - Arc::new(SortPreservingMergeExec::new(sort_exprs, input)) -} - fn limit_exec(input: Arc) -> Arc { Arc::new(GlobalLimitExec::new( Arc::new(LocalLimitExec::new(input, 100)), diff --git a/datafusion/core/tests/physical_optimizer/mod.rs b/datafusion/core/tests/physical_optimizer/mod.rs index 4e2827d4750b..a621684b4cfc 100644 --- a/datafusion/core/tests/physical_optimizer/mod.rs +++ b/datafusion/core/tests/physical_optimizer/mod.rs @@ -17,8 +17,11 @@ //! Physical Optimizer integration tests +mod combine_partial_final_agg; +mod enforce_distribution; mod enforce_sorting; mod limited_distinct_aggregation; +mod replace_with_order_preserving_variants; mod sanity_checker; use arrow_schema::SchemaRef; diff --git a/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs b/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs index c0a202fcfd02..3a4ad097db73 100644 --- a/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs +++ b/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs @@ -17,6 +17,7 @@ use std::sync::Arc; +use datafusion::prelude::SessionContext; use arrow::array::{ArrayRef, Int32Array}; use arrow::compute::SortOptions; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; @@ -43,6 +44,8 @@ use datafusion_common::tree_node::{TransformedResult, TreeNode}; use datafusion_physical_optimizer::enforce_sorting::replace_with_order_preserving_variants::{replace_with_order_preserving_variants, OrderPreservationContext}; use datafusion_common::config::ConfigOptions; +use object_store::memory::InMemory; +use object_store::ObjectStore; use rstest::rstest; use url::Url; @@ -149,7 +152,7 @@ macro_rules! assert_optimized { let mut config = ConfigOptions::new(); config.optimizer.prefer_existing_sort=$PREFER_EXISTING_SORT; let plan_with_pipeline_fixer = OrderPreservationContext::new_default(physical_plan); - let parallel = plan_with_pipeline_fixer.transform_up(|plan_with_pipeline_fixer| replace_with_order_preserving_variants(plan_with_pipeline_fixer, false, false, config.options())).data().and_then(check_integrity)?; + let parallel = plan_with_pipeline_fixer.transform_up(|plan_with_pipeline_fixer| replace_with_order_preserving_variants(plan_with_pipeline_fixer, false, false, &config)).data().and_then(check_integrity)?; let optimized_physical_plan = parallel.plan; // Get string representation of the plan diff --git a/datafusion/physical-optimizer/src/enforce_distribution.rs b/datafusion/physical-optimizer/src/enforce_distribution.rs index 611106941d59..c88eba7d64d3 100644 --- a/datafusion/physical-optimizer/src/enforce_distribution.rs +++ b/datafusion/physical-optimizer/src/enforce_distribution.rs @@ -236,6 +236,15 @@ impl PhysicalOptimizerRule for EnforceDistribution { } } +#[derive(Debug, Clone)] +struct JoinKeyPairs { + left_keys: Vec>, + right_keys: Vec>, +} + +/// Keeps track of parent required key orderings. +pub type PlanWithKeyRequirements = PlanContext>>; + /// When the physical planner creates the Joins, the ordering of join keys is from the original query. /// That might not match with the output partitioning of the join node's children /// A Top-Down process will use this method to adjust children's output partitioning based on the parent key reordering requirements: @@ -1366,7 +1375,7 @@ pub fn ensure_distribution( /// `SortPreservingMergeExec`, `CoalescePartitionsExec`) and their ancestors. /// Using this information, we can optimize distribution of the plan if/when /// necessary. -type DistributionContext = PlanContext; +pub type DistributionContext = PlanContext; fn update_children(mut dist_context: DistributionContext) -> Result { for child_context in dist_context.children.iter_mut() { @@ -1400,12 +1409,3 @@ fn update_children(mut dist_context: DistributionContext) -> Result>, - right_keys: Vec>, -} - -/// Keeps track of parent required key orderings. -type PlanWithKeyRequirements = PlanContext>>; From b13ba8e949321233bdd62e048dc00b51cd8f7d6a Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Wed, 22 Jan 2025 17:26:34 +0300 Subject: [PATCH 20/24] Final pass --- .../combine_partial_final_agg.rs | 3 +- .../enforce_distribution.rs | 10 +-- .../physical_optimizer/enforce_sorting.rs | 85 +++++++++++++++++-- .../limited_distinct_aggregation.rs | 1 + .../core/tests/physical_optimizer/mod.rs | 80 ++--------------- .../replace_with_order_preserving_variants.rs | 17 ++-- .../physical_optimizer/sanity_checker.rs | 8 +- .../src/enforce_distribution.rs | 28 +++--- .../src/enforce_sorting/mod.rs | 22 ++--- .../replace_with_order_preserving_variants.rs | 4 +- .../src/enforce_sorting/sort_pushdown.rs | 4 +- datafusion/physical-optimizer/src/lib.rs | 1 - datafusion/physical-optimizer/src/utils.rs | 9 +- .../tests/enforce_sorting.rs | 3 - .../tests/limited_distinct_aggregation.rs | 6 +- .../tests/sanity_checker.rs | 6 +- 16 files changed, 137 insertions(+), 150 deletions(-) diff --git a/datafusion/core/tests/physical_optimizer/combine_partial_final_agg.rs b/datafusion/core/tests/physical_optimizer/combine_partial_final_agg.rs index 0f832f44e92d..85efebf2386a 100644 --- a/datafusion/core/tests/physical_optimizer/combine_partial_final_agg.rs +++ b/datafusion/core/tests/physical_optimizer/combine_partial_final_agg.rs @@ -20,11 +20,10 @@ //! Note these tests are not in the same module as the optimizer pass because //! they rely on `ParquetExec` which is in the core crate. -//! Tests for [`CombinePartialFinalAggregate`] physical optimizer rule - use std::sync::Arc; use crate::physical_optimizer::parquet_exec; + use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use datafusion::physical_optimizer::combine_partial_final_agg::CombinePartialFinalAggregate; use datafusion::physical_optimizer::test_utils::trim_plan_display; diff --git a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs index 57219fea26fb..ea75e7d0e5f4 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_distribution.rs @@ -20,6 +20,7 @@ use std::ops::Deref; use std::sync::Arc; use crate::physical_optimizer::parquet_exec_with_sort; + use arrow::compute::SortOptions; use datafusion::config::ConfigOptions; use datafusion::datasource::file_format::file_compression_type::FileCompressionType; @@ -39,6 +40,7 @@ use datafusion_physical_expr_common::sort_expr::LexRequirement; use datafusion_physical_optimizer::enforce_distribution::*; use datafusion_physical_optimizer::enforce_sorting::EnforceSorting; use datafusion_physical_optimizer::output_requirements::OutputRequirements; +use datafusion_physical_optimizer::test_utils::trim_plan_display; use datafusion_physical_optimizer::test_utils::{ check_integrity, coalesce_partitions_exec, repartition_exec, schema, sort_merge_join_exec, sort_preserving_merge_exec, @@ -63,7 +65,6 @@ use datafusion_physical_plan::{displayable, DisplayAs, DisplayFormatType, Statis /// Models operators like BoundedWindowExec that require an input /// ordering but is easy to construct -/// #[derive(Debug)] struct SortRequiredExec { input: Arc, @@ -340,13 +341,6 @@ fn sort_required_exec_with_req( Arc::new(SortRequiredExec::new_with_requirement(input, sort_exprs)) } -pub(crate) fn trim_plan_display(plan: &str) -> Vec<&str> { - plan.split('\n') - .map(|s| s.trim()) - .filter(|s| !s.is_empty()) - .collect() -} - fn ensure_distribution_helper( plan: Arc, target_partitions: usize, diff --git a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs index 61d4a8abfa10..4fe04753fd9c 100644 --- a/datafusion/core/tests/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/tests/physical_optimizer/enforce_sorting.rs @@ -15,17 +15,19 @@ // specific language governing permissions and limitations // under the License. -use datafusion_physical_plan::displayable; use std::sync::Arc; +use crate::physical_optimizer::parquet_exec; + +use datafusion_physical_plan::displayable; use arrow::compute::SortOptions; +use arrow_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_expr::Partitioning; -use datafusion_physical_expr_common::sort_expr::LexOrdering; +use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; use datafusion_physical_optimizer::enforce_sorting::{EnforceSorting,PlanWithCorrespondingCoalescePartitions,PlanWithCorrespondingSort,parallelize_sorts,ensure_sorting}; use datafusion_physical_optimizer::enforce_sorting::replace_with_order_preserving_variants::{replace_with_order_preserving_variants,OrderPreservationContext}; use datafusion_physical_optimizer::enforce_sorting::sort_pushdown::{SortPushDown, assign_initial_requirements, pushdown_sorts}; @@ -33,11 +35,84 @@ use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; use datafusion_physical_plan::repartition::RepartitionExec; use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; use datafusion_physical_plan::{get_plan_string, ExecutionPlan}; -use rstest::rstest; use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::{TreeNode, TransformedResult}; use datafusion_physical_optimizer::test_utils::{check_integrity,bounded_window_exec, coalesce_partitions_exec, create_test_schema, create_test_schema2, create_test_schema3, filter_exec, global_limit_exec, hash_join_exec, limit_exec, local_limit_exec, memory_exec, repartition_exec, sort_exec, sort_expr, sort_expr_options, sort_merge_join_exec, sort_preserving_merge_exec, spr_repartition_exec, stream_exec_ordered, union_exec}; -use crate::physical_optimizer::{csv_exec_ordered, csv_exec_sorted, parquet_exec, parquet_exec_sorted}; +use datafusion::datasource::physical_plan::{CsvExec, FileScanConfig, ParquetExec}; +use datafusion_execution::object_store::ObjectStoreUrl; +use datafusion::datasource::listing::PartitionedFile; +use datafusion::datasource::file_format::file_compression_type::FileCompressionType; + +use rstest::rstest; + +/// Create a csv exec for tests +fn csv_exec_ordered( + schema: &SchemaRef, + sort_exprs: impl IntoIterator, +) -> Arc { + let sort_exprs = sort_exprs.into_iter().collect(); + + Arc::new( + CsvExec::builder( + FileScanConfig::new( + ObjectStoreUrl::parse("test:///").unwrap(), + schema.clone(), + ) + .with_file(PartitionedFile::new("file_path".to_string(), 100)) + .with_output_ordering(vec![sort_exprs]), + ) + .with_has_header(true) + .with_delimeter(0) + .with_quote(b'"') + .with_escape(None) + .with_comment(None) + .with_newlines_in_values(false) + .with_file_compression_type(FileCompressionType::UNCOMPRESSED) + .build(), + ) +} + +/// 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() +} + +/// Create a sorted Csv exec +fn csv_exec_sorted( + schema: &SchemaRef, + sort_exprs: impl IntoIterator, +) -> Arc { + let sort_exprs = sort_exprs.into_iter().collect(); + + Arc::new( + CsvExec::builder( + FileScanConfig::new( + ObjectStoreUrl::parse("test:///").unwrap(), + schema.clone(), + ) + .with_file(PartitionedFile::new("x".to_string(), 100)) + .with_output_ordering(vec![sort_exprs]), + ) + .with_has_header(false) + .with_delimeter(0) + .with_quote(0) + .with_escape(None) + .with_comment(None) + .with_newlines_in_values(false) + .with_file_compression_type(FileCompressionType::UNCOMPRESSED) + .build(), + ) +} /// Runs the sort enforcement optimizer and asserts the plan /// against the original and expected plans diff --git a/datafusion/core/tests/physical_optimizer/limited_distinct_aggregation.rs b/datafusion/core/tests/physical_optimizer/limited_distinct_aggregation.rs index 774fdbc83349..4373dc40de38 100644 --- a/datafusion/core/tests/physical_optimizer/limited_distinct_aggregation.rs +++ b/datafusion/core/tests/physical_optimizer/limited_distinct_aggregation.rs @@ -20,6 +20,7 @@ use std::sync::Arc; use crate::physical_optimizer::parquet_exec_with_sort; + use arrow::{compute::SortOptions, util::pretty::pretty_format_batches}; use datafusion::prelude::SessionContext; use datafusion_common::Result; diff --git a/datafusion/core/tests/physical_optimizer/mod.rs b/datafusion/core/tests/physical_optimizer/mod.rs index a621684b4cfc..e4128d8fb8ef 100644 --- a/datafusion/core/tests/physical_optimizer/mod.rs +++ b/datafusion/core/tests/physical_optimizer/mod.rs @@ -24,15 +24,14 @@ mod limited_distinct_aggregation; mod replace_with_order_preserving_variants; mod sanity_checker; +use std::sync::Arc; + use arrow_schema::SchemaRef; -use datafusion::datasource::file_format::file_compression_type::FileCompressionType; use datafusion::datasource::listing::PartitionedFile; -use datafusion::datasource::physical_plan::{CsvExec, FileScanConfig, ParquetExec}; +use datafusion::datasource::physical_plan::{FileScanConfig, ParquetExec}; use datafusion_execution::object_store::ObjectStoreUrl; -use datafusion_physical_expr_common::sort_expr::{LexOrdering, PhysicalSortExpr}; +use datafusion_physical_expr_common::sort_expr::LexOrdering; use datafusion_physical_optimizer::test_utils::schema; -use datafusion_physical_plan::ExecutionPlan; -use std::sync::Arc; /// Create a non sorted parquet exec pub fn parquet_exec(schema: &SchemaRef) -> Arc { @@ -43,22 +42,7 @@ pub fn parquet_exec(schema: &SchemaRef) -> Arc { .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() -} - -/// create a single parquet file that is sorted +/// Create a single parquet file that is sorted pub(crate) fn parquet_exec_with_sort( output_ordering: Vec, ) -> Arc { @@ -69,57 +53,3 @@ pub(crate) fn parquet_exec_with_sort( ) .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(), - ) -} - -/// 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/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs b/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs index 3a4ad097db73..912683083738 100644 --- a/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs +++ b/datafusion/core/tests/physical_optimizer/replace_with_order_preserving_variants.rs @@ -36,14 +36,14 @@ use datafusion_physical_plan::{ displayable, get_plan_string, ExecutionPlan, Partitioning, }; 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, stream_exec_ordered_with_projection}; - use datafusion_physical_optimizer::enforce_sorting::replace_with_order_preserving_variants::{replace_with_order_preserving_variants, OrderPreservationContext}; - +use datafusion_common::Result; +use datafusion_expr::{JoinType, Operator}; +use datafusion_physical_expr::expressions::{self, col, Column}; +use datafusion_physical_expr::PhysicalSortExpr; +use datafusion_physical_optimizer::test_utils::{check_integrity, stream_exec_ordered_with_projection}; +use datafusion_physical_optimizer::enforce_sorting::replace_with_order_preserving_variants::{replace_with_order_preserving_variants, OrderPreservationContext}; use datafusion_common::config::ConfigOptions; + use object_store::memory::InMemory; use object_store::ObjectStore; use rstest::rstest; @@ -1118,9 +1118,6 @@ async fn test_with_multiple_child_trees( Ok(()) } -// End test cases -// Start test helpers - fn sort_expr(name: &str, schema: &Schema) -> PhysicalSortExpr { let sort_opts = SortOptions { nulls_first: false, diff --git a/datafusion/core/tests/physical_optimizer/sanity_checker.rs b/datafusion/core/tests/physical_optimizer/sanity_checker.rs index 58a11e2807d0..7636df9b3db9 100644 --- a/datafusion/core/tests/physical_optimizer/sanity_checker.rs +++ b/datafusion/core/tests/physical_optimizer/sanity_checker.rs @@ -15,11 +15,13 @@ // specific language governing permissions and limitations // under the License. -use async_trait::async_trait; +use std::sync::Arc; + use datafusion::datasource::stream::{FileStreamProvider, StreamConfig, StreamTable}; use datafusion::prelude::{CsvReadOptions, SessionContext}; use datafusion_common::Result; -use std::sync::Arc; + +use async_trait::async_trait; async fn register_current_csv( ctx: &SessionContext, @@ -75,6 +77,7 @@ impl SqlTestCase for UnaryTestCase { self.expect_fail } } + /// [BinaryTestCase] is designed for binary input [ExecutionPlan]s. pub struct BinaryTestCase { pub source_types: (SourceType, SourceType), @@ -117,6 +120,7 @@ impl QueryCase { } Ok(()) } + async fn run_case(&self, ctx: SessionContext, error: Option<&String>) -> Result<()> { let dataframe = ctx.sql(self.sql.as_str()).await?; let plan = dataframe.create_physical_plan().await; diff --git a/datafusion/physical-optimizer/src/enforce_distribution.rs b/datafusion/physical-optimizer/src/enforce_distribution.rs index c88eba7d64d3..2f37c672bcda 100644 --- a/datafusion/physical-optimizer/src/enforce_distribution.rs +++ b/datafusion/physical-optimizer/src/enforce_distribution.rs @@ -31,13 +31,23 @@ use crate::utils::{ is_sort_preserving_merge, }; +use arrow::compute::SortOptions; use datafusion_common::config::ConfigOptions; use datafusion_common::error::Result; - +use datafusion_common::stats::Precision; +use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; +use datafusion_expr::logical_plan::JoinType; +use datafusion_physical_expr::expressions::{Column, NoOp}; +use datafusion_physical_expr::utils::map_columns_before_projection; +use datafusion_physical_expr::{ + physical_exprs_equal, EquivalenceProperties, PhysicalExpr, PhysicalExprRef, +}; +use datafusion_physical_expr_common::sort_expr::LexOrdering; use datafusion_physical_plan::aggregates::{ AggregateExec, AggregateMode, PhysicalGroupBy, }; use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; +use datafusion_physical_plan::execution_plan::EmissionType; use datafusion_physical_plan::joins::{ CrossJoinExec, HashJoinExec, PartitionMode, SortMergeJoinExec, }; @@ -47,23 +57,9 @@ use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeE use datafusion_physical_plan::tree_node::PlanContext; use datafusion_physical_plan::union::{can_interleave, InterleaveExec, UnionExec}; use datafusion_physical_plan::windows::WindowAggExec; -use datafusion_physical_plan::{Distribution, ExecutionPlan, Partitioning}; - -use arrow::compute::SortOptions; -use datafusion_common::stats::Precision; -use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; -use datafusion_expr::logical_plan::JoinType; -use datafusion_physical_expr::expressions::{Column, NoOp}; -use datafusion_physical_expr::utils::map_columns_before_projection; -use datafusion_physical_expr::{ - physical_exprs_equal, EquivalenceProperties, PhysicalExpr, PhysicalExprRef, -}; -use datafusion_physical_expr_common::sort_expr::LexOrdering; -// use datafusion_physical_optimizer::output_requirements::OutputRequirementExec; -// use datafusion_physical_optimizer::PhysicalOptimizerRule; -use datafusion_physical_plan::execution_plan::EmissionType; use datafusion_physical_plan::windows::{get_best_fitting_window, BoundedWindowAggExec}; use datafusion_physical_plan::ExecutionPlanProperties; +use datafusion_physical_plan::{Distribution, ExecutionPlan, Partitioning}; use itertools::izip; diff --git a/datafusion/physical-optimizer/src/enforce_sorting/mod.rs b/datafusion/physical-optimizer/src/enforce_sorting/mod.rs index f4a79fbe859d..f098f16d9b84 100644 --- a/datafusion/physical-optimizer/src/enforce_sorting/mod.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/mod.rs @@ -37,20 +37,9 @@ pub mod replace_with_order_preserving_variants; pub mod sort_pushdown; -// mod utils; 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, }; @@ -62,14 +51,25 @@ use crate::utils::{ is_repartition, is_sort, is_sort_preserving_merge, is_union, is_window, }; use crate::PhysicalOptimizerRule; + use datafusion_common::config::ConfigOptions; +use datafusion_common::plan_err; +use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; +use datafusion_common::Result; +use datafusion_physical_expr::{Distribution, Partitioning}; +use datafusion_physical_expr_common::sort_expr::{LexOrdering, LexRequirement}; use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; +use datafusion_physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; +use datafusion_physical_plan::repartition::RepartitionExec; +use datafusion_physical_plan::sorts::partial_sort::PartialSortExec; use datafusion_physical_plan::sorts::sort::SortExec; use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; use datafusion_physical_plan::tree_node::PlanContext; use datafusion_physical_plan::windows::{ get_best_fitting_window, BoundedWindowAggExec, WindowAggExec, }; +use datafusion_physical_plan::{ExecutionPlan, ExecutionPlanProperties, InputOrderMode}; + use itertools::izip; /// This rule inspects [`SortExec`]'s in the given physical plan and removes the diff --git a/datafusion/physical-optimizer/src/enforce_sorting/replace_with_order_preserving_variants.rs b/datafusion/physical-optimizer/src/enforce_sorting/replace_with_order_preserving_variants.rs index 1f1a35efb251..c542f9261a24 100644 --- a/datafusion/physical-optimizer/src/enforce_sorting/replace_with_order_preserving_variants.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/replace_with_order_preserving_variants.rs @@ -31,11 +31,11 @@ use datafusion_common::Result; use datafusion_physical_expr_common::sort_expr::LexOrdering; use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; use datafusion_physical_plan::execution_plan::EmissionType; +use datafusion_physical_plan::repartition::RepartitionExec; +use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; use datafusion_physical_plan::tree_node::PlanContext; use datafusion_physical_plan::ExecutionPlanProperties; -use datafusion_physical_plan::repartition::RepartitionExec; -use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; use itertools::izip; /// For a given `plan`, this object carries the information one needs from its diff --git a/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs b/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs index d2f7bcacd089..166113807e37 100644 --- a/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs +++ b/datafusion/physical-optimizer/src/enforce_sorting/sort_pushdown.rs @@ -18,11 +18,11 @@ use std::fmt::Debug; use std::sync::Arc; -use arrow_schema::SchemaRef; - use crate::utils::{ add_sort_above, is_sort, is_sort_preserving_merge, is_union, is_window, }; + +use arrow_schema::SchemaRef; use datafusion_common::tree_node::{ ConcreteTreeNode, Transformed, TreeNode, TreeNodeRecursion, }; diff --git a/datafusion/physical-optimizer/src/lib.rs b/datafusion/physical-optimizer/src/lib.rs index 507cc6c75ad5..bcb87944f5fd 100644 --- a/datafusion/physical-optimizer/src/lib.rs +++ b/datafusion/physical-optimizer/src/lib.rs @@ -32,7 +32,6 @@ pub mod pruning; pub mod sanity_checker; pub mod topk_aggregation; pub mod update_aggr_exprs; - pub use optimizer::PhysicalOptimizerRule; pub mod test_utils; pub mod utils; diff --git a/datafusion/physical-optimizer/src/utils.rs b/datafusion/physical-optimizer/src/utils.rs index 66596174cdf6..636e78a06ce7 100644 --- a/datafusion/physical-optimizer/src/utils.rs +++ b/datafusion/physical-optimizer/src/utils.rs @@ -17,19 +17,18 @@ use std::sync::Arc; +use datafusion_physical_expr::LexRequirement; +use datafusion_physical_expr_common::sort_expr::LexOrdering; use datafusion_physical_plan::coalesce_partitions::CoalescePartitionsExec; +use datafusion_physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; use datafusion_physical_plan::repartition::RepartitionExec; 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}; -use datafusion_physical_expr::LexRequirement; -use datafusion_physical_expr_common::sort_expr::LexOrdering; -use datafusion_physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; -use datafusion_physical_plan::tree_node::PlanContext; - /// This utility function adds a `SortExec` above an operator according to the /// given ordering requirements while preserving the original partitioning. pub fn add_sort_above( diff --git a/datafusion/physical-optimizer/tests/enforce_sorting.rs b/datafusion/physical-optimizer/tests/enforce_sorting.rs index 12b2a993d330..df1a484e1585 100644 --- a/datafusion/physical-optimizer/tests/enforce_sorting.rs +++ b/datafusion/physical-optimizer/tests/enforce_sorting.rs @@ -22,9 +22,7 @@ use datafusion_common::Result; use datafusion_physical_expr::expressions::{col, NotExpr}; use datafusion_physical_optimizer::PhysicalOptimizerRule; use datafusion_physical_plan::displayable; - use datafusion_physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; - use datafusion_physical_expr_common::sort_expr::LexOrdering; use datafusion_physical_optimizer::enforce_sorting::{EnforceSorting,PlanWithCorrespondingCoalescePartitions,PlanWithCorrespondingSort,parallelize_sorts,ensure_sorting}; use datafusion_physical_optimizer::enforce_sorting::replace_with_order_preserving_variants::{replace_with_order_preserving_variants,OrderPreservationContext}; @@ -44,7 +42,6 @@ use datafusion_physical_optimizer::test_utils::{check_integrity,aggregate_exec, /// `$EXPECTED_OPTIMIZED_PLAN_LINES`: optimized plan /// `$PLAN`: the plan to optimized /// `REPARTITION_SORTS`: Flag to set `config.options.optimizer.repartition_sorts` option. -/// macro_rules! assert_optimized { ($EXPECTED_PLAN_LINES: expr, $EXPECTED_OPTIMIZED_PLAN_LINES: expr, $PLAN: expr, $REPARTITION_SORTS: expr) => { let mut config = ConfigOptions::new(); diff --git a/datafusion/physical-optimizer/tests/limited_distinct_aggregation.rs b/datafusion/physical-optimizer/tests/limited_distinct_aggregation.rs index 23ed1d7fc617..6427d4def1a8 100644 --- a/datafusion/physical-optimizer/tests/limited_distinct_aggregation.rs +++ b/datafusion/physical-optimizer/tests/limited_distinct_aggregation.rs @@ -16,9 +16,8 @@ // under the License. //! Tests for [`LimitedDistinctAggregation`] physical optimizer rule -//! -//! Note these tests are not in the same module as the optimizer pass because -//! they rely on `ParquetExec` which is in the core crate. + +use std::sync::Arc; use arrow_schema::DataType; use datafusion_common::Result; @@ -33,7 +32,6 @@ use datafusion_physical_plan::{ limit::LocalLimitExec, ExecutionPlan, }; -use std::sync::Arc; #[test] fn test_no_group_by() -> Result<()> { diff --git a/datafusion/physical-optimizer/tests/sanity_checker.rs b/datafusion/physical-optimizer/tests/sanity_checker.rs index b9bee41229e1..e21385cd9fd0 100644 --- a/datafusion/physical-optimizer/tests/sanity_checker.rs +++ b/datafusion/physical-optimizer/tests/sanity_checker.rs @@ -16,9 +16,8 @@ // under the License. //! Tests for [`SanityCheckPlan`] physical optimizer rule -//! -//! Note these tests are not in the same module as the optimizer pass because -//! they rely on `ParquetExec` which is in the core crate. + +use std::sync::Arc; use arrow::compute::SortOptions; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; @@ -35,7 +34,6 @@ use datafusion_physical_optimizer::{sanity_checker::*, PhysicalOptimizerRule}; use datafusion_physical_plan::displayable; use datafusion_physical_plan::repartition::RepartitionExec; use datafusion_physical_plan::ExecutionPlan; -use std::sync::Arc; fn create_test_schema() -> SchemaRef { Arc::new(Schema::new(vec![Field::new("c9", DataType::Int32, true)])) From 9e1dfbb2885755d0df0f271bd9da7064048c28d6 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Wed, 22 Jan 2025 17:47:38 +0300 Subject: [PATCH 21/24] Update datafusion-testing --- datafusion-testing | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion-testing b/datafusion-testing index 5b424aefd7f6..36283d195c72 160000 --- a/datafusion-testing +++ b/datafusion-testing @@ -1 +1 @@ -Subproject commit 5b424aefd7f6bf198220c37f59d39dbb25b47695 +Subproject commit 36283d195c728f26b16b517ba999fd62509b6649 From b7373fb4df6b06d7b742fd22d0f346a67c353c98 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Wed, 22 Jan 2025 17:54:27 +0300 Subject: [PATCH 22/24] Update test_utils.rs --- datafusion/physical-optimizer/src/test_utils.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/physical-optimizer/src/test_utils.rs b/datafusion/physical-optimizer/src/test_utils.rs index a4a88a576c8b..56830d3b3f89 100644 --- a/datafusion/physical-optimizer/src/test_utils.rs +++ b/datafusion/physical-optimizer/src/test_utils.rs @@ -27,8 +27,8 @@ use datafusion_common::tree_node::{Transformed, TransformedResult, TreeNode}; use datafusion_common::utils::expr::COUNT_STAR_EXPANSION; use datafusion_common::{JoinType, Result}; use datafusion_execution::{SendableRecordBatchStream, TaskContext}; -use datafusion_expr::test::function_stub::count_udaf; use datafusion_expr::{WindowFrame, WindowFunctionDefinition}; +use datafusion_functions_aggregate::count::count_udaf; use datafusion_physical_expr::aggregate::{AggregateExprBuilder, AggregateFunctionExpr}; use datafusion_physical_expr::expressions::col; use datafusion_physical_expr::{expressions, PhysicalExpr}; From a8fe5ca96b141a59323f24721c7d261214cdb6c0 Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Wed, 22 Jan 2025 18:08:56 +0300 Subject: [PATCH 23/24] fix the dep --- datafusion-cli/Cargo.lock | 1 + datafusion/physical-optimizer/Cargo.toml | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/datafusion-cli/Cargo.lock b/datafusion-cli/Cargo.lock index 86567f487c1b..ae9c82fa1249 100644 --- a/datafusion-cli/Cargo.lock +++ b/datafusion-cli/Cargo.lock @@ -1592,6 +1592,7 @@ dependencies = [ "datafusion-execution", "datafusion-expr", "datafusion-expr-common", + "datafusion-functions-aggregate", "datafusion-physical-expr", "datafusion-physical-expr-common", "datafusion-physical-plan", diff --git a/datafusion/physical-optimizer/Cargo.toml b/datafusion/physical-optimizer/Cargo.toml index c6c75a0e0939..07aff8a59e24 100644 --- a/datafusion/physical-optimizer/Cargo.toml +++ b/datafusion/physical-optimizer/Cargo.toml @@ -44,6 +44,7 @@ datafusion-expr-common = { workspace = true, default-features = true } datafusion-physical-expr = { workspace = true } datafusion-physical-expr-common = { workspace = true } datafusion-physical-plan = { workspace = true } +datafusion-functions-aggregate = { workspace = true } futures = { workspace = true } itertools = { workspace = true } log = { workspace = true } @@ -52,7 +53,6 @@ url = { workspace = true } [dev-dependencies] datafusion-expr = { workspace = true } -datafusion-functions-aggregate = { workspace = true } datafusion-functions-nested = { workspace = true } rstest = { workspace = true } tokio = { workspace = true } From 99a419f0e3721b26359482441b8685ea195c9baf Mon Sep 17 00:00:00 2001 From: berkaysynnada Date: Wed, 22 Jan 2025 18:09:39 +0300 Subject: [PATCH 24/24] Update Cargo.toml --- datafusion/physical-optimizer/Cargo.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/datafusion/physical-optimizer/Cargo.toml b/datafusion/physical-optimizer/Cargo.toml index 07aff8a59e24..a40827bda209 100644 --- a/datafusion/physical-optimizer/Cargo.toml +++ b/datafusion/physical-optimizer/Cargo.toml @@ -41,10 +41,10 @@ datafusion-common = { workspace = true, default-features = true } datafusion-execution = { workspace = true } datafusion-expr = { workspace = true } datafusion-expr-common = { workspace = true, default-features = true } +datafusion-functions-aggregate = { workspace = true } datafusion-physical-expr = { workspace = true } datafusion-physical-expr-common = { workspace = true } datafusion-physical-plan = { workspace = true } -datafusion-functions-aggregate = { workspace = true } futures = { workspace = true } itertools = { workspace = true } log = { workspace = true }