From 54131304a8f33d98181c25ce6fddc7f2d129c796 Mon Sep 17 00:00:00 2001 From: "mingmwang@ebay.com" Date: Fri, 10 Feb 2023 17:17:52 +0800 Subject: [PATCH 01/13] Top Down Sort Enforer --- datafusion/core/src/execution/context.rs | 5 +- .../physical_optimizer/dist_enforcement.rs | 141 +- datafusion/core/src/physical_optimizer/mod.rs | 1 + .../src/physical_optimizer/repartition.rs | 56 +- .../physical_optimizer/sort_enforcement.rs | 51 +- .../physical_optimizer/sort_enforcement2.rs | 2101 +++++++++++++++++ .../core/src/physical_optimizer/utils.rs | 5 +- .../physical_plan/joins/sort_merge_join.rs | 18 +- datafusion/core/src/physical_plan/mod.rs | 4 +- datafusion/core/src/physical_plan/planner.rs | 1 + .../core/src/physical_plan/sorts/sort.rs | 7 +- .../sorts/sort_preserving_merge.rs | 15 +- datafusion/core/src/physical_plan/union.rs | 19 +- .../windows/bounded_window_agg_exec.rs | 33 +- .../physical_plan/windows/window_agg_exec.rs | 29 +- datafusion/core/tests/sql/explain_analyze.rs | 2 +- datafusion/core/tests/sql/joins.rs | 24 +- datafusion/core/tests/sql/window.rs | 71 +- datafusion/physical-expr/src/lib.rs | 3 +- datafusion/physical-expr/src/sort_expr.rs | 52 + datafusion/physical-expr/src/utils.rs | 217 +- 21 files changed, 2653 insertions(+), 202 deletions(-) create mode 100644 datafusion/core/src/physical_optimizer/sort_enforcement2.rs diff --git a/datafusion/core/src/execution/context.rs b/datafusion/core/src/execution/context.rs index 99a49d04da7de..eac60c5ecb5dd 100644 --- a/datafusion/core/src/execution/context.rs +++ b/datafusion/core/src/execution/context.rs @@ -97,7 +97,7 @@ use crate::execution::memory_pool::MemoryPool; use crate::physical_optimizer::global_sort_selection::GlobalSortSelection; use crate::physical_optimizer::pipeline_checker::PipelineChecker; use crate::physical_optimizer::pipeline_fixer::PipelineFixer; -use crate::physical_optimizer::sort_enforcement::EnforceSorting; +use crate::physical_optimizer::sort_enforcement2::TopDownEnforceSorting; use datafusion_optimizer::OptimizerConfig; use datafusion_sql::planner::object_name_to_table_reference; use uuid::Uuid; @@ -1068,6 +1068,7 @@ impl QueryPlanner for DefaultQueryPlanner { session_state: &SessionState, ) -> Result> { let planner = DefaultPhysicalPlanner::default(); + println!("optimized logical plan {:?}", logical_plan); planner .create_physical_plan(logical_plan, session_state) .await @@ -1487,7 +1488,7 @@ impl SessionState { // ordering. Please make sure that the whole plan tree is determined before this rule. // Note that one should always run this rule after running the EnforceDistribution rule // as the latter may break local sorting requirements. - Arc::new(EnforceSorting::new()), + Arc::new(TopDownEnforceSorting::new()), // The CoalesceBatches rule will not influence the distribution and ordering of the // whole plan tree. Therefore, to avoid influencing other rules, it should run last. Arc::new(CoalesceBatches::new()), diff --git a/datafusion/core/src/physical_optimizer/dist_enforcement.rs b/datafusion/core/src/physical_optimizer/dist_enforcement.rs index c6c2bd40e39f7..4fe76ac30d94c 100644 --- a/datafusion/core/src/physical_optimizer/dist_enforcement.rs +++ b/datafusion/core/src/physical_optimizer/dist_enforcement.rs @@ -38,11 +38,11 @@ use datafusion_expr::logical_plan::JoinType; use datafusion_physical_expr::equivalence::EquivalenceProperties; use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::expressions::NoOp; +use datafusion_physical_expr::utils::map_columns_before_projection; use datafusion_physical_expr::{ expr_list_eq_strict_order, normalize_expr_with_equivalence_properties, AggregateExpr, PhysicalExpr, }; -use std::collections::HashMap; use std::sync::Arc; /// The EnforceDistribution rule ensures that distribution requirements are met @@ -492,30 +492,6 @@ fn reorder_aggregate_keys( } } -fn map_columns_before_projection( - parent_required: &[Arc], - proj_exprs: &[(Arc, String)], -) -> Vec> { - let mut column_mapping = HashMap::new(); - for (expression, name) in proj_exprs.iter() { - if let Some(column) = expression.as_any().downcast_ref::() { - column_mapping.insert(name.clone(), column.clone()); - }; - } - let new_required: Vec> = parent_required - .iter() - .filter_map(|r| { - if let Some(column) = r.as_any().downcast_ref::() { - column_mapping.get(column.name()) - } else { - None - } - }) - .map(|e| Arc::new(e.clone()) as Arc) - .collect::>(); - new_required -} - fn shift_right_required( parent_required: &[Arc], left_columns_len: usize, @@ -969,7 +945,7 @@ mod tests { use super::*; use crate::datasource::listing::PartitionedFile; use crate::datasource::object_store::ObjectStoreUrl; - use crate::physical_optimizer::sort_enforcement::EnforceSorting; + use crate::physical_optimizer::sort_enforcement2::TopDownEnforceSorting; use crate::physical_plan::aggregates::{ AggregateExec, AggregateMode, PhysicalGroupBy, }; @@ -1015,6 +991,27 @@ mod tests { )) } + fn parquet_multiple_exec() -> Arc { + Arc::new(ParquetExec::new( + FileScanConfig { + object_store_url: ObjectStoreUrl::parse("test:///").unwrap(), + file_schema: schema(), + file_groups: vec![ + vec![PartitionedFile::new("x".to_string(), 100)], + vec![PartitionedFile::new("y".to_string(), 100)], + ], + statistics: Statistics::default(), + projection: None, + limit: None, + table_partition_cols: vec![], + output_ordering: None, + infinite_source: false, + }, + None, + None, + )) + } + fn projection_exec_with_alias( input: Arc, alias_pairs: Vec<(String, String)>, @@ -1134,7 +1131,7 @@ mod tests { // `EnforceSorting` and `EnfoceDistribution`. // TODO: Orthogonalize the tests here just to verify `EnforceDistribution` and create // new tests for the cascade. - let optimizer = EnforceSorting {}; + let optimizer = TopDownEnforceSorting {}; let optimized = optimizer.optimize(optimized, &config)?; // Now format correctly @@ -1873,7 +1870,7 @@ mod tests { #[test] fn multi_smj_joins() -> Result<()> { - let left = parquet_exec(); + let left = parquet_multiple_exec(); let alias_pairs: Vec<(String, String)> = vec![ ("a".to_string(), "a1".to_string()), ("b".to_string(), "b1".to_string()), @@ -1881,7 +1878,7 @@ mod tests { ("d".to_string(), "d1".to_string()), ("e".to_string(), "e1".to_string()), ]; - let right = projection_exec_with_alias(parquet_exec(), alias_pairs); + let right = projection_exec_with_alias(parquet_multiple_exec(), alias_pairs); // SortMergeJoin does not support RightSemi and RightAnti join now let join_types = vec![ @@ -1912,7 +1909,7 @@ mod tests { )]; let top_join = sort_merge_join_exec( join.clone(), - parquet_exec(), + parquet_multiple_exec(), &top_join_on, &join_type, ); @@ -1925,33 +1922,33 @@ mod tests { vec![ top_join_plan.as_str(), join_plan.as_str(), - "SortExec: [a@0 ASC]", - "RepartitionExec: partitioning=Hash([Column { name: \"a\", index: 0 }], 10), input_partitions=1", - "ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[a, b, c, d, e]", - "SortExec: [b1@1 ASC]", - "RepartitionExec: partitioning=Hash([Column { name: \"b1\", index: 1 }], 10), input_partitions=1", + "SortExec: [a@0 ASC], global=false", + "RepartitionExec: partitioning=Hash([Column { name: \"a\", index: 0 }], 10), input_partitions=2", + "ParquetExec: limit=None, partitions={2 groups: [[x], [y]]}, projection=[a, b, c, d, e]", + "SortExec: [b1@1 ASC], global=false", + "RepartitionExec: partitioning=Hash([Column { name: \"b1\", index: 1 }], 10), input_partitions=2", "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - "ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[a, b, c, d, e]", - "SortExec: [c@2 ASC]", - "RepartitionExec: partitioning=Hash([Column { name: \"c\", index: 2 }], 10), input_partitions=1", - "ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[a, b, c, d, e]", + "ParquetExec: limit=None, partitions={2 groups: [[x], [y]]}, projection=[a, b, c, d, e]", + "SortExec: [c@2 ASC], global=false", + "RepartitionExec: partitioning=Hash([Column { name: \"c\", index: 2 }], 10), input_partitions=2", + "ParquetExec: limit=None, partitions={2 groups: [[x], [y]]}, projection=[a, b, c, d, e]", ], // Should include 4 RepartitionExecs _ => vec![ top_join_plan.as_str(), - "SortExec: [a@0 ASC]", + "SortExec: [a@0 ASC], global=false", "RepartitionExec: partitioning=Hash([Column { name: \"a\", index: 0 }], 10), input_partitions=10", join_plan.as_str(), - "SortExec: [a@0 ASC]", - "RepartitionExec: partitioning=Hash([Column { name: \"a\", index: 0 }], 10), input_partitions=1", - "ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[a, b, c, d, e]", - "SortExec: [b1@1 ASC]", - "RepartitionExec: partitioning=Hash([Column { name: \"b1\", index: 1 }], 10), input_partitions=1", + "SortExec: [a@0 ASC], global=false", + "RepartitionExec: partitioning=Hash([Column { name: \"a\", index: 0 }], 10), input_partitions=2", + "ParquetExec: limit=None, partitions={2 groups: [[x], [y]]}, projection=[a, b, c, d, e]", + "SortExec: [b1@1 ASC], global=false", + "RepartitionExec: partitioning=Hash([Column { name: \"b1\", index: 1 }], 10), input_partitions=2", "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - "ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[a, b, c, d, e]", - "SortExec: [c@2 ASC]", - "RepartitionExec: partitioning=Hash([Column { name: \"c\", index: 2 }], 10), input_partitions=1", - "ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[a, b, c, d, e]", + "ParquetExec: limit=None, partitions={2 groups: [[x], [y]]}, projection=[a, b, c, d, e]", + "SortExec: [c@2 ASC], global=false", + "RepartitionExec: partitioning=Hash([Column { name: \"c\", index: 2 }], 10), input_partitions=2", + "ParquetExec: limit=None, partitions={2 groups: [[x], [y]]}, projection=[a, b, c, d, e]", ], }; assert_optimized!(expected, top_join); @@ -1966,7 +1963,7 @@ mod tests { )]; let top_join = sort_merge_join_exec( join, - parquet_exec(), + parquet_multiple_exec(), &top_join_on, &join_type, ); @@ -1978,33 +1975,33 @@ mod tests { JoinType::Inner | JoinType::Right => vec![ top_join_plan.as_str(), join_plan.as_str(), - "SortExec: [a@0 ASC]", - "RepartitionExec: partitioning=Hash([Column { name: \"a\", index: 0 }], 10), input_partitions=1", - "ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[a, b, c, d, e]", - "SortExec: [b1@1 ASC]", - "RepartitionExec: partitioning=Hash([Column { name: \"b1\", index: 1 }], 10), input_partitions=1", + "SortExec: [a@0 ASC], global=false", + "RepartitionExec: partitioning=Hash([Column { name: \"a\", index: 0 }], 10), input_partitions=2", + "ParquetExec: limit=None, partitions={2 groups: [[x], [y]]}, projection=[a, b, c, d, e]", + "SortExec: [b1@1 ASC], global=false", + "RepartitionExec: partitioning=Hash([Column { name: \"b1\", index: 1 }], 10), input_partitions=2", "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - "ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[a, b, c, d, e]", - "SortExec: [c@2 ASC]", - "RepartitionExec: partitioning=Hash([Column { name: \"c\", index: 2 }], 10), input_partitions=1", - "ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[a, b, c, d, e]", + "ParquetExec: limit=None, partitions={2 groups: [[x], [y]]}, projection=[a, b, c, d, e]", + "SortExec: [c@2 ASC], global=false", + "RepartitionExec: partitioning=Hash([Column { name: \"c\", index: 2 }], 10), input_partitions=2", + "ParquetExec: limit=None, partitions={2 groups: [[x], [y]]}, projection=[a, b, c, d, e]", ], // Should include 4 RepartitionExecs and 4 SortExecs _ => vec![ top_join_plan.as_str(), - "SortExec: [b1@6 ASC]", + "SortExec: [b1@6 ASC], global=false", "RepartitionExec: partitioning=Hash([Column { name: \"b1\", index: 6 }], 10), input_partitions=10", join_plan.as_str(), - "SortExec: [a@0 ASC]", - "RepartitionExec: partitioning=Hash([Column { name: \"a\", index: 0 }], 10), input_partitions=1", - "ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[a, b, c, d, e]", - "SortExec: [b1@1 ASC]", - "RepartitionExec: partitioning=Hash([Column { name: \"b1\", index: 1 }], 10), input_partitions=1", + "SortExec: [a@0 ASC], global=false", + "RepartitionExec: partitioning=Hash([Column { name: \"a\", index: 0 }], 10), input_partitions=2", + "ParquetExec: limit=None, partitions={2 groups: [[x], [y]]}, projection=[a, b, c, d, e]", + "SortExec: [b1@1 ASC], global=false", + "RepartitionExec: partitioning=Hash([Column { name: \"b1\", index: 1 }], 10), input_partitions=2", "ProjectionExec: expr=[a@0 as a1, b@1 as b1, c@2 as c1, d@3 as d1, e@4 as e1]", - "ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[a, b, c, d, e]", - "SortExec: [c@2 ASC]", - "RepartitionExec: partitioning=Hash([Column { name: \"c\", index: 2 }], 10), input_partitions=1", - "ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[a, b, c, d, e]", + "ParquetExec: limit=None, partitions={2 groups: [[x], [y]]}, projection=[a, b, c, d, e]", + "SortExec: [c@2 ASC], global=false", + "RepartitionExec: partitioning=Hash([Column { name: \"c\", index: 2 }], 10), input_partitions=2", + "ParquetExec: limit=None, partitions={2 groups: [[x], [y]]}, projection=[a, b, c, d, e]", ], }; assert_optimized!(expected, top_join); @@ -2065,15 +2062,15 @@ mod tests { // Only two RepartitionExecs added let expected = &[ "SortMergeJoin: join_type=Inner, on=[(Column { name: \"b3\", index: 1 }, Column { name: \"b2\", index: 1 }), (Column { name: \"a3\", index: 0 }, Column { name: \"a2\", index: 0 })]", - "SortExec: [b3@1 ASC,a3@0 ASC]", "ProjectionExec: expr=[a1@0 as a3, b1@1 as b3]", "ProjectionExec: expr=[a1@1 as a1, b1@0 as b1]", + "SortExec: [b1@0 ASC,a1@1 ASC], global=false", "AggregateExec: mode=FinalPartitioned, gby=[b1@0 as b1, a1@1 as a1], aggr=[]", "RepartitionExec: partitioning=Hash([Column { name: \"b1\", index: 0 }, Column { name: \"a1\", index: 1 }], 10), input_partitions=1", "AggregateExec: mode=Partial, gby=[b@1 as b1, a@0 as a1], aggr=[]", "ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[a, b, c, d, e]", - "SortExec: [b2@1 ASC,a2@0 ASC]", "ProjectionExec: expr=[a@1 as a2, b@0 as b2]", + "SortExec: [b@0 ASC,a@1 ASC], global=false", "AggregateExec: mode=FinalPartitioned, gby=[b@0 as b, a@1 as a], aggr=[]", "RepartitionExec: partitioning=Hash([Column { name: \"b\", index: 0 }, Column { name: \"a\", index: 1 }], 10), input_partitions=1", "AggregateExec: mode=Partial, gby=[b@1 as b, a@0 as a], aggr=[]", @@ -2103,8 +2100,8 @@ mod tests { // The optimizer should not add an additional SortExec as the // data is already sorted + // SortPreservingMergeExec is also removed from the final plan let expected = &[ - "SortPreservingMergeExec: [a@0 ASC]", "CoalesceBatchesExec: target_batch_size=4096", "ParquetExec: limit=None, partitions={1 group: [[x]]}, output_ordering=[a@0 ASC], projection=[a, b, c, d, e]", ]; diff --git a/datafusion/core/src/physical_optimizer/mod.rs b/datafusion/core/src/physical_optimizer/mod.rs index 3958a546a92df..d0b0a917616e8 100644 --- a/datafusion/core/src/physical_optimizer/mod.rs +++ b/datafusion/core/src/physical_optimizer/mod.rs @@ -31,6 +31,7 @@ pub mod sort_enforcement; mod utils; pub mod pipeline_fixer; +pub mod sort_enforcement2; #[cfg(test)] pub mod test_utils; diff --git a/datafusion/core/src/physical_optimizer/repartition.rs b/datafusion/core/src/physical_optimizer/repartition.rs index 1285b9089c8d8..84501b5cfbe05 100644 --- a/datafusion/core/src/physical_optimizer/repartition.rs +++ b/datafusion/core/src/physical_optimizer/repartition.rs @@ -307,6 +307,7 @@ mod tests { use crate::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; use crate::physical_plan::union::UnionExec; use crate::physical_plan::{displayable, DisplayFormatType, Statistics}; + use datafusion_physical_expr::{new_sort_requirements, PhysicalSortRequirements}; fn schema() -> SchemaRef { Arc::new(Schema::new(vec![Field::new("c1", DataType::Boolean, true)])) @@ -355,6 +356,33 @@ mod tests { )) } + // Created a sorted parquet exec with multiple files + fn parquet_exec_multiple_sorted() -> Arc { + let sort_exprs = vec![PhysicalSortExpr { + expr: col("c1", &schema()).unwrap(), + options: SortOptions::default(), + }]; + + Arc::new(ParquetExec::new( + FileScanConfig { + object_store_url: ObjectStoreUrl::parse("test:///").unwrap(), + file_schema: schema(), + file_groups: vec![ + vec![PartitionedFile::new("x".to_string(), 100)], + vec![PartitionedFile::new("y".to_string(), 100)], + ], + statistics: Statistics::default(), + projection: None, + limit: None, + table_partition_cols: vec![], + output_ordering: Some(sort_exprs), + infinite_source: false, + }, + None, + None, + )) + } + fn sort_preserving_merge_exec( input: Arc, ) -> Arc { @@ -556,7 +584,7 @@ mod tests { "GlobalLimitExec: skip=0, fetch=100", "LocalLimitExec: fetch=100", // data is sorted so can't repartition here - "SortExec: [c1@0 ASC]", + "SortExec: [c1@0 ASC], global=true", "ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[c1]", ]; @@ -574,7 +602,7 @@ mod tests { "FilterExec: c1@0", // data is sorted so can't repartition here even though // filter would benefit from parallelism, the answers might be wrong - "SortExec: [c1@0 ASC]", + "SortExec: [c1@0 ASC], global=true", "ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[c1]", ]; @@ -662,7 +690,7 @@ mod tests { // need repartiton and resort as the data was not sorted correctly let expected = &[ "SortPreservingMergeExec: [c1@0 ASC]", - "SortExec: [c1@0 ASC]", + "SortExec: [c1@0 ASC], global=false", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[c1]", ]; @@ -674,12 +702,12 @@ mod tests { #[test] fn repartition_ignores_sort_preserving_merge() -> Result<()> { // sort preserving merge already sorted input, - let plan = sort_preserving_merge_exec(parquet_exec_sorted()); + let plan = sort_preserving_merge_exec(parquet_exec_multiple_sorted()); // should not repartition / sort (as the data was already sorted) let expected = &[ "SortPreservingMergeExec: [c1@0 ASC]", - "ParquetExec: limit=None, partitions={1 group: [[x]]}, output_ordering=[c1@0 ASC], projection=[c1]", + "ParquetExec: limit=None, partitions={2 groups: [[x], [y]]}, output_ordering=[c1@0 ASC], projection=[c1]", ]; assert_optimized!(expected, plan); @@ -762,7 +790,7 @@ mod tests { // needs to repartition / sort as the data was not sorted correctly let expected = &[ "SortPreservingMergeExec: [c1@0 ASC]", - "SortExec: [c1@0 ASC]", + "SortExec: [c1@0 ASC], global=false", "ProjectionExec: expr=[c1@0 as c1]", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[c1]", @@ -775,13 +803,14 @@ mod tests { #[test] fn repartition_ignores_transitively_with_projection() -> Result<()> { // sorted input - let plan = sort_preserving_merge_exec(projection_exec(parquet_exec_sorted())); + let plan = + sort_preserving_merge_exec(projection_exec(parquet_exec_multiple_sorted())); // data should not be repartitioned / resorted let expected = &[ "SortPreservingMergeExec: [c1@0 ASC]", "ProjectionExec: expr=[c1@0 as c1]", - "ParquetExec: limit=None, partitions={1 group: [[x]]}, output_ordering=[c1@0 ASC], projection=[c1]", + "ParquetExec: limit=None, partitions={2 groups: [[x], [y]]}, output_ordering=[c1@0 ASC], projection=[c1]", ]; assert_optimized!(expected, plan); @@ -796,7 +825,7 @@ mod tests { let expected = &[ "SortPreservingMergeExec: [c1@0 ASC]", // Expect repartition on the input to the sort (as it can benefit from additional parallelism) - "SortExec: [c1@0 ASC]", + "SortExec: [c1@0 ASC], global=false", "ProjectionExec: expr=[c1@0 as c1]", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[c1]", @@ -814,7 +843,7 @@ mod tests { let expected = &[ "SortPreservingMergeExec: [c1@0 ASC]", // Expect repartition on the input to the sort (as it can benefit from additional parallelism) - "SortExec: [c1@0 ASC]", + "SortExec: [c1@0 ASC], global=false", "FilterExec: c1@0", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[c1]", @@ -834,7 +863,7 @@ mod tests { let expected = &[ "SortPreservingMergeExec: [c1@0 ASC]", // Expect repartition on the input to the sort (as it can benefit from additional parallelism) - "SortExec: [c1@0 ASC]", + "SortExec: [c1@0 ASC], global=false", "ProjectionExec: expr=[c1@0 as c1]", "FilterExec: c1@0", // repartition is lowest down @@ -881,8 +910,9 @@ mod tests { } // model that it requires the output ordering of its input - fn required_input_ordering(&self) -> Vec> { - vec![self.input.output_ordering()] + fn required_input_ordering(&self) -> Vec>> { + let ordering_requirements = new_sort_requirements(self.output_ordering()); + vec![ordering_requirements] } fn with_new_children( diff --git a/datafusion/core/src/physical_optimizer/sort_enforcement.rs b/datafusion/core/src/physical_optimizer/sort_enforcement.rs index c9a3c8fec293e..d0d983597abad 100644 --- a/datafusion/core/src/physical_optimizer/sort_enforcement.rs +++ b/datafusion/core/src/physical_optimizer/sort_enforcement.rs @@ -39,7 +39,10 @@ use crate::physical_plan::windows::{BoundedWindowAggExec, WindowAggExec}; use crate::physical_plan::{with_new_children_if_necessary, ExecutionPlan}; use arrow::datatypes::SchemaRef; use datafusion_common::{reverse_sort_options, DataFusionError}; -use datafusion_physical_expr::utils::{ordering_satisfy, ordering_satisfy_concrete}; +use datafusion_physical_expr::utils::{ + create_sort_expr_from_requirement, ordering_satisfy, + ordering_satisfy_requirement_concrete, +}; use datafusion_physical_expr::window::WindowExpr; use datafusion_physical_expr::{PhysicalExpr, PhysicalSortExpr}; use itertools::izip; @@ -177,16 +180,16 @@ fn ensure_sorting( let physical_ordering = child.output_ordering(); match (required_ordering, physical_ordering) { (Some(required_ordering), Some(physical_ordering)) => { - let is_ordering_satisfied = ordering_satisfy_concrete( + let is_ordering_satisfied = ordering_satisfy_requirement_concrete( physical_ordering, - required_ordering, + &required_ordering, || child.equivalence_properties(), ); if !is_ordering_satisfied { // Make sure we preserve the ordering requirements: update_child_to_remove_unnecessary_sort(child, sort_onwards)?; - let sort_expr = required_ordering.to_vec(); - *child = add_sort_above_child(child, sort_expr)?; + let sort_expr = create_sort_expr_from_requirement(&required_ordering); + *child = add_sort_above_child(child, sort_expr, None)?; sort_onwards.push((idx, child.clone())) } if let [first, ..] = sort_onwards.as_slice() { @@ -236,8 +239,8 @@ fn ensure_sorting( } (Some(required), None) => { // Ordering requirement is not met, we should add a SortExec to the plan. - let sort_expr = required.to_vec(); - *child = add_sort_above_child(child, sort_expr)?; + let sort_expr = create_sort_expr_from_requirement(&required); + *child = add_sort_above_child(child, sort_expr, None)?; *sort_onwards = vec![(idx, child.clone())]; } (None, Some(_)) => { @@ -654,12 +657,12 @@ mod tests { let physical_plan = sort_exec(vec![sort_expr("nullable_col", &schema)], input); let expected_input = vec![ - "SortExec: [nullable_col@0 ASC]", - " SortExec: [non_nullable_col@1 ASC]", + "SortExec: [nullable_col@0 ASC], global=true", + " SortExec: [non_nullable_col@1 ASC], global=true", " MemoryExec: partitions=0, partition_sizes=[]", ]; let expected_optimized = vec![ - "SortExec: [nullable_col@0 ASC]", + "SortExec: [nullable_col@0 ASC], global=true", " MemoryExec: partitions=0, partition_sizes=[]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan); @@ -708,9 +711,9 @@ mod tests { let expected_input = vec![ "WindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow }]", " FilterExec: NOT non_nullable_col@1", - " SortExec: [non_nullable_col@1 ASC NULLS LAST]", + " SortExec: [non_nullable_col@1 ASC NULLS LAST], global=true", " WindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow }]", - " SortExec: [non_nullable_col@1 DESC]", + " SortExec: [non_nullable_col@1 DESC], global=true", " MemoryExec: partitions=0, partition_sizes=[]", ]; @@ -718,7 +721,7 @@ mod tests { "WindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(NULL) }]", " FilterExec: NOT non_nullable_col@1", " WindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow }]", - " SortExec: [non_nullable_col@1 DESC]", + " SortExec: [non_nullable_col@1 DESC], global=true", " MemoryExec: partitions=0, partition_sizes=[]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan); @@ -740,7 +743,7 @@ mod tests { ]; let expected_optimized = vec![ "SortPreservingMergeExec: [nullable_col@0 ASC]", - " SortExec: [nullable_col@0 ASC]", + " SortExec: [nullable_col@0 ASC], global=true", " MemoryExec: partitions=0, partition_sizes=[]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan); @@ -748,7 +751,7 @@ mod tests { } #[tokio::test] - async fn test_remove_unnecessary_sort1() -> Result<()> { + async fn test_remove_unnecessary_sort3() -> Result<()> { let schema = create_test_schema()?; let source = memory_exec(&schema); let sort_exprs = vec![sort_expr("nullable_col", &schema)]; @@ -760,15 +763,15 @@ mod tests { let physical_plan = sort_preserving_merge_exec(sort_exprs, sort); let expected_input = vec![ "SortPreservingMergeExec: [nullable_col@0 ASC]", - " SortExec: [nullable_col@0 ASC]", + " SortExec: [nullable_col@0 ASC], global=true", " SortPreservingMergeExec: [nullable_col@0 ASC]", - " SortExec: [nullable_col@0 ASC]", + " SortExec: [nullable_col@0 ASC], global=true", " MemoryExec: partitions=0, partition_sizes=[]", ]; let expected_optimized = vec![ "SortPreservingMergeExec: [nullable_col@0 ASC]", " SortPreservingMergeExec: [nullable_col@0 ASC]", - " SortExec: [nullable_col@0 ASC]", + " SortExec: [nullable_col@0 ASC], global=true", " MemoryExec: partitions=0, partition_sizes=[]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan); @@ -787,12 +790,12 @@ mod tests { let physical_plan = sort_preserving_merge_exec(sort_exprs, sort); let expected_input = vec![ "SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", - " SortExec: [nullable_col@0 ASC]", + " SortExec: [nullable_col@0 ASC], global=true", " MemoryExec: partitions=0, partition_sizes=[]", ]; let expected_optimized = vec![ "SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", - " SortExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", + " SortExec: [nullable_col@0 ASC,non_nullable_col@1 ASC], global=true", " MemoryExec: partitions=0, partition_sizes=[]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan); @@ -817,7 +820,7 @@ mod tests { "SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", " ParquetExec: limit=None, partitions={1 group: [[x]]}, output_ordering=[nullable_col@0 ASC], projection=[nullable_col, non_nullable_col]", - " SortExec: [nullable_col@0 ASC]", + " SortExec: [nullable_col@0 ASC], global=true", " ParquetExec: limit=None, partitions={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 @@ -848,7 +851,7 @@ mod tests { "SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", " ParquetExec: limit=None, partitions={1 group: [[x]]}, output_ordering=[nullable_col@0 ASC, non_nullable_col@1 ASC], projection=[nullable_col, non_nullable_col]", - " SortExec: [nullable_col@0 ASC]", + " SortExec: [nullable_col@0 ASC], global=true", " ParquetExec: limit=None, partitions={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 @@ -882,13 +885,13 @@ mod tests { "SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", " UnionExec", " ParquetExec: limit=None, partitions={1 group: [[x]]}, output_ordering=[nullable_col@0 ASC], projection=[nullable_col, non_nullable_col]", - " SortExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", + " SortExec: [nullable_col@0 ASC,non_nullable_col@1 ASC], global=true", " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", ]; // should remove unnecessary sorting from below and move it to top let expected_optimized = vec![ "SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", - " SortExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", + " SortExec: [nullable_col@0 ASC,non_nullable_col@1 ASC], global=false", " UnionExec", " ParquetExec: limit=None, partitions={1 group: [[x]]}, output_ordering=[nullable_col@0 ASC], projection=[nullable_col, non_nullable_col]", " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", diff --git a/datafusion/core/src/physical_optimizer/sort_enforcement2.rs b/datafusion/core/src/physical_optimizer/sort_enforcement2.rs new file mode 100644 index 0000000000000..984d817db9a63 --- /dev/null +++ b/datafusion/core/src/physical_optimizer/sort_enforcement2.rs @@ -0,0 +1,2101 @@ +// 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 +//! "SortExec: [nullable_col@0 ASC]", +//! " SortExec: [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 crate::config::ConfigOptions; +use crate::error::Result; +use crate::execution::context::TaskContext; +use crate::physical_optimizer::utils::add_sort_above_child; +use crate::physical_optimizer::PhysicalOptimizerRule; +use crate::physical_plan::filter::FilterExec; +use crate::physical_plan::joins::utils::JoinSide; +use crate::physical_plan::joins::SortMergeJoinExec; +use crate::physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; +use crate::physical_plan::projection::ProjectionExec; +use crate::physical_plan::repartition::RepartitionExec; +use crate::physical_plan::rewrite::TreeNodeRewritable; +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::{ + displayable, with_new_children_if_necessary, DisplayFormatType, ExecutionPlan, + Partitioning, SendableRecordBatchStream, +}; +use arrow::datatypes::SchemaRef; +use datafusion_common::{reverse_sort_options, DataFusionError, Statistics}; +use datafusion_physical_expr::expressions::Column; +use datafusion_physical_expr::utils::{ + create_sort_expr_from_requirement, map_requirement_before_projection, + ordering_satisfy, ordering_satisfy_requirement, requirements_compatible, +}; +use datafusion_physical_expr::{ + EquivalenceProperties, PhysicalExpr, PhysicalSortExpr, PhysicalSortRequirements, +}; +use itertools::izip; +use std::any::Any; +use std::iter::zip; +use std::sync::Arc; + +/// This rule inspects SortExec's in the given physical plan and removes the +/// ones it can prove unnecessary. +#[derive(Default)] +pub struct TopDownEnforceSorting {} + +impl TopDownEnforceSorting { + #[allow(missing_docs)] + pub fn new() -> Self { + Self {} + } +} + +/// This is a "data class" we use within the [EnforceSorting] rule +#[derive(Debug, Clone)] +struct PlanWithSortRequirements { + /// Current plan + plan: Arc, + /// Whether the plan could impact the final result ordering + impact_result_ordering: bool, + /// Parent required sort ordering + required_ordering: Option>, + /// The adjusted request sort ordering to children. + /// By default they are the same as the plan's required input ordering, but can be adjusted based on parent required sort ordering properties. + adjusted_request_ordering: Vec>>, +} + +impl PlanWithSortRequirements { + pub fn init(plan: Arc) -> Self { + let impact_result_ordering = plan.output_ordering().is_some() + || plan.output_partitioning().partition_count() == 1 + || plan.as_any().downcast_ref::().is_some() + || plan.as_any().downcast_ref::().is_some(); + let request_ordering = plan.required_input_ordering(); + PlanWithSortRequirements { + plan, + impact_result_ordering, + required_ordering: None, + adjusted_request_ordering: request_ordering, + } + } + + pub fn new_without_impact_result_ordering(plan: Arc) -> Self { + let request_ordering = plan.required_input_ordering(); + PlanWithSortRequirements { + plan, + impact_result_ordering: false, + required_ordering: None, + adjusted_request_ordering: request_ordering, + } + } + + pub fn children(&self) -> Vec { + let plan_children = self.plan.children(); + assert_eq!(plan_children.len(), self.adjusted_request_ordering.len()); + let child_impact_result_ordering = if self + .plan + .as_any() + .downcast_ref::() + .is_some() + || self + .plan + .as_any() + .downcast_ref::() + .is_some() + { + true + } else if self.plan.as_any().downcast_ref::().is_some() { + false + } else { + self.plan.maintains_input_order().iter().all(|o| *o) + && self.impact_result_ordering + }; + println!( + "child_impact_result_ordering {:?}", + child_impact_result_ordering + ); + plan_children + .into_iter() + .zip(self.adjusted_request_ordering.clone().into_iter()) + .map(|(child, required)| { + let from_parent = required; + let child_request_ordering = child.required_input_ordering(); + PlanWithSortRequirements { + plan: child, + impact_result_ordering: child_impact_result_ordering, + required_ordering: from_parent, + adjusted_request_ordering: child_request_ordering, + } + }) + .collect() + } +} + +impl TreeNodeRewritable for PlanWithSortRequirements { + fn map_children(self, transform: F) -> Result + where + F: FnMut(Self) -> Result, + { + let children = self.children(); + if children.is_empty() { + Ok(self) + } else { + let new_children = children + .into_iter() + .map(transform) + .collect::>>()?; + + let children_plans = new_children + .iter() + .map(|elem| elem.plan.clone()) + .collect::>(); + let plan = with_new_children_if_necessary(self.plan, children_plans)?; + Ok(PlanWithSortRequirements { + plan, + impact_result_ordering: self.impact_result_ordering, + required_ordering: self.required_ordering, + adjusted_request_ordering: self.adjusted_request_ordering, + }) + } + } +} + +impl PhysicalOptimizerRule for TopDownEnforceSorting { + fn optimize( + &self, + plan: Arc, + _config: &ConfigOptions, + ) -> Result> { + // Execute a Top-Down process(Preorder Traversal) to ensure the sort requirements: + let plan_requirements = PlanWithSortRequirements::init(plan); + let adjusted = plan_requirements.transform_down(&ensure_sorting)?; + // Remove the TombStoneExec + let final_plan = adjusted.plan.transform_up(&|plan| { + if let Some(tombstone_exec) = plan.as_any().downcast_ref::() { + Ok(Some(tombstone_exec.input.clone())) + } else { + Ok(None) + } + })?; + Ok(final_plan) + } + + fn name(&self) -> &str { + "EnforceSorting2" + } + + fn schema_check(&self) -> bool { + true + } +} + +fn ensure_sorting( + requirements: PlanWithSortRequirements, +) -> Result> { + println!( + "=== Current plan ===\n{}\n", + displayable(requirements.plan.as_ref()).indent() + ); + println!( + "impact_result_ordering: {:?}, parent required_ordering {:?}, adjusted request ordering {:?}", + requirements.impact_result_ordering, requirements.required_ordering, requirements.adjusted_request_ordering, + ); + if let Some(sort_exec) = requirements.plan.as_any().downcast_ref::() { + // Remove unnecessary global SortExec + if !sort_exec.preserve_partitioning() { + if !requirements.impact_result_ordering + && requirements.required_ordering.is_none() + { + println!("remove sort_exec due to no need to keep ordering"); + return Ok(Some(PlanWithSortRequirements { + plan: Arc::new(TombStoneExec::new(sort_exec.input().clone())), + impact_result_ordering: false, + required_ordering: None, + adjusted_request_ordering: vec![None], + })); + } else if ordering_satisfy( + sort_exec.input().output_ordering(), + sort_exec.output_ordering(), + || sort_exec.input().equivalence_properties(), + ) && sort_exec.input().output_partitioning().partition_count() == 1 + { + println!("remove sort_exec due to child already satisfy"); + return Ok(Some(PlanWithSortRequirements { + plan: Arc::new(TombStoneExec::new(sort_exec.input().clone())), + impact_result_ordering: true, + required_ordering: None, + adjusted_request_ordering: vec![requirements.required_ordering], + })); + } + } + } else if let Some(sort_pres_exec) = requirements + .plan + .as_any() + .downcast_ref::() + { + // SortPreservingMergeExec + SortExec(local/global) is the same as the global SortExec + // Remove unnecessary SortPreservingMergeExec + SortExec(local/global) + if let Some(child_sort_exec) = + sort_pres_exec.input().as_any().downcast_ref::() + { + if sort_pres_exec.expr() == child_sort_exec.expr() { + if !requirements.impact_result_ordering + && requirements.required_ordering.is_none() + { + println!("remove SortPreservingMergeExec + SortExec due to no need to keep ordering"); + return Ok(Some(PlanWithSortRequirements { + plan: Arc::new(TombStoneExec::new( + child_sort_exec.input().clone(), + )), + impact_result_ordering: false, + required_ordering: None, + adjusted_request_ordering: vec![None], + })); + } else if ordering_satisfy( + child_sort_exec.input().output_ordering(), + child_sort_exec.output_ordering(), + || child_sort_exec.input().equivalence_properties(), + ) && child_sort_exec + .input() + .output_partitioning() + .partition_count() + == 1 + { + println!("remove SortPreservingMergeExec + SortExec due to child already satisfy"); + return Ok(Some(PlanWithSortRequirements { + plan: Arc::new(TombStoneExec::new( + child_sort_exec.input().clone(), + )), + impact_result_ordering: true, + required_ordering: None, + adjusted_request_ordering: vec![requirements.required_ordering], + })); + } + } + } else { + // Remove unnecessary SortPreservingMergeExec only + if !requirements.impact_result_ordering { + println!( + "remove SortPreservingMergeExec due to no need to keep ordering" + ); + return Ok(Some(PlanWithSortRequirements { + plan: Arc::new(TombStoneExec::new(sort_pres_exec.input().clone())), + impact_result_ordering: false, + required_ordering: None, + adjusted_request_ordering: vec![requirements.required_ordering], + })); + } else if ordering_satisfy( + sort_pres_exec.input().output_ordering(), + Some(sort_pres_exec.expr()), + || sort_pres_exec.input().equivalence_properties(), + ) && sort_pres_exec + .input() + .output_partitioning() + .partition_count() + == 1 + { + println!("remove SortPreservingMergeExec due to child already satisfy"); + return Ok(Some(PlanWithSortRequirements { + plan: Arc::new(TombStoneExec::new(sort_pres_exec.input().clone())), + impact_result_ordering: true, + required_ordering: None, + adjusted_request_ordering: vec![requirements.required_ordering], + })); + } + } + } + println!("no removing"); + let plan = &requirements.plan; + let parent_required = requirements.required_ordering.as_deref(); + if ordering_satisfy_requirement(plan.output_ordering(), parent_required, || { + plan.equivalence_properties() + }) { + // Can satisfy the parent requirements, clear the requirements + println!( + "Can satisfy the parent requirements, impact_result_ordering {:?}", + requirements.impact_result_ordering + ); + if plan.as_any().downcast_ref::().is_some() + || plan + .as_any() + .downcast_ref::() + .is_some() + { + let request_child = requirements.adjusted_request_ordering[0].as_deref(); + let reversed_request_child = reverse_window_sort_requirements(request_child); + + if should_reverse_window_sort_requirements( + plan.clone(), + request_child, + reversed_request_child.as_deref(), + ) { + println!("Should reverse top window sort_requirements"); + let (window_expr, input_schema, partition_keys) = if let Some(exec) = + plan.as_any().downcast_ref::() + { + ( + exec.window_expr(), + exec.input_schema(), + exec.partition_keys.clone(), + ) + } else if let Some(exec) = plan.as_any().downcast_ref::() { + ( + exec.window_expr(), + exec.input_schema(), + exec.partition_keys.clone(), + ) + } else { + return Err(DataFusionError::Plan( + "Expects to receive either WindowAggExec of BoundedWindowAggExec" + .to_string(), + )); + }; + let new_window_expr = window_expr + .iter() + .map(|e| e.get_reverse_expr()) + .collect::>>(); + let new_physical_ordering = create_sort_expr_from_requirement( + reversed_request_child.clone().unwrap().as_ref(), + ); + if let Some(window_expr) = new_window_expr { + let uses_bounded_memory = + window_expr.iter().all(|e| e.uses_bounded_memory()); + // If all window expressions can run with bounded memory, choose the + // bounded window variant: + let new_plan = if uses_bounded_memory { + Arc::new(BoundedWindowAggExec::try_new( + window_expr, + plan.children()[0].clone(), + input_schema, + partition_keys, + Some(new_physical_ordering), + )?) as _ + } else { + Arc::new(WindowAggExec::try_new( + window_expr, + plan.children()[0].clone(), + input_schema, + partition_keys, + Some(new_physical_ordering), + )?) as _ + }; + println!("Reverse WindowAggExec expressions and push down the reversed requirements"); + + return Ok(Some(PlanWithSortRequirements { + plan: new_plan, + impact_result_ordering: false, + required_ordering: None, + adjusted_request_ordering: vec![reversed_request_child], + })); + } + } else { + println!("Should not reverse top window sort_requirements"); + } + } else if let Some(_) = plan.as_any().downcast_ref::() { + return Ok(Some(PlanWithSortRequirements { + plan: plan.clone(), + impact_result_ordering: false, + required_ordering: None, + adjusted_request_ordering: requirements.adjusted_request_ordering.clone(), + })); + } + return Ok(Some(PlanWithSortRequirements { + plan: plan.clone(), + impact_result_ordering: requirements.impact_result_ordering, + required_ordering: None, + adjusted_request_ordering: requirements.adjusted_request_ordering, + })); + } else if let Some(sort_exec) = plan.as_any().downcast_ref::() { + println!("Modify current SortExec to satisfy the parent requirements"); + // If the current plan is a SortExec, update the SortExec to satisfy the parent requirements + let parent_required_expr = + create_sort_expr_from_requirement(parent_required.unwrap()); + let new_plan = add_sort_above_child( + &sort_exec.input, + parent_required_expr, + sort_exec.fetch(), + )?; + return Ok(Some( + PlanWithSortRequirements::new_without_impact_result_ordering(new_plan), + )); + } else { + println!("Can not satisfy the parent requirements, try to push down"); + // Can not satisfy the parent requirements, check whether should push down the requirements. Add new SortExec when the parent requirements can not be pushed down + let parent_required_expr = + create_sort_expr_from_requirement(parent_required.unwrap()); + let maintains_input_order = plan.maintains_input_order(); + // 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. + // For UnionExec, we can always push down + if (maintains_input_order.is_empty() + || !maintains_input_order.iter().any(|o| *o) + || plan.as_any().downcast_ref::().is_some() + || plan.as_any().downcast_ref::().is_some() + || plan.as_any().downcast_ref::().is_some() + || plan.as_any().downcast_ref::().is_some()) + && plan.as_any().downcast_ref::().is_none() + { + let new_plan = add_sort_above_child(plan, parent_required_expr, None)?; + return Ok(Some( + PlanWithSortRequirements::new_without_impact_result_ordering(new_plan), + )); + } else if let Some(window_agg_exec) = + plan.as_any().downcast_ref::() + { + let window_expr = window_agg_exec.window_expr(); + let request_child = requirements.adjusted_request_ordering[0].as_deref(); + if requirements_compatible(request_child, parent_required, || { + plan.children()[0].equivalence_properties() + }) { + println!("WindowAggExec child requirements are more specific, no need to add SortExec"); + return Ok(Some(PlanWithSortRequirements { + plan: plan.clone(), + impact_result_ordering: true, + required_ordering: None, + adjusted_request_ordering: requirements.adjusted_request_ordering, + })); + } else if requirements_compatible(parent_required, request_child, || { + plan.children()[0].equivalence_properties() + }) { + println!("Parent requirements are more specific, adjust WindowAggExec child requirements and push down the requirements"); + let adjusted = parent_required.map(|r| r.to_vec()); + return Ok(Some(PlanWithSortRequirements { + plan: plan.clone(), + impact_result_ordering: true, + required_ordering: None, + adjusted_request_ordering: vec![adjusted], + })); + } else { + let should_reverse = can_reverse_window_request( + window_expr[0].partition_by(), + parent_required, + request_child, + &window_agg_exec.input().schema(), + ); + if should_reverse { + let new_window_expr = window_expr + .iter() + .map(|e| e.get_reverse_expr()) + .collect::>>(); + if let Some(window_expr) = new_window_expr { + let new_plan = Arc::new(WindowAggExec::try_new( + window_expr, + window_agg_exec.children()[0].clone(), + window_agg_exec.input_schema(), + window_agg_exec.partition_keys.clone(), + Some(parent_required_expr.to_vec()), + )?) as _; + println!("Reverse WindowAggExec expressions and push down the requirements"); + return Ok(Some( + PlanWithSortRequirements::new_without_impact_result_ordering( + new_plan, + ), + )); + } else { + println!("Can not push down, add new SortExec"); + let new_plan = + add_sort_above_child(plan, parent_required_expr, None)?; + return Ok(Some( + PlanWithSortRequirements::new_without_impact_result_ordering( + new_plan, + ), + )); + } + } else { + // Can not push down, add new SortExec + println!("Can not push down, add new SortExec"); + let new_plan = + add_sort_above_child(plan, parent_required_expr, None)?; + return Ok(Some( + PlanWithSortRequirements::new_without_impact_result_ordering( + new_plan, + ), + )); + } + } + } else if let Some(window_agg_exec) = + plan.as_any().downcast_ref::() + { + let window_expr = window_agg_exec.window_expr(); + let request_child = &plan.required_input_ordering()[0]; + if requirements_compatible(request_child.as_deref(), parent_required, || { + plan.children()[0].equivalence_properties() + }) { + println!("BoundedWindowAggExec child requirements are more specific, no need to add SortExec"); + return Ok(Some(PlanWithSortRequirements { + plan: plan.clone(), + impact_result_ordering: true, + required_ordering: None, + adjusted_request_ordering: requirements.adjusted_request_ordering, + })); + } else if requirements_compatible( + parent_required, + request_child.as_deref(), + || plan.children()[0].equivalence_properties(), + ) { + println!("Parent requirements are more specific, adjust BoundedWindowAggExec child requirements and push down the requirements"); + let adjusted = parent_required.map(|r| r.to_vec()); + return Ok(Some(PlanWithSortRequirements { + plan: plan.clone(), + impact_result_ordering: true, + required_ordering: None, + adjusted_request_ordering: vec![adjusted], + })); + } else { + let should_reverse = can_reverse_window_request( + window_expr[0].partition_by(), + parent_required, + request_child.as_deref(), + &window_agg_exec.input().schema(), + ); + if should_reverse { + let new_window_expr = window_expr + .iter() + .map(|e| e.get_reverse_expr()) + .collect::>>(); + if let Some(window_expr) = new_window_expr { + let new_plan = Arc::new(BoundedWindowAggExec::try_new( + window_expr, + window_agg_exec.children()[0].clone(), + window_agg_exec.input_schema(), + window_agg_exec.partition_keys.clone(), + Some(parent_required_expr.to_vec()), + )?) as _; + println!("Reverse BoundedWindowAggExec expressions and push down the requirements"); + return Ok(Some( + PlanWithSortRequirements::new_without_impact_result_ordering( + new_plan, + ), + )); + } else { + println!("Can not push down, add new SortExec"); + let new_plan = + add_sort_above_child(plan, parent_required_expr, None)?; + return Ok(Some( + PlanWithSortRequirements::new_without_impact_result_ordering( + new_plan, + ), + )); + } + } else { + // Can not push down, add new SortExec + println!("Can not push down, add new SortExec"); + let new_plan = + add_sort_above_child(plan, parent_required_expr, None)?; + return Ok(Some( + PlanWithSortRequirements::new_without_impact_result_ordering( + new_plan, + ), + )); + } + } + } 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 expr_source_side = + expr_source_sides(&parent_required_expr, left_columns_len); + match expr_source_side { + Some(JoinSide::Left) if maintains_input_order[0] => { + if requirements_compatible( + plan.required_input_ordering()[0].as_deref(), + parent_required, + || plan.children()[0].equivalence_properties(), + ) { + println!("Requirements are compatible with SMJ"); + return Ok(Some(PlanWithSortRequirements { + plan: plan.clone(), + impact_result_ordering: true, + required_ordering: None, + adjusted_request_ordering: requirements + .adjusted_request_ordering, + })); + } else { + // Can not push down, add new SortExec + println!("Can not push down, add new SortExec"); + let new_plan = + add_sort_above_child(plan, parent_required_expr, None)?; + return Ok(Some( + PlanWithSortRequirements::new_without_impact_result_ordering( + new_plan, + ), + )); + } + } + Some(JoinSide::Right) if maintains_input_order[1] => { + let shift_right_required = + shift_right_required(parent_required.unwrap(), left_columns_len); + if requirements_compatible( + plan.required_input_ordering()[1].as_deref(), + shift_right_required.as_deref(), + || plan.children()[1].equivalence_properties(), + ) { + println!("Requirements are compatible with SMJ"); + return Ok(Some(PlanWithSortRequirements { + plan: plan.clone(), + impact_result_ordering: true, + required_ordering: None, + adjusted_request_ordering: requirements + .adjusted_request_ordering, + })); + } else { + // Can not push down, add new SortExec + println!("Can not push down, add new SortExec"); + let new_plan = + add_sort_above_child(plan, parent_required_expr, None)?; + return Ok(Some( + PlanWithSortRequirements::new_without_impact_result_ordering( + new_plan, + ), + )); + } + } + _ => { + println!("Can not decide the expr side for SortMergeJoinExec, can not push down, add SortExec"); + let new_plan = + add_sort_above_child(plan, parent_required_expr, None)?; + return Ok(Some( + PlanWithSortRequirements::new_without_impact_result_ordering( + new_plan, + ), + )); + } + } + } else if plan.required_input_ordering().iter().any(Option::is_some) { + // If the current plan has its own ordering requirements to its children, check whether the requirements + // are compatible with the parent requirements. + println!( + "the current plan has its own ordering requirements, {:?}", + plan.required_input_ordering() + ); + + let plan_children = plan.children(); + let compatible_with_children = izip!( + maintains_input_order.iter(), + plan.required_input_ordering().into_iter(), + plan_children.iter() + ) + .map(|(can_push_down, request_child, child)| { + *can_push_down + && requirements_compatible( + request_child.as_deref(), + parent_required, + || child.equivalence_properties(), + ) + }) + .collect::>(); + println!( + "plan.equivalence_properties() {:?}", + plan.equivalence_properties() + ); + println!("compatible_with_children {:?}", compatible_with_children); + if compatible_with_children.iter().all(|a| *a) { + // Requirements are compatible, not need to push down. + println!("Requirements are compatible, no need to push down"); + return Ok(Some(PlanWithSortRequirements { + plan: plan.clone(), + impact_result_ordering: true, + required_ordering: None, + adjusted_request_ordering: requirements.adjusted_request_ordering, + })); + } else { + let can_adjust_child_requirements = plan + .required_input_ordering() + .into_iter() + .zip(plan_children.iter()) + .map(|(request_child, child)| { + requirements_compatible( + parent_required, + request_child.as_deref(), + || child.equivalence_properties(), + ) + }) + .collect::>(); + if can_adjust_child_requirements.iter().all(|a| *a) { + // Adjust child requirements and push down the requirements + println!("Adjust child requirements and push down the requirements"); + let adjusted = parent_required.map(|r| r.to_vec()); + return Ok(Some(PlanWithSortRequirements { + plan: plan.clone(), + impact_result_ordering: true, + required_ordering: None, + adjusted_request_ordering: vec![ + adjusted; + can_adjust_child_requirements + .len() + ], + })); + } else { + // Can not push down, add new SortExec + println!("Can not push down, add new SortExec"); + let new_plan = + add_sort_above_child(plan, parent_required_expr, None)?; + return Ok(Some( + PlanWithSortRequirements::new_without_impact_result_ordering( + new_plan, + ), + )); + } + } + } else { + // The current plan does not have its own ordering requirements to its children, consider push down the requirements + if let Some(ProjectionExec { expr, .. }) = + plan.as_any().downcast_ref::() + { + // For Projection, we need to transform the requirements to the columns before the Projection + // And then to push down the requirements + let new_requirement = + map_requirement_before_projection(parent_required, expr); + if new_requirement.is_some() { + println!("Push requirements down to Projection"); + Ok(Some(PlanWithSortRequirements { + plan: plan.clone(), + impact_result_ordering: true, + required_ordering: None, + adjusted_request_ordering: vec![new_requirement], + })) + } else { + // Can not push down, add new SortExec + println!( + "Can not push requirements down to Projection, add SortExec" + ); + let new_plan = + add_sort_above_child(plan, parent_required_expr, None)?; + return Ok(Some( + PlanWithSortRequirements::new_without_impact_result_ordering( + new_plan, + ), + )); + } + } else { + println!("Push down requirements."); + return Ok(Some(PlanWithSortRequirements { + plan: plan.clone(), + impact_result_ordering: requirements.impact_result_ordering, + required_ordering: None, + adjusted_request_ordering: vec![ + requirements.required_ordering; + requirements + .adjusted_request_ordering + .len() + ], + })); + } + } + } +} + +fn expr_source_sides( + required_exprs: &[PhysicalSortExpr], + left_columns_len: usize, +) -> Option { + let all_column_sides = required_exprs + .iter() + .filter_map(|r| { + if let Some(col) = r.expr.as_any().downcast_ref::() { + if col.index() < left_columns_len { + Some(JoinSide::Left) + } else { + Some(JoinSide::Right) + } + } else { + None + } + }) + .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 + } +} + +fn shift_right_required( + parent_required: &[PhysicalSortRequirements], + left_columns_len: usize, +) -> Option> { + let new_right_required: Vec = parent_required + .iter() + .filter_map(|r| { + if let Some(col) = r.expr.as_any().downcast_ref::() { + if col.index() >= left_columns_len { + Some(PhysicalSortRequirements { + expr: Arc::new(Column::new( + col.name(), + col.index() - left_columns_len, + )) as Arc, + sort_options: r.sort_options.clone(), + }) + } else { + None + } + } else { + None + } + }) + .collect::>(); + + // if the parent required are all comming from the right side, the requirements can be pushdown + if new_right_required.len() != parent_required.len() { + None + } else { + Some(new_right_required) + } +} + +#[derive(Debug)] +/// This structure stores extra column information required to remove unnecessary sorts. +pub struct ColumnInfo { + reverse: bool, + is_partition: bool, +} + +fn can_reverse_window_request( + partition_keys: &[Arc], + required: Option<&[PhysicalSortRequirements]>, + request_ordering: Option<&[PhysicalSortRequirements]>, + input_schema: &SchemaRef, +) -> bool { + match (required, request_ordering) { + (_, None) => false, + (None, Some(_)) => false, + (Some(required), Some(request_ordering)) => { + if required.len() > request_ordering.len() { + return false; + } + let mut col_infos = vec![]; + for (required_expr, request_expr) in zip(required, request_ordering) { + let column = required_expr.expr.clone(); + let is_partition = partition_keys.iter().any(|e| e.eq(&column)); + let reverse = check_alignment(input_schema, request_expr, required_expr); + col_infos.push(ColumnInfo { + reverse, + is_partition, + }); + } + let order_by_sections = col_infos + .iter() + .filter(|elem| !elem.is_partition) + .collect::>(); + let should_reverse_order_bys = if order_by_sections.is_empty() { + false + } else { + let first_reverse = order_by_sections[0].reverse; + first_reverse + }; + should_reverse_order_bys + } + } +} + +/// Compares window expression's `window_request` and `parent_required_expr` ordering, returns +/// whether we should reverse the window expression's ordering in order to meet parent's requirements. +fn check_alignment( + input_schema: &SchemaRef, + window_request: &PhysicalSortRequirements, + parent_required_expr: &PhysicalSortRequirements, +) -> bool { + if parent_required_expr.expr.eq(&window_request.expr) + && window_request.sort_options.is_some() + && parent_required_expr.sort_options.is_some() + { + let nullable = parent_required_expr.expr.nullable(input_schema).unwrap(); + let window_request_opts = window_request.sort_options.unwrap(); + let parent_required_opts = parent_required_expr.sort_options.unwrap(); + let is_reversed = if nullable { + window_request_opts == reverse_sort_options(parent_required_opts) + } else { + // If the column is not nullable, NULLS FIRST/LAST is not important. + window_request_opts.descending != parent_required_opts.descending + }; + is_reversed + } else { + false + } +} + +fn reverse_window_sort_requirements( + request_child: Option<&[PhysicalSortRequirements]>, +) -> Option> { + let reversed_request = request_child.map(|request| { + request + .iter() + .map(|req| match req.sort_options { + None => req.clone(), + Some(ops) => PhysicalSortRequirements { + expr: req.expr.clone(), + sort_options: Some(reverse_sort_options(ops)), + }, + }) + .collect::>() + }); + reversed_request +} + +fn should_reverse_window_sort_requirements( + window_plan: Arc, + top_requirement: Option<&[PhysicalSortRequirements]>, + top_reversed_requirement: Option<&[PhysicalSortRequirements]>, +) -> bool { + if top_requirement.is_none() { + return false; + } + let flags = window_plan + .children() + .into_iter() + .map(|child| { + // If the child is leaf node, check the output ordering + if child.children().is_empty() + && ordering_satisfy_requirement( + child.output_ordering(), + top_requirement, + || child.equivalence_properties(), + ) + { + false + } else if child.children().is_empty() + && ordering_satisfy_requirement( + child.output_ordering(), + top_reversed_requirement, + || child.equivalence_properties(), + ) + { + true + } else if child.as_any().downcast_ref::().is_some() + || child + .as_any() + .downcast_ref::() + .is_some() + { + // If the child is WindowExec, check the child requirements + if requirements_compatible( + top_requirement, + child.required_input_ordering()[0].as_deref(), + || child.equivalence_properties(), + ) || requirements_compatible( + child.required_input_ordering()[0].as_deref(), + top_requirement, + || child.equivalence_properties(), + ) || requirements_compatible( + top_reversed_requirement, + child.required_input_ordering()[0].as_deref(), + || child.equivalence_properties(), + ) || requirements_compatible( + child.required_input_ordering()[0].as_deref(), + top_reversed_requirement, + || child.equivalence_properties(), + ) { + should_reverse_window_sort_requirements( + child, + top_requirement, + top_reversed_requirement, + ) + } else { + false + } + } else if requirements_compatible( + top_reversed_requirement, + window_plan.required_input_ordering()[0].as_deref(), + || window_plan.equivalence_properties(), + ) || requirements_compatible( + window_plan.required_input_ordering()[0].as_deref(), + top_reversed_requirement, + || window_plan.equivalence_properties(), + ) { + true + } else { + false + } + }) + .collect::>(); + + flags.iter().all(|o| *o) +} + +/// A TombStoneExec execution plan generated during optimization process, should be removed finally +#[derive(Debug)] +struct TombStoneExec { + /// The input plan + pub input: Arc, +} + +impl TombStoneExec { + pub fn new(input: Arc) -> Self { + Self { input } + } +} + +impl ExecutionPlan for TombStoneExec { + fn as_any(&self) -> &dyn Any { + self + } + + fn schema(&self) -> SchemaRef { + self.input.schema() + } + + fn output_partitioning(&self) -> Partitioning { + self.input.output_partitioning() + } + + fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { + self.input.output_ordering() + } + + fn maintains_input_order(&self) -> Vec { + vec![true] + } + + fn equivalence_properties(&self) -> EquivalenceProperties { + self.input.equivalence_properties() + } + + fn children(&self) -> Vec> { + vec![self.input.clone()] + } + + fn with_new_children( + self: Arc, + children: Vec>, + ) -> Result> { + Ok(Arc::new(TombStoneExec::new(children[0].clone()))) + } + + fn execute( + &self, + _partition: usize, + _context: Arc, + ) -> Result { + Err(DataFusionError::Internal(format!( + "TombStoneExec, invalid plan" + ))) + } + + fn fmt_as( + &self, + t: DisplayFormatType, + f: &mut std::fmt::Formatter, + ) -> std::fmt::Result { + match t { + DisplayFormatType::Default => { + write!(f, "TombStoneExec") + } + } + } + + fn statistics(&self) -> Statistics { + Statistics::default() + } +} + +#[cfg(test)] +mod tests { + use super::*; + use crate::datasource::listing::PartitionedFile; + use crate::datasource::object_store::ObjectStoreUrl; + use crate::physical_plan::displayable; + use crate::physical_plan::file_format::{FileScanConfig, ParquetExec}; + use crate::physical_plan::filter::FilterExec; + use crate::physical_plan::memory::MemoryExec; + use crate::physical_plan::repartition::RepartitionExec; + use crate::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; + use crate::physical_plan::union::UnionExec; + use crate::physical_plan::windows::create_window_expr; + use crate::prelude::SessionContext; + use arrow::compute::SortOptions; + use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; + use datafusion_common::{Result, Statistics}; + use datafusion_expr::{AggregateFunction, WindowFrame, WindowFunction}; + use datafusion_physical_expr::expressions::{col, NotExpr}; + use datafusion_physical_expr::PhysicalSortExpr; + use std::sync::Arc; + + 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) + } + + #[tokio::test] + async fn test_is_column_aligned_nullable() -> Result<()> { + let schema = create_test_schema()?; + let params = vec![ + ((true, true), (false, false), true), + ((true, true), (false, true), false), + ((true, true), (true, false), false), + ((true, false), (false, true), true), + ((true, false), (false, false), false), + ((true, false), (true, true), false), + ]; + for ( + (physical_desc, physical_nulls_first), + (req_desc, req_nulls_first), + reverse_expected, + ) in params + { + let physical_ordering = PhysicalSortRequirements { + expr: col("nullable_col", &schema)?, + sort_options: Some(SortOptions { + descending: physical_desc, + nulls_first: physical_nulls_first, + }), + }; + let required_ordering = PhysicalSortRequirements { + expr: col("nullable_col", &schema)?, + sort_options: Some(SortOptions { + descending: req_desc, + nulls_first: req_nulls_first, + }), + }; + let reverse = + check_alignment(&schema, &physical_ordering, &required_ordering); + assert_eq!(reverse, reverse_expected); + } + + Ok(()) + } + + #[tokio::test] + async fn test_is_column_aligned_non_nullable() -> Result<()> { + let schema = create_test_schema()?; + + let params = vec![ + ((true, true), (false, false), true), + ((true, true), (false, true), true), + ((true, true), (true, false), false), + ((true, false), (false, true), true), + ((true, false), (false, false), true), + ((true, false), (true, true), false), + ]; + for ( + (physical_desc, physical_nulls_first), + (req_desc, req_nulls_first), + reverse_expected, + ) in params + { + let physical_ordering = PhysicalSortRequirements { + expr: col("non_nullable_col", &schema)?, + sort_options: Some(SortOptions { + descending: physical_desc, + nulls_first: physical_nulls_first, + }), + }; + let required_ordering = PhysicalSortRequirements { + expr: col("non_nullable_col", &schema)?, + sort_options: Some(SortOptions { + descending: req_desc, + nulls_first: req_nulls_first, + }), + }; + let reverse = + check_alignment(&schema, &physical_ordering, &required_ordering); + assert_eq!(reverse, reverse_expected); + } + + Ok(()) + } + + /// 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 + /// + macro_rules! assert_optimized { + ($EXPECTED_PLAN_LINES: expr, $EXPECTED_OPTIMIZED_PLAN_LINES: expr, $PLAN: expr) => { + let session_ctx = SessionContext::new(); + let state = session_ctx.state(); + + let physical_plan = $PLAN; + let formatted = displayable(physical_plan.as_ref()).indent().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 = + TopDownEnforceSorting::new().optimize(physical_plan, state.config_options())?; + + let formatted = displayable(optimized_physical_plan.as_ref()) + .indent() + .to_string(); + let actual: Vec<&str> = formatted.trim().lines().collect(); + 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_not_remove_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); + + let window_agg = window_exec("non_nullable_col", sort_exprs, sort); + + 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); + let filter = filter_exec( + Arc::new(NotExpr::new( + col("non_nullable_col", schema.as_ref()).unwrap(), + )), + sort, + ); + + // let filter_exec = sort_exec; + let physical_plan = window_exec("non_nullable_col", sort_exprs, filter); + + let expected_input = vec![ + "WindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow }]", + " FilterExec: NOT non_nullable_col@1", + " SortExec: [non_nullable_col@1 ASC NULLS LAST], global=true", + " WindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow }]", + " SortExec: [non_nullable_col@1 DESC], global=true", + " MemoryExec: partitions=0, partition_sizes=[]", + ]; + + // let expected_optimized = vec![ + // "WindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(NULL) }]", + // " FilterExec: NOT non_nullable_col@1", + // " WindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow }]", + // " SortExec: [non_nullable_col@1 DESC]", + // " MemoryExec: partitions=0, partition_sizes=[]", + // ]; + assert_optimized!(expected_input, expected_input, physical_plan); + 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 = window_exec("non_nullable_col", sort_exprs1.clone(), sort1); + let window_agg2 = window_exec("non_nullable_col", sort_exprs2, window_agg1); + // let filter_exec = sort_exec; + let physical_plan = window_exec("non_nullable_col", sort_exprs1, window_agg2); + + let expected_input = vec![ + "WindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow }]", + " WindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow }]", + " WindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow }]", + " SortExec: [nullable_col@0 ASC], global=true", + " MemoryExec: partitions=0, partition_sizes=[]", + ]; + + let expected_optimized = vec![ + "WindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow }]", + " WindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow }]", + " WindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow }]", + " SortExec: [nullable_col@0 ASC,non_nullable_col@1 ASC], global=true", + " MemoryExec: partitions=0, partition_sizes=[]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan); + 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 = vec![ + "SortPreservingMergeExec: [nullable_col@0 ASC]", + " MemoryExec: partitions=0, partition_sizes=[]", + ]; + let expected_optimized = vec![ + "SortPreservingMergeExec: [nullable_col@0 ASC]", + " SortExec: [nullable_col@0 ASC], global=true", + " MemoryExec: partitions=0, partition_sizes=[]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan); + Ok(()) + } + + #[tokio::test] + async fn test_remove_unnecessary_sort1() -> 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 = vec![ + "SortExec: [nullable_col@0 ASC], global=true", + " SortExec: [non_nullable_col@1 ASC], global=true", + " MemoryExec: partitions=0, partition_sizes=[]", + ]; + // Keep the top SortExec + let expected_optimized = [ + "SortExec: [nullable_col@0 ASC], global=true", + " MemoryExec: partitions=0, partition_sizes=[]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan); + Ok(()) + } + + #[tokio::test] + async fn test_remove_unnecessary_sort2() -> Result<()> { + let schema = create_test_schema()?; + let source = memory_exec(&schema); + + let input = sort_exec(vec![sort_expr("non_nullable_col", &schema)], source); + let input2 = sort_exec( + vec![ + sort_expr("nullable_col", &schema), + sort_expr("non_nullable_col", &schema), + ], + input, + ); + let physical_plan = sort_exec(vec![sort_expr("nullable_col", &schema)], input2); + + let expected_input = vec![ + "SortExec: [nullable_col@0 ASC], global=true", + " SortExec: [nullable_col@0 ASC,non_nullable_col@1 ASC], global=true", + " SortExec: [non_nullable_col@1 ASC], global=true", + " MemoryExec: partitions=0, partition_sizes=[]", + ]; + // Keep the middle SortExec + let expected_optimized = [ + "SortExec: [nullable_col@0 ASC,non_nullable_col@1 ASC], global=true", + " MemoryExec: partitions=0, partition_sizes=[]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan); + 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("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 = vec![ + "SortPreservingMergeExec: [nullable_col@0 ASC]", + " SortExec: [nullable_col@0 ASC], global=true", + " SortPreservingMergeExec: [nullable_col@0 ASC]", + " SortExec: [nullable_col@0 ASC], global=true", + " MemoryExec: partitions=0, partition_sizes=[]", + ]; + let expected_optimized = vec![ + "SortPreservingMergeExec: [nullable_col@0 ASC]", + " SortExec: [nullable_col@0 ASC], global=true", + " MemoryExec: partitions=0, partition_sizes=[]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan); + Ok(()) + } + + #[tokio::test] + async fn test_remove_unnecessary_sort4() -> 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.clone(), spm2); + let physical_plan = sort_preserving_merge_exec(sort_exprs, sort3); + + let expected_input = vec![ + "SortPreservingMergeExec: [nullable_col@0 ASC]", + " SortExec: [nullable_col@0 ASC], global=true", + " SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", + " SortExec: [nullable_col@0 ASC,non_nullable_col@1 ASC], global=true", + " SortPreservingMergeExec: [non_nullable_col@1 ASC]", + " SortExec: [non_nullable_col@1 ASC], global=true", + " MemoryExec: partitions=0, partition_sizes=[]", + ]; + // Keep the middle SortPreservingMergeExec + SortExec + let expected_optimized = vec![ + "SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", + " SortExec: [nullable_col@0 ASC,non_nullable_col@1 ASC], global=true", + " MemoryExec: partitions=0, partition_sizes=[]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan); + Ok(()) + } + + #[tokio::test] + async fn test_remove_unnecessary_sort5() -> 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.clone(), spm2); + let physical_plan = repartition_exec(repartition_exec(sort3)); + + let expected_input = vec![ + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " SortExec: [nullable_col@0 ASC], global=true", + " SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", + " SortExec: [nullable_col@0 ASC,non_nullable_col@1 ASC], global=true", + " SortPreservingMergeExec: [non_nullable_col@1 ASC]", + " SortExec: [non_nullable_col@1 ASC], global=true", + " MemoryExec: partitions=0, partition_sizes=[]", + ]; + + let expected_optimized = vec![ + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=0", + " MemoryExec: partitions=0, partition_sizes=[]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan); + 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 physical_plan = sort_exec(vec![sort_expr("nullable_col", &schema)], input); + + let expected_input = vec![ + "SortExec: [nullable_col@0 ASC], global=true", + " SortPreservingMergeExec: [non_nullable_col@1 ASC]", + " MemoryExec: partitions=0, partition_sizes=[]", + ]; + let expected_optimized = vec![ + "SortExec: [nullable_col@0 ASC], global=true", + " MemoryExec: partitions=0, partition_sizes=[]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan); + Ok(()) + } + + #[tokio::test] + async fn test_remove_unnecessary_spm2() -> 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 = vec![ + "SortPreservingMergeExec: [nullable_col@0 ASC]", + " SortPreservingMergeExec: [non_nullable_col@1 ASC]", + " SortPreservingMergeExec: [non_nullable_col@1 ASC]", + " MemoryExec: partitions=0, partition_sizes=[]", + ]; + let expected_optimized = vec![ + "SortPreservingMergeExec: [nullable_col@0 ASC]", + " SortExec: [nullable_col@0 ASC], global=true", + " MemoryExec: partitions=0, partition_sizes=[]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan); + 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 = vec![ + "SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", + " SortExec: [nullable_col@0 ASC], global=true", + " MemoryExec: partitions=0, partition_sizes=[]", + ]; + let expected_optimized = vec![ + "SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", + " SortExec: [nullable_col@0 ASC,non_nullable_col@1 ASC], global=true", + " MemoryExec: partitions=0, partition_sizes=[]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan); + 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 = vec![ + "SortPreservingMergeExec: [non_nullable_col@1 ASC]", + " SortExec: [nullable_col@0 ASC], global=true", + " SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", + " MemoryExec: partitions=0, partition_sizes=[]", + ]; + let expected_optimized = vec![ + "SortPreservingMergeExec: [non_nullable_col@1 ASC]", + " SortExec: [non_nullable_col@1 ASC], global=true", + " MemoryExec: partitions=0, partition_sizes=[]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan); + 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: limit=None, partitions={1 group: [[x]]}, output_ordering=[nullable_col@0 ASC], projection=[nullable_col, non_nullable_col]", + " SortExec: [nullable_col@0 ASC], global=true", + " ParquetExec: limit=None, partitions={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); + 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: limit=None, partitions={1 group: [[x]]}, output_ordering=[nullable_col@0 ASC, non_nullable_col@1 ASC], projection=[nullable_col, non_nullable_col]", + " SortExec: [nullable_col@0 ASC], global=true", + " ParquetExec: limit=None, partitions={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); + 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 required ordering + // of SortPreservingMergeExec. Hence rule should remove unnecessary sort for second child of the UnionExec + // and put a sort above Union to satisfy required ordering. + let expected_input = vec![ + "SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", + " UnionExec", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, output_ordering=[nullable_col@0 ASC], projection=[nullable_col, non_nullable_col]", + " SortExec: [nullable_col@0 ASC,non_nullable_col@1 ASC], global=true", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + ]; + + let expected_optimized = vec![ + "SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", + " UnionExec", + " SortExec: [nullable_col@0 ASC,non_nullable_col@1 ASC], global=true", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, output_ordering=[nullable_col@0 ASC], projection=[nullable_col, non_nullable_col]", + " SortExec: [nullable_col@0 ASC,non_nullable_col@1 ASC], global=true", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan); + 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)]; + let sort_exprs2 = vec![ + sort_expr("nullable_col", &schema), + sort_expr("non_nullable_col", &schema), + ]; + let sort = sort_exec(sort_exprs1.clone(), source1); + + let source2 = parquet_exec_sorted(&schema, sort_exprs2.clone()); + + let union = union_exec(vec![source2, sort]); + let physical_plan = sort_preserving_merge_exec(sort_exprs2, union); + + // Input is an invalid plan. In this case rule should add required sorting in appropriate places. + let expected_input = vec![ + "SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", + " UnionExec", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, output_ordering=[nullable_col@0 ASC, non_nullable_col@1 ASC], projection=[nullable_col, non_nullable_col]", + " SortExec: [nullable_col@0 ASC], global=true", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + ]; + // expect to replace the wrong SortExec with the correct one + let expected_optimized = vec![ + "SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", + " UnionExec", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, output_ordering=[nullable_col@0 ASC, non_nullable_col@1 ASC], projection=[nullable_col, non_nullable_col]", + " SortExec: [nullable_col@0 ASC,non_nullable_col@1 ASC], global=true", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan); + 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 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 = vec![ + "SortPreservingMergeExec: [nullable_col@0 ASC]", + " UnionExec", + " SortExec: [nullable_col@0 ASC,non_nullable_col@1 ASC], global=true", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, output_ordering=[nullable_col@0 ASC], projection=[nullable_col, non_nullable_col]", + " SortExec: [nullable_col@0 ASC], global=true", + " ParquetExec: limit=None, partitions={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 = vec![ + "SortPreservingMergeExec: [nullable_col@0 ASC]", + " UnionExec", + " SortExec: [nullable_col@0 ASC,non_nullable_col@1 ASC], global=true", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, output_ordering=[nullable_col@0 ASC], projection=[nullable_col, non_nullable_col]", + " SortExec: [nullable_col@0 ASC], global=true", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan); + 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)]; + 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); + + // 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 = vec![ + "SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", + " UnionExec", + " SortExec: [nullable_col@0 ASC], global=true", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, output_ordering=[nullable_col@0 ASC], projection=[nullable_col, non_nullable_col]", + " SortExec: [nullable_col@0 ASC], global=true", + " ParquetExec: limit=None, partitions={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 = vec![ + "SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", + " UnionExec", + " SortExec: [nullable_col@0 ASC,non_nullable_col@1 ASC], global=true", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " SortExec: [nullable_col@0 ASC,non_nullable_col@1 ASC], global=true", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, output_ordering=[nullable_col@0 ASC], projection=[nullable_col, non_nullable_col]", + " SortExec: [nullable_col@0 ASC,non_nullable_col@1 ASC], global=true", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan); + 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), + 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); + + // Union doesn't preserve any of the inputs ordering. However, we should be able to change unnecessarily fine + // SortExecs under UnionExec with required SortExecs that are absolutely necessary. + let expected_input = vec![ + "SortPreservingMergeExec: [nullable_col@0 ASC]", + " UnionExec", + " SortExec: [nullable_col@0 ASC,non_nullable_col@1 ASC], global=true", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " SortExec: [nullable_col@0 ASC,non_nullable_col@1 DESC NULLS LAST], global=true", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + ]; + assert_optimized!(expected_input, expected_input, physical_plan); + 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 = vec![ + "SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", + " UnionExec", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, output_ordering=[nullable_col@0 ASC], projection=[nullable_col, non_nullable_col]", + " GlobalLimitExec: skip=0, fetch=100", + " LocalLimitExec: fetch=100", + " SortExec: [nullable_col@0 ASC,non_nullable_col@1 ASC], global=true", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + ]; + + // expect to keep the bottom SortExec + let expected_optimized = vec![ + "SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", + " SortExec: [nullable_col@0 ASC,non_nullable_col@1 ASC], global=false", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", + " UnionExec", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, output_ordering=[nullable_col@0 ASC], projection=[nullable_col, non_nullable_col]", + " GlobalLimitExec: skip=0, fetch=100", + " LocalLimitExec: fetch=100", + " SortExec: [nullable_col@0 ASC,non_nullable_col@1 ASC], global=true", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan); + Ok(()) + } + + /// make PhysicalSortExpr with default options + fn sort_expr(name: &str, schema: &Schema) -> PhysicalSortExpr { + sort_expr_options(name, schema, SortOptions::default()) + } + + /// PhysicalSortExpr with specified options + fn sort_expr_options( + name: &str, + schema: &Schema, + options: SortOptions, + ) -> PhysicalSortExpr { + PhysicalSortExpr { + expr: col(name, schema).unwrap(), + options, + } + } + + fn memory_exec(schema: &SchemaRef) -> Arc { + Arc::new(MemoryExec::try_new(&[], schema.clone(), None).unwrap()) + } + + fn sort_exec( + sort_exprs: impl IntoIterator, + input: Arc, + ) -> Arc { + let sort_exprs = sort_exprs.into_iter().collect(); + Arc::new(SortExec::try_new(sort_exprs, input, None).unwrap()) + } + + 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 filter_exec( + predicate: Arc, + input: Arc, + ) -> Arc { + Arc::new(FilterExec::try_new(predicate, input).unwrap()) + } + + fn limit_exec(input: Arc) -> Arc { + Arc::new(GlobalLimitExec::new( + Arc::new(LocalLimitExec::new(input, 100)), + 0, + Some(100), + )) + } + + fn repartition_exec(input: Arc) -> Arc { + Arc::new( + RepartitionExec::try_new(input, Partitioning::RoundRobinBatch(10)).unwrap(), + ) + } + + fn window_exec( + col_name: &str, + sort_exprs: impl IntoIterator, + input: Arc, + ) -> Arc { + let sort_exprs: Vec<_> = sort_exprs.into_iter().collect(); + let schema = input.schema(); + + Arc::new( + WindowAggExec::try_new( + vec![create_window_expr( + &WindowFunction::AggregateFunction(AggregateFunction::Count), + "count".to_owned(), + &[col(col_name, &schema).unwrap()], + &[], + &sort_exprs, + Arc::new(WindowFrame::new(true)), + schema.as_ref(), + ) + .unwrap()], + input.clone(), + input.schema(), + vec![], + Some(sort_exprs), + ) + .unwrap(), + ) + } + + /// Create a non sorted parquet exec + fn parquet_exec(schema: &SchemaRef) -> Arc { + Arc::new(ParquetExec::new( + FileScanConfig { + object_store_url: ObjectStoreUrl::parse("test:///").unwrap(), + file_schema: schema.clone(), + file_groups: vec![vec![PartitionedFile::new("x".to_string(), 100)]], + statistics: Statistics::default(), + projection: None, + limit: None, + table_partition_cols: vec![], + output_ordering: None, + infinite_source: false, + }, + None, + None, + )) + } + + // Created a sorted parquet exec + fn parquet_exec_sorted( + schema: &SchemaRef, + sort_exprs: impl IntoIterator, + ) -> Arc { + let sort_exprs = sort_exprs.into_iter().collect(); + + Arc::new(ParquetExec::new( + FileScanConfig { + object_store_url: ObjectStoreUrl::parse("test:///").unwrap(), + file_schema: schema.clone(), + file_groups: vec![vec![PartitionedFile::new("x".to_string(), 100)]], + statistics: Statistics::default(), + projection: None, + limit: None, + table_partition_cols: vec![], + output_ordering: Some(sort_exprs), + infinite_source: false, + }, + None, + None, + )) + } + + fn union_exec(input: Vec>) -> Arc { + Arc::new(UnionExec::new(input)) + } +} diff --git a/datafusion/core/src/physical_optimizer/utils.rs b/datafusion/core/src/physical_optimizer/utils.rs index 13e04bbc2ae83..0fcb5c411a18c 100644 --- a/datafusion/core/src/physical_optimizer/utils.rs +++ b/datafusion/core/src/physical_optimizer/utils.rs @@ -53,16 +53,17 @@ pub fn optimize_children( pub fn add_sort_above_child( child: &Arc, sort_expr: Vec, + fetch: Option, ) -> Result> { let new_child = if child.output_partitioning().partition_count() > 1 { Arc::new(SortExec::new_with_partitioning( sort_expr, child.clone(), true, - None, + fetch, )) as Arc } else { - Arc::new(SortExec::try_new(sort_expr, child.clone(), None)?) + Arc::new(SortExec::try_new(sort_expr, child.clone(), fetch)?) as Arc }; Ok(new_child) diff --git a/datafusion/core/src/physical_plan/joins/sort_merge_join.rs b/datafusion/core/src/physical_plan/joins/sort_merge_join.rs index 28df317a8ac3b..edbfdf042c331 100644 --- a/datafusion/core/src/physical_plan/joins/sort_merge_join.rs +++ b/datafusion/core/src/physical_plan/joins/sort_merge_join.rs @@ -34,6 +34,7 @@ use arrow::compute::{concat_batches, take, SortOptions}; use arrow::datatypes::{DataType, SchemaRef, TimeUnit}; use arrow::error::{ArrowError, Result as ArrowResult}; use arrow::record_batch::RecordBatch; +use datafusion_physical_expr::{new_sort_requirements, PhysicalSortRequirements}; use futures::{Stream, StreamExt}; use crate::error::DataFusionError; @@ -221,8 +222,10 @@ impl ExecutionPlan for SortMergeJoinExec { ] } - fn required_input_ordering(&self) -> Vec> { - vec![Some(&self.left_sort_exprs), Some(&self.right_sort_exprs)] + fn required_input_ordering(&self) -> Vec>> { + let left_requirements = new_sort_requirements(Some(&self.left_sort_exprs)); + let right_requirements = new_sort_requirements(Some(&self.right_sort_exprs)); + vec![left_requirements, right_requirements] } fn output_partitioning(&self) -> Partitioning { @@ -239,6 +242,17 @@ impl ExecutionPlan for SortMergeJoinExec { self.output_ordering.as_deref() } + fn maintains_input_order(&self) -> Vec { + match self.join_type { + JoinType::Inner => vec![true, true], + JoinType::Left | JoinType::LeftSemi | JoinType::LeftAnti => vec![true, false], + JoinType::Right | JoinType::RightSemi | JoinType::RightAnti => { + vec![false, true] + } + _ => vec![false, false], + } + } + fn equivalence_properties(&self) -> EquivalenceProperties { let left_columns_len = self.left.schema().fields.len(); combine_join_equivalence_properties( diff --git a/datafusion/core/src/physical_plan/mod.rs b/datafusion/core/src/physical_plan/mod.rs index 0ec7b16ef1316..e02ed421bf4d9 100644 --- a/datafusion/core/src/physical_plan/mod.rs +++ b/datafusion/core/src/physical_plan/mod.rs @@ -142,7 +142,7 @@ pub trait ExecutionPlan: Debug + Send + Sync { /// NOTE that checking `!is_empty()` does **not** check for a /// required input ordering. Instead, the correct check is that at /// least one entry must be `Some` - fn required_input_ordering(&self) -> Vec> { + fn required_input_ordering(&self) -> Vec>> { vec![None; self.children().len()] } @@ -592,11 +592,11 @@ impl Distribution { use datafusion_physical_expr::expressions::Column; pub use datafusion_physical_expr::window::WindowExpr; -use datafusion_physical_expr::EquivalenceProperties; use datafusion_physical_expr::{ expr_list_eq_strict_order, normalize_expr_with_equivalence_properties, }; pub use datafusion_physical_expr::{AggregateExpr, PhysicalExpr}; +use datafusion_physical_expr::{EquivalenceProperties, PhysicalSortRequirements}; /// Applies an optional projection to a [`SchemaRef`], returning the /// projected schema diff --git a/datafusion/core/src/physical_plan/planner.rs b/datafusion/core/src/physical_plan/planner.rs index a3cd42ecba28e..5370ab0a8c70b 100644 --- a/datafusion/core/src/physical_plan/planner.rs +++ b/datafusion/core/src/physical_plan/planner.rs @@ -1893,6 +1893,7 @@ mod tests { let session_state = make_session_state(); // optimize the logical plan let logical_plan = session_state.optimize(logical_plan)?; + println!("optimized logical plan {:?}", logical_plan); let planner = DefaultPhysicalPlanner::default(); planner .create_physical_plan(&logical_plan, &session_state) diff --git a/datafusion/core/src/physical_plan/sorts/sort.rs b/datafusion/core/src/physical_plan/sorts/sort.rs index b75fe0d80d6c1..4b6199cd88310 100644 --- a/datafusion/core/src/physical_plan/sorts/sort.rs +++ b/datafusion/core/src/physical_plan/sorts/sort.rs @@ -798,7 +798,12 @@ impl ExecutionPlan for SortExec { match t { DisplayFormatType::Default => { let expr: Vec = self.expr.iter().map(|e| e.to_string()).collect(); - write!(f, "SortExec: [{}]", expr.join(",")) + write!( + f, + "SortExec: [{}], global={}", + expr.join(","), + !self.preserve_partitioning + ) } } } diff --git a/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs b/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs index 658a5f9fc1767..38a67cb4a7f49 100644 --- a/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs +++ b/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs @@ -48,7 +48,9 @@ use crate::physical_plan::{ Distribution, ExecutionPlan, Partitioning, PhysicalExpr, RecordBatchStream, SendableRecordBatchStream, Statistics, }; -use datafusion_physical_expr::EquivalenceProperties; +use datafusion_physical_expr::{ + new_sort_requirements, EquivalenceProperties, PhysicalSortRequirements, +}; /// Sort preserving merge execution plan /// @@ -127,12 +129,17 @@ impl ExecutionPlan for SortPreservingMergeExec { vec![Distribution::UnspecifiedDistribution] } - fn required_input_ordering(&self) -> Vec> { - vec![Some(&self.expr)] + fn required_input_ordering(&self) -> Vec>> { + let ordering_requirements = new_sort_requirements(Some(&self.expr)); + vec![ordering_requirements] } fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - Some(&self.expr) + self.input.output_ordering() + } + + fn maintains_input_order(&self) -> Vec { + vec![true] } fn equivalence_properties(&self) -> EquivalenceProperties { diff --git a/datafusion/core/src/physical_plan/union.rs b/datafusion/core/src/physical_plan/union.rs index df78058082f59..c05700e61bf78 100644 --- a/datafusion/core/src/physical_plan/union.rs +++ b/datafusion/core/src/physical_plan/union.rs @@ -48,7 +48,6 @@ use crate::{ error::Result, physical_plan::{expressions, metrics::BaselineMetrics}, }; -use datafusion_physical_expr::utils::ordering_satisfy; use tokio::macros::support::thread_rng_n; /// `UnionExec`: `UNION ALL` execution plan. @@ -232,22 +231,8 @@ impl ExecutionPlan for UnionExec { } fn maintains_input_order(&self) -> Vec { - // If the Union has an output ordering, it maintains at least one - // child's ordering (i.e. the meet). - // For instance, assume that the first child is SortExpr('a','b','c'), - // the second child is SortExpr('a','b') and the third child is - // SortExpr('a','b'). The output ordering would be SortExpr('a','b'), - // which is the "meet" of all input orderings. In this example, this - // function will return vec![false, true, true], indicating that we - // preserve the orderings for the 2nd and the 3rd children. - self.inputs() - .iter() - .map(|child| { - ordering_satisfy(self.output_ordering(), child.output_ordering(), || { - child.equivalence_properties() - }) - }) - .collect() + let main_input_order = self.output_ordering().is_some(); + vec![main_input_order; self.inputs.len()] } fn with_new_children( diff --git a/datafusion/core/src/physical_plan/windows/bounded_window_agg_exec.rs b/datafusion/core/src/physical_plan/windows/bounded_window_agg_exec.rs index 7fc3c638097fa..b4e4685228b19 100644 --- a/datafusion/core/src/physical_plan/windows/bounded_window_agg_exec.rs +++ b/datafusion/core/src/physical_plan/windows/bounded_window_agg_exec.rs @@ -54,7 +54,9 @@ use datafusion_physical_expr::window::{ PartitionBatchState, PartitionBatches, PartitionKey, PartitionWindowAggStates, WindowAggState, WindowState, }; -use datafusion_physical_expr::{EquivalenceProperties, PhysicalExpr}; +use datafusion_physical_expr::{ + EquivalenceProperties, PhysicalExpr, PhysicalSortRequirements, +}; use indexmap::IndexMap; use log::debug; @@ -123,7 +125,7 @@ impl BoundedWindowAggExec { let mut result = vec![]; // All window exprs have the same partition by, so we just use the first one: let partition_by = self.window_expr()[0].partition_by(); - let sort_keys = self.sort_keys.as_deref().unwrap_or(&[]); + let sort_keys = self.output_ordering().unwrap_or(&[]); for item in partition_by { if let Some(a) = sort_keys.iter().find(|&e| e.expr.eq(item)) { result.push(a.clone()); @@ -167,9 +169,28 @@ impl ExecutionPlan for BoundedWindowAggExec { self.input().output_ordering() } - fn required_input_ordering(&self) -> Vec> { - let sort_keys = self.sort_keys.as_deref(); - vec![sort_keys] + fn required_input_ordering(&self) -> Vec>> { + let partition_keys = self.window_expr()[0].partition_by(); + let requirements = self.sort_keys.as_deref().map(|ordering| { + ordering + .iter() + .map(|o| { + let is_partition = partition_keys.iter().any(|e| e.eq(&o.expr)); + if is_partition { + PhysicalSortRequirements { + expr: o.expr.clone(), + sort_options: None, + } + } else { + PhysicalSortRequirements { + expr: o.expr.clone(), + sort_options: Some(o.options.clone()), + } + } + }) + .collect::>() + }); + vec![requirements] } fn required_input_distribution(&self) -> Vec { @@ -177,7 +198,6 @@ impl ExecutionPlan for BoundedWindowAggExec { debug!("No partition defined for BoundedWindowAggExec!!!"); vec![Distribution::SinglePartition] } else { - //TODO support PartitionCollections if there is no common partition columns in the window_expr vec![Distribution::HashPartitioned(self.partition_keys.clone())] } } @@ -436,6 +456,7 @@ impl SortedPartitionByBoundedWindowStream { ) -> Self { let state = window_expr.iter().map(|_| IndexMap::new()).collect(); let empty_batch = RecordBatch::new_empty(schema.clone()); + println!("partition_by_sort_keys {:?}", partition_by_sort_keys); Self { schema, input, diff --git a/datafusion/core/src/physical_plan/windows/window_agg_exec.rs b/datafusion/core/src/physical_plan/windows/window_agg_exec.rs index fbd05fa884857..e5a17043f6cc7 100644 --- a/datafusion/core/src/physical_plan/windows/window_agg_exec.rs +++ b/datafusion/core/src/physical_plan/windows/window_agg_exec.rs @@ -39,6 +39,7 @@ use arrow::{ record_batch::RecordBatch, }; use datafusion_common::DataFusionError; +use datafusion_physical_expr::PhysicalSortRequirements; use futures::stream::Stream; use futures::{ready, StreamExt}; use log::debug; @@ -114,7 +115,7 @@ impl WindowAggExec { let mut result = vec![]; // All window exprs have the same partition by, so we just use the first one: let partition_by = self.window_expr()[0].partition_by(); - let sort_keys = self.sort_keys.as_deref().unwrap_or(&[]); + let sort_keys = self.output_ordering().unwrap_or(&[]); for item in partition_by { if let Some(a) = sort_keys.iter().find(|&e| e.expr.eq(item)) { result.push(a.clone()); @@ -172,9 +173,28 @@ impl ExecutionPlan for WindowAggExec { vec![true] } - fn required_input_ordering(&self) -> Vec> { - let sort_keys = self.sort_keys.as_deref(); - vec![sort_keys] + fn required_input_ordering(&self) -> Vec>> { + let partition_keys = self.window_expr()[0].partition_by(); + let requirements = self.sort_keys.as_deref().map(|ordering| { + ordering + .iter() + .map(|o| { + let is_partition = partition_keys.iter().any(|e| e.eq(&o.expr)); + if is_partition { + PhysicalSortRequirements { + expr: o.expr.clone(), + sort_options: None, + } + } else { + PhysicalSortRequirements { + expr: o.expr.clone(), + sort_options: Some(o.options.clone()), + } + } + }) + .collect::>() + }); + vec![requirements] } fn required_input_distribution(&self) -> Vec { @@ -182,7 +202,6 @@ impl ExecutionPlan for WindowAggExec { debug!("No partition defined for WindowAggExec!!!"); vec![Distribution::SinglePartition] } else { - //TODO support PartitionCollections if there is no common partition columns in the window_expr vec![Distribution::HashPartitioned(self.partition_keys.clone())] } } diff --git a/datafusion/core/tests/sql/explain_analyze.rs b/datafusion/core/tests/sql/explain_analyze.rs index 01bd94e8e4bbe..9d75cb974052f 100644 --- a/datafusion/core/tests/sql/explain_analyze.rs +++ b/datafusion/core/tests/sql/explain_analyze.rs @@ -605,7 +605,7 @@ async fn test_physical_plan_display_indent() { let expected = vec![ "GlobalLimitExec: skip=0, fetch=10", " SortPreservingMergeExec: [the_min@2 DESC]", - " SortExec: [the_min@2 DESC]", + " SortExec: [the_min@2 DESC], global=false", " ProjectionExec: expr=[c1@0 as c1, MAX(aggregate_test_100.c12)@1 as MAX(aggregate_test_100.c12), MIN(aggregate_test_100.c12)@2 as the_min]", " AggregateExec: mode=FinalPartitioned, gby=[c1@0 as c1], aggr=[MAX(aggregate_test_100.c12), MIN(aggregate_test_100.c12)]", " CoalesceBatchesExec: target_batch_size=4096", diff --git a/datafusion/core/tests/sql/joins.rs b/datafusion/core/tests/sql/joins.rs index e0bd1a523c4ad..06489d3082744 100644 --- a/datafusion/core/tests/sql/joins.rs +++ b/datafusion/core/tests/sql/joins.rs @@ -1881,12 +1881,12 @@ async fn sort_merge_join_on_date32() -> Result<()> { let expected = vec![ "ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, c3@2 as c3, c4@3 as c4, c1@4 as c1, c2@5 as c2, c3@6 as c3, c4@7 as c4]", " SortMergeJoin: join_type=Inner, on=[(Column { name: \"c1\", index: 0 }, Column { name: \"c1\", index: 0 })]", - " SortExec: [c1@0 ASC]", + " SortExec: [c1@0 ASC], global=false", " CoalesceBatchesExec: target_batch_size=4096", " RepartitionExec: partitioning=Hash([Column { name: \"c1\", index: 0 }], 2), input_partitions=2", " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", " MemoryExec: partitions=1, partition_sizes=[1]", - " SortExec: [c1@0 ASC]", + " SortExec: [c1@0 ASC], global=false", " CoalesceBatchesExec: target_batch_size=4096", " RepartitionExec: partitioning=Hash([Column { name: \"c1\", index: 0 }], 2), input_partitions=2", " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", @@ -1927,13 +1927,13 @@ async fn sort_merge_join_on_decimal() -> Result<()> { "ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, c3@2 as c3, c4@3 as c4, c1@4 as c1, c2@5 as c2, c3@6 as c3, c4@7 as c4]", " ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, c3@2 as c3, c4@3 as c4, c1@5 as c1, c2@6 as c2, c3@7 as c3, c4@8 as c4]", " SortMergeJoin: join_type=Right, on=[(Column { name: \"CAST(t1.c3 AS Decimal128(10, 2))\", index: 4 }, Column { name: \"c3\", index: 2 })]", - " SortExec: [CAST(t1.c3 AS Decimal128(10, 2))@4 ASC]", + " SortExec: [CAST(t1.c3 AS Decimal128(10, 2))@4 ASC], global=false", " CoalesceBatchesExec: target_batch_size=4096", " RepartitionExec: partitioning=Hash([Column { name: \"CAST(t1.c3 AS Decimal128(10, 2))\", index: 4 }], 2), input_partitions=2", " ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, c3@2 as c3, c4@3 as c4, CAST(c3@2 AS Decimal128(10, 2)) as CAST(t1.c3 AS Decimal128(10, 2))]", " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", " MemoryExec: partitions=1, partition_sizes=[1]", - " SortExec: [c3@2 ASC]", + " SortExec: [c3@2 ASC], global=false", " CoalesceBatchesExec: target_batch_size=4096", " RepartitionExec: partitioning=Hash([Column { name: \"c3\", index: 2 }], 2), input_partitions=2", " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", @@ -1980,7 +1980,7 @@ async fn left_semi_join() -> Result<()> { let physical_plan = dataframe.create_physical_plan().await?; let expected = if repartition_joins { vec![ - "SortExec: [t1_id@0 ASC NULLS LAST]", + "SortExec: [t1_id@0 ASC NULLS LAST], global=true", " CoalescePartitionsExec", " ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name]", " CoalesceBatchesExec: target_batch_size=4096", @@ -1997,7 +1997,7 @@ async fn left_semi_join() -> Result<()> { ] } else { vec![ - "SortExec: [t1_id@0 ASC NULLS LAST]", + "SortExec: [t1_id@0 ASC NULLS LAST], global=true", " CoalescePartitionsExec", " ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name]", " CoalesceBatchesExec: target_batch_size=4096", @@ -2062,7 +2062,7 @@ async fn left_semi_join() -> Result<()> { let physical_plan = dataframe.create_physical_plan().await?; let expected = if repartition_joins { vec![ - "SortExec: [t1_id@0 ASC NULLS LAST]", + "SortExec: [t1_id@0 ASC NULLS LAST], global=true", " CoalescePartitionsExec", " ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name]", " CoalesceBatchesExec: target_batch_size=4096", @@ -2078,7 +2078,7 @@ async fn left_semi_join() -> Result<()> { ] } else { vec![ - "SortExec: [t1_id@0 ASC NULLS LAST]", + "SortExec: [t1_id@0 ASC NULLS LAST], global=true", " CoalescePartitionsExec", " ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name]", " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", @@ -2259,7 +2259,7 @@ async fn right_semi_join() -> Result<()> { let dataframe = ctx.sql(sql).await.expect(&msg); let physical_plan = dataframe.create_physical_plan().await?; let expected = if repartition_joins { - vec![ "SortExec: [t1_id@0 ASC NULLS LAST]", + vec![ "SortExec: [t1_id@0 ASC NULLS LAST], global=true", " CoalescePartitionsExec", " ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int]", " CoalesceBatchesExec: target_batch_size=4096", @@ -2275,7 +2275,7 @@ async fn right_semi_join() -> Result<()> { ] } else { vec![ - "SortExec: [t1_id@0 ASC NULLS LAST]", + "SortExec: [t1_id@0 ASC NULLS LAST], global=true", " CoalescePartitionsExec", " ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int]", " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", @@ -2307,7 +2307,7 @@ async fn right_semi_join() -> Result<()> { let dataframe = ctx.sql(sql).await.expect(&msg); let physical_plan = dataframe.create_physical_plan().await?; let expected = if repartition_joins { - vec![ "SortExec: [t1_id@0 ASC NULLS LAST]", + vec![ "SortExec: [t1_id@0 ASC NULLS LAST], global=true", " CoalescePartitionsExec", " ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int]", " CoalesceBatchesExec: target_batch_size=4096", @@ -2323,7 +2323,7 @@ async fn right_semi_join() -> Result<()> { ] } else { vec![ - "SortExec: [t1_id@0 ASC NULLS LAST]", + "SortExec: [t1_id@0 ASC NULLS LAST], global=true", " CoalescePartitionsExec", " ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int]", " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", diff --git a/datafusion/core/tests/sql/window.rs b/datafusion/core/tests/sql/window.rs index 22feeed2cb49b..22ed983b6b1ad 100644 --- a/datafusion/core/tests/sql/window.rs +++ b/datafusion/core/tests/sql/window.rs @@ -1686,7 +1686,7 @@ async fn test_window_agg_sort() -> Result<()> { " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", " BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9): Ok(Field { name: \"SUM(aggregate_test_100.c9)\", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(NULL)), end_bound: CurrentRow }]", " BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9): Ok(Field { name: \"SUM(aggregate_test_100.c9)\", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(NULL)), end_bound: CurrentRow }]", - " SortExec: [c9@1 ASC NULLS LAST,c8@0 ASC NULLS LAST]", + " SortExec: [c9@1 ASC NULLS LAST,c8@0 ASC NULLS LAST], global=true", ] }; @@ -1719,7 +1719,7 @@ async fn over_order_by_sort_keys_sorting_prefix_compacting() -> Result<()> { " WindowAggExec: wdw=[SUM(aggregate_test_100.c9): Ok(Field { name: \"SUM(aggregate_test_100.c9)\", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)) }]", " BoundedWindowAggExec: wdw=[MAX(aggregate_test_100.c9): Ok(Field { name: \"MAX(aggregate_test_100.c9)\", data_type: UInt32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(NULL)), end_bound: CurrentRow }]", " BoundedWindowAggExec: wdw=[MIN(aggregate_test_100.c9): Ok(Field { name: \"MIN(aggregate_test_100.c9)\", data_type: UInt32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(NULL)), end_bound: CurrentRow }]", - " SortExec: [c2@0 ASC NULLS LAST,c9@1 ASC NULLS LAST]", + " SortExec: [c2@0 ASC NULLS LAST,c9@1 ASC NULLS LAST], global=true", ] }; @@ -1747,15 +1747,15 @@ async fn over_order_by_sort_keys_sorting_global_order_compacting() -> Result<()> // 3 SortExec are added let expected = { vec![ - "SortExec: [c2@0 ASC NULLS LAST]", + "SortExec: [c2@0 ASC NULLS LAST], global=true", " CoalescePartitionsExec", " ProjectionExec: expr=[c2@0 as c2, MAX(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as MAX(aggregate_test_100.c9), SUM(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@4 as SUM(aggregate_test_100.c9), MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as MIN(aggregate_test_100.c9)]", " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", " WindowAggExec: wdw=[SUM(aggregate_test_100.c9): Ok(Field { name: \"SUM(aggregate_test_100.c9)\", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)) }]", " BoundedWindowAggExec: wdw=[MAX(aggregate_test_100.c9): Ok(Field { name: \"MAX(aggregate_test_100.c9)\", data_type: UInt32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(NULL)), end_bound: CurrentRow }]", - " SortExec: [c9@1 ASC NULLS LAST,c2@0 ASC NULLS LAST]", + " SortExec: [c9@1 ASC NULLS LAST,c2@0 ASC NULLS LAST], global=true", " BoundedWindowAggExec: wdw=[MIN(aggregate_test_100.c9): Ok(Field { name: \"MIN(aggregate_test_100.c9)\", data_type: UInt32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(NULL)), end_bound: CurrentRow }]", - " SortExec: [c2@0 ASC NULLS LAST,c9@1 ASC NULLS LAST]", + " SortExec: [c2@0 ASC NULLS LAST,c9@1 ASC NULLS LAST], global=true", ] }; @@ -1791,11 +1791,11 @@ async fn test_window_partition_by_order_by() -> Result<()> { vec![ "ProjectionExec: expr=[SUM(aggregate_test_100.c4) PARTITION BY [aggregate_test_100.c1, aggregate_test_100.c2] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@3 as SUM(aggregate_test_100.c4), COUNT(UInt8(1)) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING@4 as COUNT(UInt8(1))]", " BoundedWindowAggExec: wdw=[COUNT(UInt8(1)): Ok(Field { name: \"COUNT(UInt8(1))\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)) }]", - " SortExec: [c1@0 ASC NULLS LAST,c2@1 ASC NULLS LAST]", + " SortExec: [c1@0 ASC NULLS LAST,c2@1 ASC NULLS LAST], global=false", " CoalesceBatchesExec: target_batch_size=4096", " RepartitionExec: partitioning=Hash([Column { name: \"c1\", index: 0 }], 2), input_partitions=2", " BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c4): Ok(Field { name: \"SUM(aggregate_test_100.c4)\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(1)) }]", - " SortExec: [c1@0 ASC NULLS LAST,c2@1 ASC NULLS LAST]", + " SortExec: [c1@0 ASC NULLS LAST,c2@1 ASC NULLS LAST], global=false", " CoalesceBatchesExec: target_batch_size=4096", " RepartitionExec: partitioning=Hash([Column { name: \"c1\", index: 0 }, Column { name: \"c2\", index: 1 }], 2), input_partitions=2", " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", @@ -1835,7 +1835,7 @@ async fn test_window_agg_sort_reversed_plan() -> Result<()> { " GlobalLimitExec: skip=0, fetch=5", " BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9): Ok(Field { name: \"SUM(aggregate_test_100.c9)\", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: Following(UInt64(1)) }]", " BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9): Ok(Field { name: \"SUM(aggregate_test_100.c9)\", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)) }]", - " SortExec: [c9@0 DESC]", + " SortExec: [c9@0 DESC], global=true", ] }; @@ -1891,7 +1891,7 @@ async fn test_window_agg_sort_reversed_plan_builtin() -> Result<()> { " GlobalLimitExec: skip=0, fetch=5", " BoundedWindowAggExec: wdw=[FIRST_VALUE(aggregate_test_100.c9): Ok(Field { name: \"FIRST_VALUE(aggregate_test_100.c9)\", data_type: UInt32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: Following(UInt64(1)) }, LAG(aggregate_test_100.c9,Int64(2),Int64(10101)): Ok(Field { name: \"LAG(aggregate_test_100.c9,Int64(2),Int64(10101))\", data_type: UInt32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(UInt32(NULL)) }, LEAD(aggregate_test_100.c9,Int64(2),Int64(10101)): Ok(Field { name: \"LEAD(aggregate_test_100.c9,Int64(2),Int64(10101))\", data_type: UInt32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(UInt32(NULL)) }]", " BoundedWindowAggExec: wdw=[FIRST_VALUE(aggregate_test_100.c9): Ok(Field { name: \"FIRST_VALUE(aggregate_test_100.c9)\", data_type: UInt32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)) }, LAG(aggregate_test_100.c9,Int64(2),Int64(10101)): Ok(Field { name: \"LAG(aggregate_test_100.c9,Int64(2),Int64(10101))\", data_type: UInt32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)) }, LEAD(aggregate_test_100.c9,Int64(2),Int64(10101)): Ok(Field { name: \"LEAD(aggregate_test_100.c9,Int64(2),Int64(10101))\", data_type: UInt32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)) }]", - " SortExec: [c9@0 DESC]", + " SortExec: [c9@0 DESC], global=true", ] }; @@ -1942,9 +1942,9 @@ async fn test_window_agg_sort_non_reversed_plan() -> Result<()> { " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", " GlobalLimitExec: skip=0, fetch=5", " BoundedWindowAggExec: wdw=[ROW_NUMBER(): Ok(Field { name: \"ROW_NUMBER()\", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)) }]", - " SortExec: [c9@0 ASC NULLS LAST]", + " SortExec: [c9@0 ASC NULLS LAST], global=true", " BoundedWindowAggExec: wdw=[ROW_NUMBER(): Ok(Field { name: \"ROW_NUMBER()\", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)) }]", - " SortExec: [c9@0 DESC]", + " SortExec: [c9@0 DESC], global=true", ] }; @@ -1995,11 +1995,10 @@ async fn test_window_agg_sort_multi_layer_non_reversed_plan() -> Result<()> { "ProjectionExec: expr=[c9@2 as c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@5 as sum1, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c1 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@3 as sum2, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@4 as rn2]", " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", " GlobalLimitExec: skip=0, fetch=5", - " BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9): Ok(Field { name: \"SUM(aggregate_test_100.c9)\", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)) }]", - " SortExec: [c9@2 ASC NULLS LAST,c1@0 ASC NULLS LAST,c2@1 ASC NULLS LAST]", - " BoundedWindowAggExec: wdw=[ROW_NUMBER(): Ok(Field { name: \"ROW_NUMBER()\", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)) }]", - " BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9): Ok(Field { name: \"SUM(aggregate_test_100.c9)\", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)) }]", - " SortExec: [c9@2 DESC,c1@0 DESC]", + " BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9): Ok(Field { name: \"SUM(aggregate_test_100.c9)\", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: Following(UInt64(1)) }]", + " BoundedWindowAggExec: wdw=[ROW_NUMBER(): Ok(Field { name: \"ROW_NUMBER()\", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)) }]", + " BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9): Ok(Field { name: \"SUM(aggregate_test_100.c9)\", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)) }]", + " SortExec: [c9@2 DESC,c1@0 DESC,c2@1 DESC], global=true", ] }; @@ -2013,15 +2012,15 @@ async fn test_window_agg_sort_multi_layer_non_reversed_plan() -> Result<()> { let actual = execute_to_batches(&ctx, sql).await; let expected = vec![ - "+-----------+------------+-----------+-----+", - "| c9 | sum1 | sum2 | rn2 |", - "+-----------+------------+-----------+-----+", - "| 28774375 | 745354217 | 91818943 | 100 |", - "| 63044568 | 988558066 | 232866360 | 99 |", - "| 141047417 | 1285934966 | 374546521 | 98 |", - "| 141680161 | 1654839259 | 519841132 | 97 |", - "| 145294611 | 1980231675 | 745354217 | 96 |", - "+-----------+------------+-----------+-----+", + "+------------+-------------+-------------+-----+", + "| c9 | sum1 | sum2 | rn2 |", + "+------------+-------------+-------------+-----+", + "| 4268716378 | 8498370520 | 24997484146 | 1 |", + "| 4229654142 | 12714811027 | 29012926487 | 2 |", + "| 4216440507 | 16858984380 | 28743001064 | 3 |", + "| 4144173353 | 20935849039 | 28472563256 | 4 |", + "| 4076864659 | 24997484146 | 28118515915 | 5 |", + "+------------+-------------+-------------+-----+", ]; assert_batches_eq!(expected, &actual); @@ -2085,15 +2084,15 @@ async fn test_window_agg_complex_plan() -> Result<()> { " GlobalLimitExec: skip=0, fetch=5", " WindowAggExec: wdw=[SUM(null_cases.c1): Ok(Field { name: \"SUM(null_cases.c1)\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(10)), end_bound: Following(Int64(11)) }, SUM(null_cases.c1): Ok(Field { name: \"SUM(null_cases.c1)\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow }, SUM(null_cases.c1): Ok(Field { name: \"SUM(null_cases.c1)\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: Following(Int64(11)) }, SUM(null_cases.c1): Ok(Field { name: \"SUM(null_cases.c1)\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(Int64(NULL)) }]", " BoundedWindowAggExec: wdw=[SUM(null_cases.c1): Ok(Field { name: \"SUM(null_cases.c1)\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow }]", - " SortExec: [c3@2 ASC NULLS LAST,c2@1 ASC NULLS LAST]", + " SortExec: [c3@2 ASC NULLS LAST,c2@1 ASC NULLS LAST], global=true", " BoundedWindowAggExec: wdw=[SUM(null_cases.c1): Ok(Field { name: \"SUM(null_cases.c1)\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow }]", - " SortExec: [c3@2 ASC NULLS LAST,c1@0 ASC]", + " SortExec: [c3@2 ASC NULLS LAST,c1@0 ASC], global=true", " WindowAggExec: wdw=[SUM(null_cases.c1): Ok(Field { name: \"SUM(null_cases.c1)\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(Int64(NULL)) }]", " WindowAggExec: wdw=[SUM(null_cases.c1): Ok(Field { name: \"SUM(null_cases.c1)\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(11)), end_bound: Following(Int64(10)) }, SUM(null_cases.c1): Ok(Field { name: \"SUM(null_cases.c1)\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(Int64(NULL)) }, SUM(null_cases.c1): Ok(Field { name: \"SUM(null_cases.c1)\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(11)), end_bound: Following(Int64(NULL)) }, SUM(null_cases.c1): Ok(Field { name: \"SUM(null_cases.c1)\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow }]", " WindowAggExec: wdw=[SUM(null_cases.c1): Ok(Field { name: \"SUM(null_cases.c1)\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(10)), end_bound: Following(Int64(11)) }, SUM(null_cases.c1): Ok(Field { name: \"SUM(null_cases.c1)\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow }, SUM(null_cases.c1): Ok(Field { name: \"SUM(null_cases.c1)\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: Following(Int64(11)) }, SUM(null_cases.c1): Ok(Field { name: \"SUM(null_cases.c1)\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(Int64(NULL)) }]", " WindowAggExec: wdw=[SUM(null_cases.c1): Ok(Field { name: \"SUM(null_cases.c1)\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(10)), end_bound: Following(Int64(11)) }, SUM(null_cases.c1): Ok(Field { name: \"SUM(null_cases.c1)\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow }, SUM(null_cases.c1): Ok(Field { name: \"SUM(null_cases.c1)\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: Following(Int64(11)) }, SUM(null_cases.c1): Ok(Field { name: \"SUM(null_cases.c1)\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(Int64(NULL)) }]", " BoundedWindowAggExec: wdw=[SUM(null_cases.c1): Ok(Field { name: \"SUM(null_cases.c1)\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow }]", - " SortExec: [c3@2 DESC,c1@0 ASC NULLS LAST]", + " SortExec: [c3@2 DESC,c1@0 ASC NULLS LAST], global=true", ] }; @@ -2134,7 +2133,7 @@ async fn test_window_agg_sort_orderby_reversed_partitionby_plan() -> Result<()> " GlobalLimitExec: skip=0, fetch=5", " BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9): Ok(Field { name: \"SUM(aggregate_test_100.c9)\", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)) }]", " BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9): Ok(Field { name: \"SUM(aggregate_test_100.c9)\", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)) }]", - " SortExec: [c1@0 ASC NULLS LAST,c9@1 DESC]", + " SortExec: [c1@0 ASC NULLS LAST,c9@1 DESC], global=true", ] }; @@ -2189,7 +2188,7 @@ async fn test_window_agg_sort_partitionby_reversed_plan() -> Result<()> { " GlobalLimitExec: skip=0, fetch=5", " BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9): Ok(Field { name: \"SUM(aggregate_test_100.c9)\", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: Following(UInt64(1)) }]", " BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9): Ok(Field { name: \"SUM(aggregate_test_100.c9)\", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)) }]", - " SortExec: [c1@0 ASC NULLS LAST,c9@1 DESC]", + " SortExec: [c1@0 ASC NULLS LAST,c9@1 DESC], global=true", ] }; @@ -2243,7 +2242,7 @@ async fn test_window_agg_sort_orderby_reversed_binary_expr() -> Result<()> { " GlobalLimitExec: skip=0, fetch=5", " WindowAggExec: wdw=[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: \"SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW\", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(Int16(NULL)) }]", " BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: \"SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c3 + aggregate_test_100.c4 DESC NULLS FIRST, aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW\", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int16(NULL)), end_bound: CurrentRow }]", - " SortExec: [CAST(c3@1 AS Int16) + c4@2 DESC,c9@3 DESC,c2@0 ASC NULLS LAST]", + " SortExec: [CAST(c3@1 AS Int16) + c4@2 DESC,c9@3 DESC,c2@0 ASC NULLS LAST], global=true", ] }; @@ -2356,7 +2355,7 @@ async fn test_window_agg_sort_orderby_reversed_partitionby_reversed_plan() -> Re " GlobalLimitExec: skip=0, fetch=5", " BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9): Ok(Field { name: \"SUM(aggregate_test_100.c9)\", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt32(NULL)), end_bound: CurrentRow }]", " BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9): Ok(Field { name: \"SUM(aggregate_test_100.c9)\", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int8(NULL)), end_bound: CurrentRow }]", - " SortExec: [c3@1 DESC,c9@2 DESC,c2@0 ASC NULLS LAST]", + " SortExec: [c3@1 DESC,c9@2 DESC,c2@0 ASC NULLS LAST], global=true", ] }; @@ -2524,7 +2523,7 @@ mod tests { vec![ "ProjectionExec: expr=[sum1@0 as sum1, sum2@1 as sum2, sum3@2 as sum3, min1@3 as min1, min2@4 as min2, min3@5 as min3, max1@6 as max1, max2@7 as max2, max3@8 as max3, cnt1@9 as cnt1, cnt2@10 as cnt2, sumr1@11 as sumr1, sumr2@12 as sumr2, sumr3@13 as sumr3, minr1@14 as minr1, minr2@15 as minr2, minr3@16 as minr3, maxr1@17 as maxr1, maxr2@18 as maxr2, maxr3@19 as maxr3, cntr1@20 as cntr1, cntr2@21 as cntr2, sum4@22 as sum4, cnt3@23 as cnt3]", " GlobalLimitExec: skip=0, fetch=5", - " SortExec: [inc_col@24 DESC]", + " SortExec: [inc_col@24 DESC], global=true", " ProjectionExec: expr=[SUM(annotated_data.inc_col) ORDER BY [annotated_data.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@14 as sum1, SUM(annotated_data.desc_col) ORDER BY [annotated_data.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@15 as sum2, SUM(annotated_data.inc_col) ORDER BY [annotated_data.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@16 as sum3, MIN(annotated_data.inc_col) ORDER BY [annotated_data.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@17 as min1, MIN(annotated_data.desc_col) ORDER BY [annotated_data.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@18 as min2, MIN(annotated_data.inc_col) ORDER BY [annotated_data.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@19 as min3, MAX(annotated_data.inc_col) ORDER BY [annotated_data.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@20 as max1, MAX(annotated_data.desc_col) ORDER BY [annotated_data.ts ASC NULLS LAST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@21 as max2, MAX(annotated_data.inc_col) ORDER BY [annotated_data.ts ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@22 as max3, COUNT(UInt8(1)) ORDER BY [annotated_data.ts ASC NULLS LAST] RANGE BETWEEN 4 PRECEDING AND 8 FOLLOWING@23 as cnt1, COUNT(UInt8(1)) ORDER BY [annotated_data.ts ASC NULLS LAST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING@24 as cnt2, SUM(annotated_data.inc_col) ORDER BY [annotated_data.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 4 FOLLOWING@3 as sumr1, SUM(annotated_data.desc_col) ORDER BY [annotated_data.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 8 FOLLOWING@4 as sumr2, SUM(annotated_data.desc_col) ORDER BY [annotated_data.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@5 as sumr3, MIN(annotated_data.inc_col) ORDER BY [annotated_data.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@6 as minr1, MIN(annotated_data.desc_col) ORDER BY [annotated_data.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@7 as minr2, MIN(annotated_data.inc_col) ORDER BY [annotated_data.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@8 as minr3, MAX(annotated_data.inc_col) ORDER BY [annotated_data.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@9 as maxr1, MAX(annotated_data.desc_col) ORDER BY [annotated_data.ts DESC NULLS FIRST] RANGE BETWEEN 5 PRECEDING AND 1 FOLLOWING@10 as maxr2, MAX(annotated_data.inc_col) ORDER BY [annotated_data.ts DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 10 FOLLOWING@11 as maxr3, COUNT(UInt8(1)) ORDER BY [annotated_data.ts DESC NULLS FIRST] RANGE BETWEEN 6 PRECEDING AND 2 FOLLOWING@12 as cntr1, COUNT(UInt8(1)) ORDER BY [annotated_data.ts DESC NULLS FIRST] ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING@13 as cntr2, SUM(annotated_data.desc_col) ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING@25 as sum4, COUNT(UInt8(1)) ROWS BETWEEN 8 PRECEDING AND 1 FOLLOWING@26 as cnt3, inc_col@1 as inc_col]", " BoundedWindowAggExec: wdw=[SUM(annotated_data.desc_col): Ok(Field { name: \"SUM(annotated_data.desc_col)\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(8)), end_bound: Following(UInt64(1)) }, COUNT(UInt8(1)): Ok(Field { name: \"COUNT(UInt8(1))\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(8)), end_bound: Following(UInt64(1)) }]", " BoundedWindowAggExec: wdw=[SUM(annotated_data.inc_col): Ok(Field { name: \"SUM(annotated_data.inc_col)\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)) }, SUM(annotated_data.desc_col): Ok(Field { name: \"SUM(annotated_data.desc_col)\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(5)), end_bound: Following(Int32(1)) }, SUM(annotated_data.inc_col): Ok(Field { name: \"SUM(annotated_data.inc_col)\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)) }, MIN(annotated_data.inc_col): Ok(Field { name: \"MIN(annotated_data.inc_col)\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)) }, MIN(annotated_data.desc_col): Ok(Field { name: \"MIN(annotated_data.desc_col)\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(5)), end_bound: Following(Int32(1)) }, MIN(annotated_data.inc_col): Ok(Field { name: \"MIN(annotated_data.inc_col)\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)) }, MAX(annotated_data.inc_col): Ok(Field { name: \"MAX(annotated_data.inc_col)\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)) }, MAX(annotated_data.desc_col): Ok(Field { name: \"MAX(annotated_data.desc_col)\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(5)), end_bound: Following(Int32(1)) }, MAX(annotated_data.inc_col): Ok(Field { name: \"MAX(annotated_data.inc_col)\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)) }, COUNT(UInt8(1)): Ok(Field { name: \"COUNT(UInt8(1))\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(4)), end_bound: Following(Int32(8)) }, COUNT(UInt8(1)): Ok(Field { name: \"COUNT(UInt8(1))\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(8)), end_bound: Following(UInt64(1)) }]", @@ -2599,7 +2598,7 @@ mod tests { vec![ "ProjectionExec: expr=[fv1@0 as fv1, fv2@1 as fv2, lv1@2 as lv1, lv2@3 as lv2, nv1@4 as nv1, nv2@5 as nv2, rn1@6 as rn1, rn2@7 as rn2, rank1@8 as rank1, rank2@9 as rank2, dense_rank1@10 as dense_rank1, dense_rank2@11 as dense_rank2, lag1@12 as lag1, lag2@13 as lag2, lead1@14 as lead1, lead2@15 as lead2, fvr1@16 as fvr1, fvr2@17 as fvr2, lvr1@18 as lvr1, lvr2@19 as lvr2, lagr1@20 as lagr1, lagr2@21 as lagr2, leadr1@22 as leadr1, leadr2@23 as leadr2]", " GlobalLimitExec: skip=0, fetch=5", - " SortExec: [ts@24 DESC]", + " SortExec: [ts@24 DESC], global=true", " ProjectionExec: expr=[FIRST_VALUE(annotated_data.inc_col) ORDER BY [annotated_data.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@10 as fv1, FIRST_VALUE(annotated_data.inc_col) ORDER BY [annotated_data.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@11 as fv2, LAST_VALUE(annotated_data.inc_col) ORDER BY [annotated_data.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@12 as lv1, LAST_VALUE(annotated_data.inc_col) ORDER BY [annotated_data.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@13 as lv2, NTH_VALUE(annotated_data.inc_col,Int64(5)) ORDER BY [annotated_data.ts ASC NULLS LAST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@14 as nv1, NTH_VALUE(annotated_data.inc_col,Int64(5)) ORDER BY [annotated_data.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@15 as nv2, ROW_NUMBER() ORDER BY [annotated_data.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@16 as rn1, ROW_NUMBER() ORDER BY [annotated_data.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@17 as rn2, RANK() ORDER BY [annotated_data.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@18 as rank1, RANK() ORDER BY [annotated_data.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@19 as rank2, DENSE_RANK() ORDER BY [annotated_data.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@20 as dense_rank1, DENSE_RANK() ORDER BY [annotated_data.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@21 as dense_rank2, LAG(annotated_data.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@22 as lag1, LAG(annotated_data.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@23 as lag2, LEAD(annotated_data.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data.ts ASC NULLS LAST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@24 as lead1, LEAD(annotated_data.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data.ts ASC NULLS LAST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@25 as lead2, FIRST_VALUE(annotated_data.inc_col) ORDER BY [annotated_data.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@2 as fvr1, FIRST_VALUE(annotated_data.inc_col) ORDER BY [annotated_data.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@3 as fvr2, LAST_VALUE(annotated_data.inc_col) ORDER BY [annotated_data.ts DESC NULLS FIRST] RANGE BETWEEN 10 PRECEDING AND 1 FOLLOWING@4 as lvr1, LAST_VALUE(annotated_data.inc_col) ORDER BY [annotated_data.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@5 as lvr2, LAG(annotated_data.inc_col,Int64(1),Int64(1001)) ORDER BY [annotated_data.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@6 as lagr1, LAG(annotated_data.inc_col,Int64(2),Int64(1002)) ORDER BY [annotated_data.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@7 as lagr2, LEAD(annotated_data.inc_col,Int64(-1),Int64(1001)) ORDER BY [annotated_data.ts DESC NULLS FIRST] RANGE BETWEEN 1 PRECEDING AND 10 FOLLOWING@8 as leadr1, LEAD(annotated_data.inc_col,Int64(4),Int64(1004)) ORDER BY [annotated_data.ts DESC NULLS FIRST] ROWS BETWEEN 10 PRECEDING AND 1 FOLLOWING@9 as leadr2, ts@0 as ts]", " BoundedWindowAggExec: wdw=[FIRST_VALUE(annotated_data.inc_col): Ok(Field { name: \"FIRST_VALUE(annotated_data.inc_col)\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)) }, FIRST_VALUE(annotated_data.inc_col): Ok(Field { name: \"FIRST_VALUE(annotated_data.inc_col)\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)) }, LAST_VALUE(annotated_data.inc_col): Ok(Field { name: \"LAST_VALUE(annotated_data.inc_col)\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)) }, LAST_VALUE(annotated_data.inc_col): Ok(Field { name: \"LAST_VALUE(annotated_data.inc_col)\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)) }, NTH_VALUE(annotated_data.inc_col,Int64(5)): Ok(Field { name: \"NTH_VALUE(annotated_data.inc_col,Int64(5))\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)) }, NTH_VALUE(annotated_data.inc_col,Int64(5)): Ok(Field { name: \"NTH_VALUE(annotated_data.inc_col,Int64(5))\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)) }, ROW_NUMBER(): Ok(Field { name: \"ROW_NUMBER()\", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)) }, ROW_NUMBER(): Ok(Field { name: \"ROW_NUMBER()\", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)) }, RANK(): Ok(Field { name: \"RANK()\", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)) }, RANK(): Ok(Field { name: \"RANK()\", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)) }, DENSE_RANK(): Ok(Field { name: \"DENSE_RANK()\", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)) }, DENSE_RANK(): Ok(Field { name: \"DENSE_RANK()\", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)) }, LAG(annotated_data.inc_col,Int64(1),Int64(1001)): Ok(Field { name: \"LAG(annotated_data.inc_col,Int64(1),Int64(1001))\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)) }, LAG(annotated_data.inc_col,Int64(2),Int64(1002)): Ok(Field { name: \"LAG(annotated_data.inc_col,Int64(2),Int64(1002))\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)) }, LEAD(annotated_data.inc_col,Int64(-1),Int64(1001)): Ok(Field { name: \"LEAD(annotated_data.inc_col,Int64(-1),Int64(1001))\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)) }, LEAD(annotated_data.inc_col,Int64(4),Int64(1004)): Ok(Field { name: \"LEAD(annotated_data.inc_col,Int64(4),Int64(1004))\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(10)), end_bound: Following(UInt64(1)) }]", " BoundedWindowAggExec: wdw=[FIRST_VALUE(annotated_data.inc_col): Ok(Field { name: \"FIRST_VALUE(annotated_data.inc_col)\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)) }, FIRST_VALUE(annotated_data.inc_col): Ok(Field { name: \"FIRST_VALUE(annotated_data.inc_col)\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)) }, LAST_VALUE(annotated_data.inc_col): Ok(Field { name: \"LAST_VALUE(annotated_data.inc_col)\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(1)), end_bound: Following(Int32(10)) }, LAST_VALUE(annotated_data.inc_col): Ok(Field { name: \"LAST_VALUE(annotated_data.inc_col)\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)) }, LAG(annotated_data.inc_col,Int64(1),Int64(1001)): Ok(Field { name: \"LAG(annotated_data.inc_col,Int64(1),Int64(1001))\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)) }, LAG(annotated_data.inc_col,Int64(2),Int64(1002)): Ok(Field { name: \"LAG(annotated_data.inc_col,Int64(2),Int64(1002))\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)) }, LEAD(annotated_data.inc_col,Int64(-1),Int64(1001)): Ok(Field { name: \"LEAD(annotated_data.inc_col,Int64(-1),Int64(1001))\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(10)), end_bound: Following(Int32(1)) }, LEAD(annotated_data.inc_col,Int64(4),Int64(1004)): Ok(Field { name: \"LEAD(annotated_data.inc_col,Int64(4),Int64(1004))\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(10)) }]", @@ -2658,7 +2657,7 @@ mod tests { vec![ "ProjectionExec: expr=[sum1@0 as sum1, sum2@1 as sum2, min1@2 as min1, min2@3 as min2, max1@4 as max1, max2@5 as max2, count1@6 as count1, count2@7 as count2, avg1@8 as avg1, avg2@9 as avg2]", " GlobalLimitExec: skip=0, fetch=5", - " SortExec: [inc_col@10 ASC NULLS LAST]", + " SortExec: [inc_col@10 ASC NULLS LAST], global=true", " ProjectionExec: expr=[SUM(annotated_data.inc_col) ORDER BY [annotated_data.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING@7 as sum1, SUM(annotated_data.inc_col) ORDER BY [annotated_data.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@2 as sum2, MIN(annotated_data.inc_col) ORDER BY [annotated_data.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING@8 as min1, MIN(annotated_data.inc_col) ORDER BY [annotated_data.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@3 as min2, MAX(annotated_data.inc_col) ORDER BY [annotated_data.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING@9 as max1, MAX(annotated_data.inc_col) ORDER BY [annotated_data.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@4 as max2, COUNT(annotated_data.inc_col) ORDER BY [annotated_data.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING@10 as count1, COUNT(annotated_data.inc_col) ORDER BY [annotated_data.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@5 as count2, AVG(annotated_data.inc_col) ORDER BY [annotated_data.ts ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND 5 FOLLOWING@11 as avg1, AVG(annotated_data.inc_col) ORDER BY [annotated_data.ts DESC NULLS FIRST] RANGE BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@6 as avg2, inc_col@1 as inc_col]", " BoundedWindowAggExec: wdw=[SUM(annotated_data.inc_col): Ok(Field { name: \"SUM(annotated_data.inc_col)\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(5)) }, MIN(annotated_data.inc_col): Ok(Field { name: \"MIN(annotated_data.inc_col)\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(5)) }, MAX(annotated_data.inc_col): Ok(Field { name: \"MAX(annotated_data.inc_col)\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(5)) }, COUNT(annotated_data.inc_col): Ok(Field { name: \"COUNT(annotated_data.inc_col)\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(5)) }, AVG(annotated_data.inc_col): Ok(Field { name: \"AVG(annotated_data.inc_col)\", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(5)) }]", " BoundedWindowAggExec: wdw=[SUM(annotated_data.inc_col): Ok(Field { name: \"SUM(annotated_data.inc_col)\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(3)) }, MIN(annotated_data.inc_col): Ok(Field { name: \"MIN(annotated_data.inc_col)\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(3)) }, MAX(annotated_data.inc_col): Ok(Field { name: \"MAX(annotated_data.inc_col)\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(3)) }, COUNT(annotated_data.inc_col): Ok(Field { name: \"COUNT(annotated_data.inc_col)\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(3)) }, AVG(annotated_data.inc_col): Ok(Field { name: \"AVG(annotated_data.inc_col)\", data_type: Float64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: Following(Int32(3)) }]", @@ -2712,7 +2711,7 @@ mod tests { vec![ "ProjectionExec: expr=[first_value1@0 as first_value1, first_value2@1 as first_value2, last_value1@2 as last_value1, last_value2@3 as last_value2, nth_value1@4 as nth_value1]", " GlobalLimitExec: skip=0, fetch=5", - " SortExec: [inc_col@5 ASC NULLS LAST]", + " SortExec: [inc_col@5 ASC NULLS LAST], global=true", " ProjectionExec: expr=[FIRST_VALUE(annotated_data.inc_col) ORDER BY [annotated_data.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@4 as first_value1, FIRST_VALUE(annotated_data.inc_col) ORDER BY [annotated_data.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@2 as first_value2, LAST_VALUE(annotated_data.inc_col) ORDER BY [annotated_data.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@5 as last_value1, LAST_VALUE(annotated_data.inc_col) ORDER BY [annotated_data.ts DESC NULLS FIRST] ROWS BETWEEN 3 PRECEDING AND UNBOUNDED FOLLOWING@3 as last_value2, NTH_VALUE(annotated_data.inc_col,Int64(2)) ORDER BY [annotated_data.ts ASC NULLS LAST] ROWS BETWEEN UNBOUNDED PRECEDING AND 1 FOLLOWING@6 as nth_value1, inc_col@1 as inc_col]", " BoundedWindowAggExec: wdw=[FIRST_VALUE(annotated_data.inc_col): Ok(Field { name: \"FIRST_VALUE(annotated_data.inc_col)\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(1)) }, LAST_VALUE(annotated_data.inc_col): Ok(Field { name: \"LAST_VALUE(annotated_data.inc_col)\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(1)) }, NTH_VALUE(annotated_data.inc_col,Int64(2)): Ok(Field { name: \"NTH_VALUE(annotated_data.inc_col,Int64(2))\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(1)) }]", " BoundedWindowAggExec: wdw=[FIRST_VALUE(annotated_data.inc_col): Ok(Field { name: \"FIRST_VALUE(annotated_data.inc_col)\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(3)) }, LAST_VALUE(annotated_data.inc_col): Ok(Field { name: \"LAST_VALUE(annotated_data.inc_col)\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(3)) }]", diff --git a/datafusion/physical-expr/src/lib.rs b/datafusion/physical-expr/src/lib.rs index c9658a048ca84..9022f39d23c69 100644 --- a/datafusion/physical-expr/src/lib.rs +++ b/datafusion/physical-expr/src/lib.rs @@ -53,8 +53,9 @@ pub use physical_expr::{AnalysisContext, ExprBoundaries, PhysicalExpr}; pub use planner::create_physical_expr; pub use scalar_function::ScalarFunctionExpr; pub use sort_expr::PhysicalSortExpr; +pub use sort_expr::PhysicalSortRequirements; pub use utils::{ - expr_list_eq_any_order, expr_list_eq_strict_order, + expr_list_eq_any_order, expr_list_eq_strict_order, new_sort_requirements, normalize_expr_with_equivalence_properties, normalize_out_expr_with_alias_schema, normalize_sort_expr_with_equivalence_properties, sort_expr_list_eq_strict_order, split_conjunction, diff --git a/datafusion/physical-expr/src/sort_expr.rs b/datafusion/physical-expr/src/sort_expr.rs index f8172dabf65aa..a25802e9980ab 100644 --- a/datafusion/physical-expr/src/sort_expr.rs +++ b/datafusion/physical-expr/src/sort_expr.rs @@ -69,4 +69,56 @@ impl PhysicalSortExpr { options: Some(self.options), }) } + + pub fn satisfy(&self, requirement: &PhysicalSortRequirements) -> bool { + if requirement.sort_options.is_some() { + self.options == requirement.sort_options.unwrap() + && self.expr.eq(&requirement.expr) + } else { + self.expr.eq(&requirement.expr) + } + } +} + +/// Represents sort requirement associated with a plan +#[derive(Clone, Debug)] +pub struct PhysicalSortRequirements { + /// Physical expression representing the column to sort + pub expr: Arc, + /// Option to specify how the given column should be sorted. + /// If not specified, the PhysicalSortRequirements does not have specific requirements on the sort options. + pub sort_options: Option, +} + +impl PartialEq for PhysicalSortRequirements { + fn eq(&self, other: &PhysicalSortRequirements) -> bool { + self.sort_options == other.sort_options && self.expr.eq(&other.expr) + } +} + +impl std::fmt::Display for PhysicalSortRequirements { + fn fmt(&self, f: &mut std::fmt::Formatter) -> std::fmt::Result { + let opts_string = if let Some(sort_options) = self.sort_options { + match (sort_options.descending, sort_options.nulls_first) { + (true, true) => "DESC", + (true, false) => "DESC NULLS LAST", + (false, true) => "ASC", + (false, false) => "ASC NULLS LAST", + } + } else { + "NA" + }; + write!(f, "{} {}", self.expr, opts_string) + } +} + +impl PhysicalSortRequirements { + /// Requirement is compatible with the other means the current requirement is equal or more specific than the other + pub fn compatible(&self, other: &PhysicalSortRequirements) -> bool { + if other.sort_options.is_some() { + self.eq(other) + } else { + self.expr.eq(&other.expr) + } + } } diff --git a/datafusion/physical-expr/src/utils.rs b/datafusion/physical-expr/src/utils.rs index d6d5054ffef03..533a606b69e85 100644 --- a/datafusion/physical-expr/src/utils.rs +++ b/datafusion/physical-expr/src/utils.rs @@ -21,11 +21,12 @@ use crate::expressions::Column; use crate::expressions::UnKnownColumn; use crate::rewrite::TreeNodeRewritable; use crate::PhysicalSortExpr; -use crate::{EquivalenceProperties, PhysicalExpr}; +use crate::{EquivalenceProperties, PhysicalExpr, PhysicalSortRequirements}; use datafusion_expr::Operator; use arrow::datatypes::SchemaRef; +use arrow_schema::SortOptions; use std::collections::HashMap; use std::sync::Arc; @@ -168,6 +169,21 @@ pub fn normalize_expr_with_equivalence_properties( .unwrap_or(expr) } +pub fn new_sort_requirements( + sort_keys: Option<&[PhysicalSortExpr]>, +) -> Option> { + let ordering_requirements = sort_keys.map(|ordering| { + ordering + .iter() + .map(|o| PhysicalSortRequirements { + expr: o.expr.clone(), + sort_options: Some(o.options.clone()), + }) + .collect::>() + }); + ordering_requirements +} + pub fn normalize_sort_expr_with_equivalence_properties( sort_expr: PhysicalSortExpr, eq_properties: &[EquivalentClass], @@ -185,6 +201,24 @@ pub fn normalize_sort_expr_with_equivalence_properties( } } +pub fn normalize_sort_requirement_with_equivalence_properties( + sort_requirement: PhysicalSortRequirements, + eq_properties: &[EquivalentClass], +) -> PhysicalSortRequirements { + let normalized_expr = normalize_expr_with_equivalence_properties( + sort_requirement.expr.clone(), + eq_properties, + ); + if sort_requirement.expr.ne(&normalized_expr) { + PhysicalSortRequirements { + expr: normalized_expr, + sort_options: sort_requirement.sort_options.clone(), + } + } else { + sort_requirement + } +} + /// Checks whether given ordering requirements are satisfied by provided [PhysicalSortExpr]s. pub fn ordering_satisfy EquivalenceProperties>( provided: Option<&[PhysicalSortExpr]>, @@ -200,7 +234,7 @@ pub fn ordering_satisfy EquivalenceProperties>( } } -pub fn ordering_satisfy_concrete EquivalenceProperties>( +fn ordering_satisfy_concrete EquivalenceProperties>( provided: &[PhysicalSortExpr], required: &[PhysicalSortExpr], equal_properties: F, @@ -235,6 +269,185 @@ pub fn ordering_satisfy_concrete EquivalenceProperties>( } } +pub fn ordering_satisfy_requirement EquivalenceProperties>( + provided: Option<&[PhysicalSortExpr]>, + required: Option<&[PhysicalSortRequirements]>, + equal_properties: F, +) -> bool { + match (provided, required) { + (_, None) => true, + (None, Some(_)) => false, + (Some(provided), Some(required)) => { + ordering_satisfy_requirement_concrete(provided, required, equal_properties) + } + } +} + +pub fn ordering_satisfy_requirement_concrete EquivalenceProperties>( + provided: &[PhysicalSortExpr], + required: &[PhysicalSortRequirements], + equal_properties: F, +) -> bool { + if required.len() > provided.len() { + false + } else if required + .iter() + .zip(provided.iter()) + .all(|(order1, order2)| order2.satisfy(order1)) + { + true + } else if let eq_classes @ [_, ..] = equal_properties().classes() { + let normalized_requirements = required + .iter() + .map(|e| { + normalize_sort_requirement_with_equivalence_properties( + e.clone(), + eq_classes, + ) + }) + .collect::>(); + let normalized_provided_exprs = provided + .iter() + .map(|e| { + normalize_sort_expr_with_equivalence_properties(e.clone(), eq_classes) + }) + .collect::>(); + normalized_requirements + .iter() + .zip(normalized_provided_exprs.iter()) + .all(|(order1, order2)| order2.satisfy(order1)) + } else { + false + } +} + +/// Provided requirements are compatible with the required, which means the provided requirements are equal or more specific than the required +pub fn requirements_compatible EquivalenceProperties>( + provided: Option<&[PhysicalSortRequirements]>, + required: Option<&[PhysicalSortRequirements]>, + equal_properties: F, +) -> bool { + match (provided, required) { + (_, None) => true, + (None, Some(_)) => false, + (Some(provided), Some(required)) => { + if required.len() > provided.len() { + false + } else if required + .iter() + .zip(provided.iter()) + .all(|(req, pro)| pro.compatible(req)) + { + true + } else if let eq_classes @ [_, ..] = equal_properties().classes() { + let normalized_required = required + .iter() + .map(|e| { + normalize_sort_requirement_with_equivalence_properties( + e.clone(), + eq_classes, + ) + }) + .collect::>(); + let normalized_provided = provided + .iter() + .map(|e| { + normalize_sort_requirement_with_equivalence_properties( + e.clone(), + eq_classes, + ) + }) + .collect::>(); + normalized_required + .iter() + .zip(normalized_provided.iter()) + .all(|(req, pro)| pro.compatible(req)) + } else { + false + } + } + } +} + +pub fn map_columns_before_projection( + parent_required: &[Arc], + proj_exprs: &[(Arc, String)], +) -> Vec> { + let mut column_mapping = HashMap::new(); + for (expression, name) in proj_exprs.iter() { + if let Some(column) = expression.as_any().downcast_ref::() { + column_mapping.insert(name.clone(), column.clone()); + }; + } + let new_required: Vec> = parent_required + .iter() + .filter_map(|r| { + if let Some(column) = r.as_any().downcast_ref::() { + column_mapping.get(column.name()) + } else { + None + } + }) + .map(|e| Arc::new(e.clone()) as Arc) + .collect::>(); + new_required +} + +pub fn map_requirement_before_projection( + parent_required: Option<&[PhysicalSortRequirements]>, + proj_exprs: &[(Arc, String)], +) -> Option> { + if let Some(requirement) = parent_required { + let required_expr = create_sort_expr_from_requirement(requirement) + .iter() + .map(|sort_expr| sort_expr.expr.clone()) + .collect::>(); + let new_exprs = map_columns_before_projection(&required_expr, proj_exprs); + if new_exprs.len() == requirement.len() { + let new_request = new_exprs + .iter() + .zip(requirement.iter()) + .map(|(new, old)| PhysicalSortRequirements { + expr: new.clone(), + sort_options: old.sort_options.clone(), + }) + .collect::>(); + Some(new_request) + } else { + None + } + } else { + None + } +} + +pub fn create_sort_expr_from_requirement( + required: &[PhysicalSortRequirements], +) -> Vec { + let parent_required_expr = required + .iter() + .map(|prop| { + if prop.sort_options.is_some() { + PhysicalSortExpr { + expr: prop.expr.clone(), + options: prop.sort_options.unwrap().clone(), + } + } else { + PhysicalSortExpr { + expr: prop.expr.clone(), + options: SortOptions { + // By default, create sort key with ASC is true and NULLS LAST to be consistent with + // PostgreSQL's rule: https://www.postgresql.org/docs/current/queries-order.html + descending: false, + nulls_first: false, + }, + } + } + }) + .collect::>(); + parent_required_expr +} + #[cfg(test)] mod tests { From 01e87d69d1bf18f2e760872d1f664e8dc515d940 Mon Sep 17 00:00:00 2001 From: "mingmwang@ebay.com" Date: Fri, 10 Feb 2023 20:06:56 +0800 Subject: [PATCH 02/13] Add support to optimize parallel sorting --- .../physical_optimizer/dist_enforcement.rs | 77 +++++++---- .../global_sort_selection.rs | 4 +- .../physical_optimizer/sort_enforcement.rs | 6 +- .../physical_optimizer/sort_enforcement2.rs | 128 +++++++++++------- .../sorts/sort_preserving_merge.rs | 18 +++ datafusion/core/tests/sql/joins.rs | 16 +-- datafusion/core/tests/sql/window.rs | 31 +++-- datafusion/physical-expr/src/utils.rs | 3 +- 8 files changed, 181 insertions(+), 102 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/dist_enforcement.rs b/datafusion/core/src/physical_optimizer/dist_enforcement.rs index 4fe76ac30d94c..3bc599c0cb0d2 100644 --- a/datafusion/core/src/physical_optimizer/dist_enforcement.rs +++ b/datafusion/core/src/physical_optimizer/dist_enforcement.rs @@ -30,6 +30,7 @@ use crate::physical_plan::projection::ProjectionExec; use crate::physical_plan::repartition::RepartitionExec; use crate::physical_plan::rewrite::TreeNodeRewritable; use crate::physical_plan::sorts::sort::SortOptions; +use crate::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; use crate::physical_plan::windows::WindowAggExec; use crate::physical_plan::Partitioning; use crate::physical_plan::{with_new_children_if_necessary, Distribution, ExecutionPlan}; @@ -38,11 +39,14 @@ use datafusion_expr::logical_plan::JoinType; use datafusion_physical_expr::equivalence::EquivalenceProperties; use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::expressions::NoOp; -use datafusion_physical_expr::utils::map_columns_before_projection; +use datafusion_physical_expr::utils::{ + create_sort_expr_from_requirement, map_columns_before_projection, +}; use datafusion_physical_expr::{ expr_list_eq_strict_order, normalize_expr_with_equivalence_properties, AggregateExpr, PhysicalExpr, }; +use itertools::izip; use std::sync::Arc; /// The EnforceDistribution rule ensures that distribution requirements are met @@ -80,7 +84,9 @@ impl PhysicalOptimizerRule for EnforceDistribution { } else { plan }; + // Distribution enforcement needs to be applied bottom-up. + let repartition_sorts = config.optimizer.repartition_sorts; new_plan.transform_up(&{ |plan| { let adjusted = if !top_down_join_key_reordering { @@ -88,7 +94,11 @@ impl PhysicalOptimizerRule for EnforceDistribution { } else { plan }; - Ok(Some(ensure_distribution(adjusted, target_partitions)?)) + Ok(Some(ensure_distribution( + adjusted, + target_partitions, + repartition_sorts, + )?)) } }) } @@ -819,6 +829,7 @@ fn new_join_conditions( fn ensure_distribution( plan: Arc, target_partitions: usize, + repartition_sort: bool, ) -> Result> { if plan.children().is_empty() { return Ok(plan); @@ -829,31 +840,43 @@ fn ensure_distribution( assert_eq!(children.len(), required_input_distributions.len()); // Add RepartitionExec to guarantee output partitioning - let new_children: Result>> = children - .into_iter() - .zip(required_input_distributions.into_iter()) - .map(|(child, required)| { - if child - .output_partitioning() - .satisfy(required.clone(), || child.equivalence_properties()) - { - Ok(child) - } else { - let new_child: Result> = match required { - Distribution::SinglePartition - if child.output_partitioning().partition_count() > 1 => - { + let new_children: Result>> = izip!( + children.into_iter(), + required_input_distributions.into_iter(), + plan.required_input_ordering().into_iter(), + ) + .map(|(child, required, required_ordering)| { + if child + .output_partitioning() + .satisfy(required.clone(), || child.equivalence_properties()) + { + Ok(child) + } else { + let new_child: Result> = match required { + Distribution::SinglePartition + if child.output_partitioning().partition_count() > 1 => + { + if repartition_sort && required_ordering.is_some() { + let new_physical_ordering = create_sort_expr_from_requirement( + required_ordering.unwrap().as_ref(), + ); + Ok(Arc::new(SortPreservingMergeExec::new_for_distribuion( + new_physical_ordering, + child.clone(), + ))) + } else { Ok(Arc::new(CoalescePartitionsExec::new(child.clone()))) } - _ => { - let partition = required.create_partitioning(target_partitions); - Ok(Arc::new(RepartitionExec::try_new(child, partition)?)) - } - }; - new_child - } - }) - .collect(); + } + _ => { + let partition = required.create_partitioning(target_partitions); + Ok(Arc::new(RepartitionExec::try_new(child, partition)?)) + } + }; + new_child + } + }) + .collect(); with_new_children_if_necessary(plan, new_children?) } @@ -1654,6 +1677,7 @@ mod tests { let bottom_left_join = ensure_distribution( hash_join_exec(left.clone(), right.clone(), &join_on, &JoinType::Inner), 10, + false, )?; // Projection(a as A, a as AA, b as B, c as C) @@ -1684,6 +1708,7 @@ mod tests { let bottom_right_join = ensure_distribution( hash_join_exec(left, right.clone(), &join_on, &JoinType::Inner), 10, + false, )?; // Join on (B == b1 and C == c and AA = a1) @@ -1773,6 +1798,7 @@ mod tests { let bottom_left_join = ensure_distribution( hash_join_exec(left.clone(), right.clone(), &join_on, &JoinType::Inner), 10, + false, )?; // Projection(a as A, a as AA, b as B, c as C) @@ -1803,6 +1829,7 @@ mod tests { let bottom_right_join = ensure_distribution( hash_join_exec(left, right.clone(), &join_on, &JoinType::Inner), 10, + false, )?; // Join on (B == b1 and C == c and AA = a1) diff --git a/datafusion/core/src/physical_optimizer/global_sort_selection.rs b/datafusion/core/src/physical_optimizer/global_sort_selection.rs index 81b4b59e3a142..ec64756b8b74b 100644 --- a/datafusion/core/src/physical_optimizer/global_sort_selection.rs +++ b/datafusion/core/src/physical_optimizer/global_sort_selection.rs @@ -48,7 +48,7 @@ impl PhysicalOptimizerRule for GlobalSortSelection { fn optimize( &self, plan: Arc, - _config: &ConfigOptions, + config: &ConfigOptions, ) -> Result> { plan.transform_up(&|plan| { Ok(plan @@ -56,9 +56,9 @@ impl PhysicalOptimizerRule for GlobalSortSelection { .downcast_ref::() .and_then(|sort_exec| { if sort_exec.input().output_partitioning().partition_count() > 1 - && sort_exec.fetch().is_some() // It's already preserving the partitioning so that it can be regarded as a local sort && !sort_exec.preserve_partitioning() + && (sort_exec.fetch().is_some() || config.optimizer.repartition_sorts) { let sort = SortExec::new_with_partitioning( sort_exec.expr().to_vec(), diff --git a/datafusion/core/src/physical_optimizer/sort_enforcement.rs b/datafusion/core/src/physical_optimizer/sort_enforcement.rs index 217744d486321..14579786c2ddb 100644 --- a/datafusion/core/src/physical_optimizer/sort_enforcement.rs +++ b/datafusion/core/src/physical_optimizer/sort_enforcement.rs @@ -693,7 +693,11 @@ fn change_corresponding_coalesce_in_sub_plan( Some(sort_expr), || coalesce_input.equivalence_properties(), ) { - return add_sort_above_child(&coalesce_input, sort_expr.to_vec(), None); + return add_sort_above_child( + &coalesce_input, + sort_expr.to_vec(), + None, + ); } } coalesce_input diff --git a/datafusion/core/src/physical_optimizer/sort_enforcement2.rs b/datafusion/core/src/physical_optimizer/sort_enforcement2.rs index 984d817db9a63..2ef3c6111b428 100644 --- a/datafusion/core/src/physical_optimizer/sort_enforcement2.rs +++ b/datafusion/core/src/physical_optimizer/sort_enforcement2.rs @@ -132,6 +132,8 @@ impl PlanWithSortRequirements { true } else if self.plan.as_any().downcast_ref::().is_some() { false + } else if self.plan.as_any().downcast_ref::().is_some() { + self.plan.output_ordering().is_some() && self.impact_result_ordering } else { self.plan.maintains_input_order().iter().all(|o| *o) && self.impact_result_ordering @@ -195,8 +197,24 @@ impl PhysicalOptimizerRule for TopDownEnforceSorting { // Execute a Top-Down process(Preorder Traversal) to ensure the sort requirements: let plan_requirements = PlanWithSortRequirements::init(plan); let adjusted = plan_requirements.transform_down(&ensure_sorting)?; + // Execute a Top-Down process(Preorder Traversal) to remove all the unnecessary Sort + let adjusted_plan = adjusted.plan.transform_down(&|plan| { + if let Some(sort_exec) = plan.as_any().downcast_ref::() { + if ordering_satisfy( + sort_exec.input().output_ordering(), + sort_exec.output_ordering(), + || sort_exec.input().equivalence_properties(), + ) { + Ok(Some(Arc::new(TombStoneExec::new(sort_exec.input().clone())))) + } else { + Ok(None) + } + } else { + Ok(None) + } + })?; // Remove the TombStoneExec - let final_plan = adjusted.plan.transform_up(&|plan| { + let final_plan = adjusted_plan.transform_up(&|plan| { if let Some(tombstone_exec) = plan.as_any().downcast_ref::() { Ok(Some(tombstone_exec.input.clone())) } else { @@ -243,7 +261,7 @@ fn ensure_sorting( sort_exec.input().output_ordering(), sort_exec.output_ordering(), || sort_exec.input().equivalence_properties(), - ) && sort_exec.input().output_partitioning().partition_count() == 1 + ) { println!("remove sort_exec due to child already satisfy"); return Ok(Some(PlanWithSortRequirements { @@ -259,38 +277,78 @@ fn ensure_sorting( .as_any() .downcast_ref::() { - // SortPreservingMergeExec + SortExec(local/global) is the same as the global SortExec - // Remove unnecessary SortPreservingMergeExec + SortExec(local/global) - if let Some(child_sort_exec) = - sort_pres_exec.input().as_any().downcast_ref::() - { - if sort_pres_exec.expr() == child_sort_exec.expr() { - if !requirements.impact_result_ordering - && requirements.required_ordering.is_none() - { - println!("remove SortPreservingMergeExec + SortExec due to no need to keep ordering"); + if !sort_pres_exec.satisfy_distribution() { + // SortPreservingMergeExec + SortExec(local/global) is the same as the global SortExec + // Remove unnecessary SortPreservingMergeExec + SortExec(local/global) + if let Some(child_sort_exec) = + sort_pres_exec.input().as_any().downcast_ref::() + { + if sort_pres_exec.expr() == child_sort_exec.expr() { + if !requirements.impact_result_ordering + && requirements.required_ordering.is_none() + { + println!("remove SortPreservingMergeExec + SortExec due to no need to keep ordering"); + return Ok(Some(PlanWithSortRequirements { + plan: Arc::new(TombStoneExec::new( + child_sort_exec.input().clone(), + )), + impact_result_ordering: false, + required_ordering: None, + adjusted_request_ordering: vec![None], + })); + } else if ordering_satisfy( + child_sort_exec.input().output_ordering(), + child_sort_exec.output_ordering(), + || child_sort_exec.input().equivalence_properties(), + ) && child_sort_exec + .input() + .output_partitioning() + .partition_count() + == 1 + { + println!("remove SortPreservingMergeExec + SortExec due to child already satisfy"); + return Ok(Some(PlanWithSortRequirements { + plan: Arc::new(TombStoneExec::new( + child_sort_exec.input().clone(), + )), + impact_result_ordering: true, + required_ordering: None, + adjusted_request_ordering: vec![ + requirements.required_ordering, + ], + })); + } + } + } else { + // Remove unnecessary SortPreservingMergeExec only + if !requirements.impact_result_ordering { + println!( + "remove SortPreservingMergeExec due to no need to keep ordering" + ); return Ok(Some(PlanWithSortRequirements { plan: Arc::new(TombStoneExec::new( - child_sort_exec.input().clone(), + sort_pres_exec.input().clone(), )), impact_result_ordering: false, required_ordering: None, - adjusted_request_ordering: vec![None], + adjusted_request_ordering: vec![requirements.required_ordering], })); } else if ordering_satisfy( - child_sort_exec.input().output_ordering(), - child_sort_exec.output_ordering(), - || child_sort_exec.input().equivalence_properties(), - ) && child_sort_exec + sort_pres_exec.input().output_ordering(), + Some(sort_pres_exec.expr()), + || sort_pres_exec.input().equivalence_properties(), + ) && sort_pres_exec .input() .output_partitioning() .partition_count() == 1 { - println!("remove SortPreservingMergeExec + SortExec due to child already satisfy"); + println!( + "remove SortPreservingMergeExec due to child already satisfy" + ); return Ok(Some(PlanWithSortRequirements { plan: Arc::new(TombStoneExec::new( - child_sort_exec.input().clone(), + sort_pres_exec.input().clone(), )), impact_result_ordering: true, required_ordering: None, @@ -298,36 +356,6 @@ fn ensure_sorting( })); } } - } else { - // Remove unnecessary SortPreservingMergeExec only - if !requirements.impact_result_ordering { - println!( - "remove SortPreservingMergeExec due to no need to keep ordering" - ); - return Ok(Some(PlanWithSortRequirements { - plan: Arc::new(TombStoneExec::new(sort_pres_exec.input().clone())), - impact_result_ordering: false, - required_ordering: None, - adjusted_request_ordering: vec![requirements.required_ordering], - })); - } else if ordering_satisfy( - sort_pres_exec.input().output_ordering(), - Some(sort_pres_exec.expr()), - || sort_pres_exec.input().equivalence_properties(), - ) && sort_pres_exec - .input() - .output_partitioning() - .partition_count() - == 1 - { - println!("remove SortPreservingMergeExec due to child already satisfy"); - return Ok(Some(PlanWithSortRequirements { - plan: Arc::new(TombStoneExec::new(sort_pres_exec.input().clone())), - impact_result_ordering: true, - required_ordering: None, - adjusted_request_ordering: vec![requirements.required_ordering], - })); - } } } println!("no removing"); diff --git a/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs b/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs index 8a73061e486f0..4a0bfce0874a2 100644 --- a/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs +++ b/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs @@ -85,6 +85,8 @@ pub struct SortPreservingMergeExec { expr: Vec, /// Execution metrics metrics: ExecutionPlanMetricsSet, + /// use SortPreservingMergeExec to satisfy the Sort Distribution + satisfy_distribution: bool } impl SortPreservingMergeExec { @@ -94,6 +96,17 @@ impl SortPreservingMergeExec { input, expr, metrics: ExecutionPlanMetricsSet::new(), + satisfy_distribution: false + } + } + + /// Create a new SortPreservingMergeExec to satisfy the Sort Distribution + pub fn new_for_distribuion(expr: Vec, input: Arc) -> Self { + Self { + input, + expr, + metrics: ExecutionPlanMetricsSet::new(), + satisfy_distribution: true } } @@ -106,6 +119,11 @@ impl SortPreservingMergeExec { pub fn expr(&self) -> &[PhysicalSortExpr] { &self.expr } + + /// satisfy the Sort Distribution requirements + pub fn satisfy_distribution(&self) -> bool{ + self.satisfy_distribution + } } impl ExecutionPlan for SortPreservingMergeExec { diff --git a/datafusion/core/tests/sql/joins.rs b/datafusion/core/tests/sql/joins.rs index c360b2f240d0a..b92242bed52b0 100644 --- a/datafusion/core/tests/sql/joins.rs +++ b/datafusion/core/tests/sql/joins.rs @@ -1981,7 +1981,7 @@ async fn left_semi_join() -> Result<()> { let expected = if repartition_joins { vec![ "SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST]", - " SortExec: [t1_id@0 ASC NULLS LAST]", + " SortExec: [t1_id@0 ASC NULLS LAST], global=false", " ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name]", " CoalesceBatchesExec: target_batch_size=4096", " HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(Column { name: \"t1_id\", index: 0 }, Column { name: \"t2_id\", index: 0 })]", @@ -1998,7 +1998,7 @@ async fn left_semi_join() -> Result<()> { } else { vec![ "SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST]", - " SortExec: [t1_id@0 ASC NULLS LAST]", + " SortExec: [t1_id@0 ASC NULLS LAST], global=false", " ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name]", " CoalesceBatchesExec: target_batch_size=4096", " HashJoinExec: mode=CollectLeft, join_type=LeftSemi, on=[(Column { name: \"t1_id\", index: 0 }, Column { name: \"t2_id\", index: 0 })]", @@ -2063,7 +2063,7 @@ async fn left_semi_join() -> Result<()> { let expected = if repartition_joins { vec![ "SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST]", - " SortExec: [t1_id@0 ASC NULLS LAST]", + " SortExec: [t1_id@0 ASC NULLS LAST], global=false", " ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name]", " CoalesceBatchesExec: target_batch_size=4096", " HashJoinExec: mode=Partitioned, join_type=LeftSemi, on=[(Column { name: \"t1_id\", index: 0 }, Column { name: \"t2_id\", index: 0 })]", @@ -2079,7 +2079,7 @@ async fn left_semi_join() -> Result<()> { } else { vec![ "SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST]", - " SortExec: [t1_id@0 ASC NULLS LAST]", + " SortExec: [t1_id@0 ASC NULLS LAST], global=false", " ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name]", " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", " CoalesceBatchesExec: target_batch_size=4096", @@ -2260,7 +2260,7 @@ async fn right_semi_join() -> Result<()> { let physical_plan = dataframe.create_physical_plan().await?; let expected = if repartition_joins { vec![ "SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST]", - " SortExec: [t1_id@0 ASC NULLS LAST]", + " SortExec: [t1_id@0 ASC NULLS LAST], global=false", " ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int]", " CoalesceBatchesExec: target_batch_size=4096", " HashJoinExec: mode=Partitioned, join_type=RightSemi, on=[(Column { name: \"t2_id\", index: 0 }, Column { name: \"t1_id\", index: 0 })], filter=BinaryExpr { left: Column { name: \"t2_name\", index: 1 }, op: NotEq, right: Column { name: \"t1_name\", index: 0 } }", @@ -2276,7 +2276,7 @@ async fn right_semi_join() -> Result<()> { } else { vec![ "SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST]", - " SortExec: [t1_id@0 ASC NULLS LAST]", + " SortExec: [t1_id@0 ASC NULLS LAST], global=false", " ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int]", " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", " CoalesceBatchesExec: target_batch_size=4096", @@ -2308,7 +2308,7 @@ async fn right_semi_join() -> Result<()> { let physical_plan = dataframe.create_physical_plan().await?; let expected = if repartition_joins { vec![ "SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST]", - " SortExec: [t1_id@0 ASC NULLS LAST]", + " SortExec: [t1_id@0 ASC NULLS LAST], global=false", " ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int]", " CoalesceBatchesExec: target_batch_size=4096", " HashJoinExec: mode=Partitioned, join_type=RightSemi, on=[(Column { name: \"t2_id\", index: 0 }, Column { name: \"t1_id\", index: 0 })], filter=BinaryExpr { left: Column { name: \"t2_name\", index: 0 }, op: NotEq, right: Column { name: \"t1_name\", index: 1 } }", @@ -2324,7 +2324,7 @@ async fn right_semi_join() -> Result<()> { } else { vec![ "SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST]", - " SortExec: [t1_id@0 ASC NULLS LAST]", + " SortExec: [t1_id@0 ASC NULLS LAST], global=false", " ProjectionExec: expr=[t1_id@0 as t1_id, t1_name@1 as t1_name, t1_int@2 as t1_int]", " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", " CoalesceBatchesExec: target_batch_size=4096", diff --git a/datafusion/core/tests/sql/window.rs b/datafusion/core/tests/sql/window.rs index 89df1045039d0..91ab5c32f3977 100644 --- a/datafusion/core/tests/sql/window.rs +++ b/datafusion/core/tests/sql/window.rs @@ -1748,7 +1748,7 @@ async fn over_order_by_sort_keys_sorting_global_order_compacting() -> Result<()> let expected = { vec![ "SortPreservingMergeExec: [c2@0 ASC NULLS LAST]", - " SortExec: [c2@0 ASC NULLS LAST]", + " SortExec: [c2@0 ASC NULLS LAST], global=false", " ProjectionExec: expr=[c2@0 as c2, MAX(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as MAX(aggregate_test_100.c9), SUM(aggregate_test_100.c9) ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@4 as SUM(aggregate_test_100.c9), MIN(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c2 ASC NULLS LAST, aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as MIN(aggregate_test_100.c9)]", " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", " WindowAggExec: wdw=[SUM(aggregate_test_100.c9): Ok(Field { name: \"SUM(aggregate_test_100.c9)\", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)) }]", @@ -2403,7 +2403,7 @@ async fn test_window_agg_global_sort() -> Result<()> { "SortPreservingMergeExec: [c1@0 ASC NULLS LAST]", " ProjectionExec: expr=[c1@0 as c1, ROW_NUMBER() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 as rn1]", " BoundedWindowAggExec: wdw=[ROW_NUMBER(): Ok(Field { name: \"ROW_NUMBER()\", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)) }]", - " SortExec: [c1@0 ASC NULLS LAST]", + " SortExec: [c1@0 ASC NULLS LAST], global=false", " CoalesceBatchesExec: target_batch_size=8192", " RepartitionExec: partitioning=Hash([Column { name: \"c1\", index: 0 }], 2), input_partitions=2", " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", @@ -2438,11 +2438,11 @@ async fn test_window_agg_global_sort_parallelize_sort_disabled() -> Result<()> { // Only 1 SortExec was added let expected = { vec![ - "SortExec: [c1@0 ASC NULLS LAST]", + "SortExec: [c1@0 ASC NULLS LAST], global=true", " CoalescePartitionsExec", " ProjectionExec: expr=[c1@0 as c1, ROW_NUMBER() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@1 as rn1]", " BoundedWindowAggExec: wdw=[ROW_NUMBER(): Ok(Field { name: \"ROW_NUMBER()\", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(NULL)), end_bound: Following(UInt64(NULL)) }]", - " SortExec: [c1@0 ASC NULLS LAST]", + " SortExec: [c1@0 ASC NULLS LAST], global=false", " CoalesceBatchesExec: target_batch_size=8192", " RepartitionExec: partitioning=Hash([Column { name: \"c1\", index: 0 }], 2), input_partitions=2", " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", @@ -2480,16 +2480,17 @@ async fn test_window_agg_global_sort_intermediate_parallel_sort() -> Result<()> // Only 1 SortExec was added let expected = { vec![ - "SortExec: [c1@0 ASC NULLS LAST]", - " ProjectionExec: expr=[c1@0 as c1, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 3 FOLLOWING@2 as sum1, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@3 as sum2]", - " BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9): Ok(Field { name: \"SUM(aggregate_test_100.c9)\", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)) }]", - " SortPreservingMergeExec: [c9@1 ASC NULLS LAST]", - " SortExec: [c9@1 ASC NULLS LAST]", - " BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9): Ok(Field { name: \"SUM(aggregate_test_100.c9)\", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(3)) }]", - " SortExec: [c1@0 ASC NULLS LAST,c9@1 ASC NULLS LAST]", - " CoalesceBatchesExec: target_batch_size=8192", - " RepartitionExec: partitioning=Hash([Column { name: \"c1\", index: 0 }], 2), input_partitions=2", - " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", + "SortPreservingMergeExec: [c1@0 ASC NULLS LAST]", + " SortExec: [c1@0 ASC NULLS LAST], global=false", + " ProjectionExec: expr=[c1@0 as c1, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 3 FOLLOWING@2 as sum1, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@3 as sum2]", + " BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9): Ok(Field { name: \"SUM(aggregate_test_100.c9)\", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)) }]", + " SortPreservingMergeExec: [c9@1 ASC NULLS LAST]", + " SortExec: [c9@1 ASC NULLS LAST], global=false", + " BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9): Ok(Field { name: \"SUM(aggregate_test_100.c9)\", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(3)) }]", + " SortExec: [c1@0 ASC NULLS LAST,c9@1 ASC NULLS LAST], global=false", + " CoalesceBatchesExec: target_batch_size=8192", + " RepartitionExec: partitioning=Hash([Column { name: \"c1\", index: 0 }], 2), input_partitions=2", + " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", ] }; @@ -2524,7 +2525,7 @@ async fn test_window_agg_with_global_limit() -> Result<()> { " AggregateExec: mode=Final, gby=[], aggr=[ARRAYAGG(aggregate_test_100.c13)]", " AggregateExec: mode=Partial, gby=[], aggr=[ARRAYAGG(aggregate_test_100.c13)]", " GlobalLimitExec: skip=0, fetch=1", - " SortExec: [c13@0 ASC NULLS LAST]", + " SortExec: [c13@0 ASC NULLS LAST], global=true", " ProjectionExec: expr=[c13@0 as c13]", ] }; diff --git a/datafusion/physical-expr/src/utils.rs b/datafusion/physical-expr/src/utils.rs index 533a606b69e85..44a33b2d45fb3 100644 --- a/datafusion/physical-expr/src/utils.rs +++ b/datafusion/physical-expr/src/utils.rs @@ -219,7 +219,7 @@ pub fn normalize_sort_requirement_with_equivalence_properties( } } -/// Checks whether given ordering requirements are satisfied by provided [PhysicalSortExpr]s. +/// Checks whether the required [PhysicalSortExpr]s are satisfied by the provided [PhysicalSortExpr]s. pub fn ordering_satisfy EquivalenceProperties>( provided: Option<&[PhysicalSortExpr]>, required: Option<&[PhysicalSortExpr]>, @@ -269,6 +269,7 @@ fn ordering_satisfy_concrete EquivalenceProperties>( } } +/// Checks whether the required ordering requirements are satisfied by the provided [PhysicalSortExpr]s. pub fn ordering_satisfy_requirement EquivalenceProperties>( provided: Option<&[PhysicalSortExpr]>, required: Option<&[PhysicalSortRequirements]>, From 06b40640e10a8d7bccb6a43be29bd816ed65e119 Mon Sep 17 00:00:00 2001 From: "mingmwang@ebay.com" Date: Mon, 13 Feb 2023 11:03:13 +0800 Subject: [PATCH 03/13] fix UT --- .../src/physical_optimizer/sort_enforcement2.rs | 14 +++++++++++++- datafusion/core/tests/sql/window.rs | 12 +++++++----- 2 files changed, 20 insertions(+), 6 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/sort_enforcement2.rs b/datafusion/core/src/physical_optimizer/sort_enforcement2.rs index 2ef3c6111b428..daee039817153 100644 --- a/datafusion/core/src/physical_optimizer/sort_enforcement2.rs +++ b/datafusion/core/src/physical_optimizer/sort_enforcement2.rs @@ -1056,7 +1056,19 @@ fn should_reverse_window_sort_requirements( top_reversed_requirement, ) } else { - false + if requirements_compatible( + top_reversed_requirement, + window_plan.required_input_ordering()[0].as_deref(), + || window_plan.equivalence_properties(), + ) || requirements_compatible( + window_plan.required_input_ordering()[0].as_deref(), + top_reversed_requirement, + || window_plan.equivalence_properties(), + ) { + true + } else { + false + } } } else if requirements_compatible( top_reversed_requirement, diff --git a/datafusion/core/tests/sql/window.rs b/datafusion/core/tests/sql/window.rs index 91ab5c32f3977..2508478c29316 100644 --- a/datafusion/core/tests/sql/window.rs +++ b/datafusion/core/tests/sql/window.rs @@ -2087,12 +2087,14 @@ async fn test_window_agg_complex_plan() -> Result<()> { " SortExec: [c3@2 ASC NULLS LAST,c2@1 ASC NULLS LAST], global=true", " BoundedWindowAggExec: wdw=[SUM(null_cases.c1): Ok(Field { name: \"SUM(null_cases.c1)\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow }]", " SortExec: [c3@2 ASC NULLS LAST,c1@0 ASC], global=true", - " WindowAggExec: wdw=[SUM(null_cases.c1): Ok(Field { name: \"SUM(null_cases.c1)\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(Int64(NULL)) }]", - " WindowAggExec: wdw=[SUM(null_cases.c1): Ok(Field { name: \"SUM(null_cases.c1)\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(11)), end_bound: Following(Int64(10)) }, SUM(null_cases.c1): Ok(Field { name: \"SUM(null_cases.c1)\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(Int64(NULL)) }, SUM(null_cases.c1): Ok(Field { name: \"SUM(null_cases.c1)\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(11)), end_bound: Following(Int64(NULL)) }, SUM(null_cases.c1): Ok(Field { name: \"SUM(null_cases.c1)\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow }]", - " WindowAggExec: wdw=[SUM(null_cases.c1): Ok(Field { name: \"SUM(null_cases.c1)\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(10)), end_bound: Following(Int64(11)) }, SUM(null_cases.c1): Ok(Field { name: \"SUM(null_cases.c1)\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow }, SUM(null_cases.c1): Ok(Field { name: \"SUM(null_cases.c1)\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: Following(Int64(11)) }, SUM(null_cases.c1): Ok(Field { name: \"SUM(null_cases.c1)\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(Int64(NULL)) }]", + " BoundedWindowAggExec: wdw=[SUM(null_cases.c1): Ok(Field { name: \"SUM(null_cases.c1)\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow }]", + " SortExec: [c3@2 ASC NULLS LAST,c1@0 DESC], global=true", + " WindowAggExec: wdw=[SUM(null_cases.c1): Ok(Field { name: \"SUM(null_cases.c1)\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(11)), end_bound: Following(Int64(10)) }, SUM(null_cases.c1): Ok(Field { name: \"SUM(null_cases.c1)\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(Int64(NULL)) }, SUM(null_cases.c1): Ok(Field { name: \"SUM(null_cases.c1)\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(11)), end_bound: Following(Int64(NULL)) }, SUM(null_cases.c1): Ok(Field { name: \"SUM(null_cases.c1)\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow }]", " WindowAggExec: wdw=[SUM(null_cases.c1): Ok(Field { name: \"SUM(null_cases.c1)\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(10)), end_bound: Following(Int64(11)) }, SUM(null_cases.c1): Ok(Field { name: \"SUM(null_cases.c1)\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow }, SUM(null_cases.c1): Ok(Field { name: \"SUM(null_cases.c1)\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: Following(Int64(11)) }, SUM(null_cases.c1): Ok(Field { name: \"SUM(null_cases.c1)\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(Int64(NULL)) }]", - " BoundedWindowAggExec: wdw=[SUM(null_cases.c1): Ok(Field { name: \"SUM(null_cases.c1)\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow }]", - " SortExec: [c3@2 DESC,c1@0 ASC NULLS LAST], global=true", + " SortExec: [c3@2 DESC NULLS LAST], global=true", + " WindowAggExec: wdw=[SUM(null_cases.c1): Ok(Field { name: \"SUM(null_cases.c1)\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(10)), end_bound: Following(Int64(11)) }, SUM(null_cases.c1): Ok(Field { name: \"SUM(null_cases.c1)\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow }, SUM(null_cases.c1): Ok(Field { name: \"SUM(null_cases.c1)\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: Following(Int64(11)) }, SUM(null_cases.c1): Ok(Field { name: \"SUM(null_cases.c1)\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(Int64(NULL)) }]", + " BoundedWindowAggExec: wdw=[SUM(null_cases.c1): Ok(Field { name: \"SUM(null_cases.c1)\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow }]", + " SortExec: [c3@2 DESC,c1@0 ASC NULLS LAST], global=true", ] }; From 601a4d08ea258f35b7c2bb3d827bbb5e22032f51 Mon Sep 17 00:00:00 2001 From: "mingmwang@ebay.com" Date: Mon, 13 Feb 2023 17:37:14 +0800 Subject: [PATCH 04/13] add more UTs to sort_enforcement2.rs --- .../physical_optimizer/sort_enforcement.rs | 30 +- .../physical_optimizer/sort_enforcement2.rs | 580 +++++++++++------- .../windows/bounded_window_agg_exec.rs | 4 +- .../physical_plan/windows/window_agg_exec.rs | 4 +- 4 files changed, 400 insertions(+), 218 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/sort_enforcement.rs b/datafusion/core/src/physical_optimizer/sort_enforcement.rs index 5253a2da437fe..eb5b4789e539c 100644 --- a/datafusion/core/src/physical_optimizer/sort_enforcement.rs +++ b/datafusion/core/src/physical_optimizer/sort_enforcement.rs @@ -27,7 +27,7 @@ //! somehow get the fragment //! //! ```text -//! SortExec: [nullable_col@0 ASC] +//! SortExec: [nullable_co l@0 ASC] //! SortExec: [non_nullable_col@1 ASC] //! ``` //! @@ -1185,6 +1185,34 @@ mod tests { 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 = vec![ + "SortPreservingMergeExec: [nullable_col@0 ASC]", + " SortExec: [nullable_col@0 ASC], global=true", + " SortPreservingMergeExec: [nullable_col@0 ASC]", + " SortExec: [nullable_col@0 ASC], global=true", + " MemoryExec: partitions=0, partition_sizes=[]", + ]; + let expected_optimized = vec![ + "SortPreservingMergeExec: [nullable_col@0 ASC]", + " SortPreservingMergeExec: [nullable_col@0 ASC]", + " SortExec: [nullable_col@0 ASC], global=true", + " MemoryExec: partitions=0, partition_sizes=[]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan); + Ok(()) + } + #[tokio::test] async fn test_remove_unnecessary_sort2() -> Result<()> { let schema = create_test_schema()?; diff --git a/datafusion/core/src/physical_optimizer/sort_enforcement2.rs b/datafusion/core/src/physical_optimizer/sort_enforcement2.rs index a4f9a27e32327..b624fd762dce0 100644 --- a/datafusion/core/src/physical_optimizer/sort_enforcement2.rs +++ b/datafusion/core/src/physical_optimizer/sort_enforcement2.rs @@ -25,8 +25,12 @@ //! //! A non-realistic but easy to follow example for sort removals: Assume that we //! somehow get the fragment -//! "SortExec: [nullable_col@0 ASC]", -//! " SortExec: [non_nullable_col@1 ASC]", +//! +//! ```text +//! SortExec: [nullable_co l@0 ASC] +//! SortExec: [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 crate::config::ConfigOptions; @@ -278,49 +282,47 @@ fn ensure_sorting( .as_any() .downcast_ref::() { - if !sort_pres_exec.satisfy_distribution() { - // SortPreservingMergeExec + SortExec(local/global) is the same as the global SortExec - // Remove unnecessary SortPreservingMergeExec + SortExec(local/global) - if let Some(child_sort_exec) = - sort_pres_exec.input().as_any().downcast_ref::() - { - if sort_pres_exec.expr() == child_sort_exec.expr() { - if !requirements.impact_result_ordering - && requirements.required_ordering.is_none() - { - println!("remove SortPreservingMergeExec + SortExec due to no need to keep ordering"); - return Ok(Some(PlanWithSortRequirements { - plan: Arc::new(TombStoneExec::new( - child_sort_exec.input().clone(), - )), - impact_result_ordering: false, - required_ordering: None, - adjusted_request_ordering: vec![None], - })); - } else if ordering_satisfy( - child_sort_exec.input().output_ordering(), - child_sort_exec.output_ordering(), - || child_sort_exec.input().equivalence_properties(), - ) && child_sort_exec - .input() - .output_partitioning() - .partition_count() - == 1 - { - println!("remove SortPreservingMergeExec + SortExec due to child already satisfy"); - return Ok(Some(PlanWithSortRequirements { - plan: Arc::new(TombStoneExec::new( - child_sort_exec.input().clone(), - )), - impact_result_ordering: true, - required_ordering: None, - adjusted_request_ordering: vec![ - requirements.required_ordering, - ], - })); - } + // SortPreservingMergeExec + SortExec(local/global) is the same as the global SortExec + // Remove unnecessary SortPreservingMergeExec + SortExec(local/global) + if let Some(child_sort_exec) = + sort_pres_exec.input().as_any().downcast_ref::() + { + if sort_pres_exec.expr() == child_sort_exec.expr() { + if !requirements.impact_result_ordering + && requirements.required_ordering.is_none() + { + println!("remove SortPreservingMergeExec + SortExec due to no need to keep ordering"); + return Ok(Some(PlanWithSortRequirements { + plan: Arc::new(TombStoneExec::new( + child_sort_exec.input().clone(), + )), + impact_result_ordering: false, + required_ordering: None, + adjusted_request_ordering: vec![None], + })); + } else if ordering_satisfy( + child_sort_exec.input().output_ordering(), + child_sort_exec.output_ordering(), + || child_sort_exec.input().equivalence_properties(), + ) && child_sort_exec + .input() + .output_partitioning() + .partition_count() + == 1 + { + println!("remove SortPreservingMergeExec + SortExec due to child already satisfy"); + return Ok(Some(PlanWithSortRequirements { + plan: Arc::new(TombStoneExec::new( + child_sort_exec.input().clone(), + )), + impact_result_ordering: true, + required_ordering: None, + adjusted_request_ordering: vec![requirements.required_ordering], + })); } - } else { + } + } else { + if !sort_pres_exec.satisfy_distribution() { // Remove unnecessary SortPreservingMergeExec only if !requirements.impact_result_ordering { println!( @@ -1198,6 +1200,13 @@ mod tests { Ok(schema) } + // Util function to get string representation of a physical plan + fn get_plan_string(plan: &Arc) -> Vec { + let formatted = displayable(plan.as_ref()).indent().to_string(); + let actual: Vec<&str> = formatted.trim().lines().collect(); + actual.iter().map(|elem| elem.to_string()).collect() + } + #[tokio::test] async fn test_is_column_aligned_nullable() -> Result<()> { let schema = create_test_schema()?; @@ -1307,11 +1316,8 @@ mod tests { // Run the actual optimizer let optimized_physical_plan = TopDownEnforceSorting::new().optimize(physical_plan, state.config_options())?; - - let formatted = displayable(optimized_physical_plan.as_ref()) - .indent() - .to_string(); - let actual: Vec<&str> = formatted.trim().lines().collect(); + // 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" @@ -1321,7 +1327,27 @@ mod tests { } #[tokio::test] - async fn test_not_remove_sort_window_multilayer() -> Result<()> { + 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 = vec![ + "SortExec: [nullable_col@0 ASC], global=true", + " SortExec: [non_nullable_col@1 ASC], global=true", + " MemoryExec: partitions=0, partition_sizes=[]", + ]; + let expected_optimized = vec![ + "SortExec: [nullable_col@0 ASC], global=true", + " MemoryExec: partitions=0, partition_sizes=[]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan); + Ok(()) + } + + #[tokio::test] + async fn test_remove_unnecessary_sort_window_multilayer() -> Result<()> { let schema = create_test_schema()?; let source = memory_exec(&schema); @@ -1347,6 +1373,8 @@ mod tests { )]; 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(), @@ -1366,14 +1394,14 @@ mod tests { " MemoryExec: partitions=0, partition_sizes=[]", ]; - // let expected_optimized = vec![ - // "WindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(NULL) }]", - // " FilterExec: NOT non_nullable_col@1", - // " WindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow }]", - // " SortExec: [non_nullable_col@1 DESC]", - // " MemoryExec: partitions=0, partition_sizes=[]", - // ]; - assert_optimized!(expected_input, expected_input, physical_plan); + let expected_optimized = vec![ + "WindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(NULL) }]", + " FilterExec: NOT non_nullable_col@1", + " WindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow }]", + " SortExec: [non_nullable_col@1 DESC]", + " MemoryExec: partitions=0, partition_sizes=[]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan); Ok(()) } @@ -1437,57 +1465,6 @@ mod tests { #[tokio::test] async fn test_remove_unnecessary_sort1() -> 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 = vec![ - "SortExec: [nullable_col@0 ASC], global=true", - " SortExec: [non_nullable_col@1 ASC], global=true", - " MemoryExec: partitions=0, partition_sizes=[]", - ]; - // Keep the top SortExec - let expected_optimized = [ - "SortExec: [nullable_col@0 ASC], global=true", - " MemoryExec: partitions=0, partition_sizes=[]", - ]; - assert_optimized!(expected_input, expected_optimized, physical_plan); - Ok(()) - } - - #[tokio::test] - async fn test_remove_unnecessary_sort2() -> Result<()> { - let schema = create_test_schema()?; - let source = memory_exec(&schema); - - let input = sort_exec(vec![sort_expr("non_nullable_col", &schema)], source); - let input2 = sort_exec( - vec![ - sort_expr("nullable_col", &schema), - sort_expr("non_nullable_col", &schema), - ], - input, - ); - let physical_plan = sort_exec(vec![sort_expr("nullable_col", &schema)], input2); - - let expected_input = vec![ - "SortExec: [nullable_col@0 ASC], global=true", - " SortExec: [nullable_col@0 ASC,non_nullable_col@1 ASC], global=true", - " SortExec: [non_nullable_col@1 ASC], global=true", - " MemoryExec: partitions=0, partition_sizes=[]", - ]; - // Keep the middle SortExec - let expected_optimized = [ - "SortExec: [nullable_col@0 ASC,non_nullable_col@1 ASC], global=true", - " MemoryExec: partitions=0, partition_sizes=[]", - ]; - assert_optimized!(expected_input, expected_optimized, physical_plan); - 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("nullable_col", &schema)]; @@ -1514,7 +1491,7 @@ mod tests { } #[tokio::test] - async fn test_remove_unnecessary_sort4() -> Result<()> { + 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)]; @@ -1529,22 +1506,23 @@ mod tests { let spm2 = sort_preserving_merge_exec(sort_exprs, sort2); let sort_exprs = vec![sort_expr("nullable_col", &schema)]; - let sort3 = sort_exec(sort_exprs.clone(), spm2); - let physical_plan = sort_preserving_merge_exec(sort_exprs, sort3); + let sort3 = sort_exec(sort_exprs, spm2); + let physical_plan = repartition_exec(repartition_exec(sort3)); let expected_input = vec![ - "SortPreservingMergeExec: [nullable_col@0 ASC]", - " SortExec: [nullable_col@0 ASC], global=true", - " SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", - " SortExec: [nullable_col@0 ASC,non_nullable_col@1 ASC], global=true", - " SortPreservingMergeExec: [non_nullable_col@1 ASC]", - " SortExec: [non_nullable_col@1 ASC], global=true", - " MemoryExec: partitions=0, partition_sizes=[]", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " SortExec: [nullable_col@0 ASC], global=true", + " SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", + " SortExec: [nullable_col@0 ASC,non_nullable_col@1 ASC], global=true", + " SortPreservingMergeExec: [non_nullable_col@1 ASC]", + " SortExec: [non_nullable_col@1 ASC], global=true", + " MemoryExec: partitions=0, partition_sizes=[]", ]; - // Keep the middle SortPreservingMergeExec + SortExec + let expected_optimized = vec![ - "SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", - " SortExec: [nullable_col@0 ASC,non_nullable_col@1 ASC], global=true", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=0", " MemoryExec: partitions=0, partition_sizes=[]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan); @@ -1552,7 +1530,7 @@ mod tests { } #[tokio::test] - async fn test_remove_unnecessary_sort5() -> Result<()> { + 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)]; @@ -1563,33 +1541,92 @@ mod tests { sort_expr("nullable_col", &schema), sort_expr("non_nullable_col", &schema), ]; - let sort2 = sort_exec(sort_exprs.clone(), spm); + let repartition_exec = repartition_exec(spm); + let sort2 = sort_exec(sort_exprs.clone(), repartition_exec); let spm2 = sort_preserving_merge_exec(sort_exprs, sort2); - let sort_exprs = vec![sort_expr("nullable_col", &schema)]; - let sort3 = sort_exec(sort_exprs.clone(), spm2); - let physical_plan = repartition_exec(repartition_exec(sort3)); + 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 = vec![ - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " SortExec: [nullable_col@0 ASC], global=true", - " SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", - " SortExec: [nullable_col@0 ASC,non_nullable_col@1 ASC], global=true", - " SortPreservingMergeExec: [non_nullable_col@1 ASC]", - " SortExec: [non_nullable_col@1 ASC], global=true", - " MemoryExec: partitions=0, partition_sizes=[]", + "AggregateExec: mode=Final, gby=[], aggr=[]", + " SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", + " SortExec: [nullable_col@0 ASC,non_nullable_col@1 ASC], global=true", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " SortPreservingMergeExec: [non_nullable_col@1 ASC]", + " SortExec: [non_nullable_col@1 ASC], global=true", + " MemoryExec: partitions=0, partition_sizes=[]", ]; let expected_optimized = vec![ - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=0", + "AggregateExec: mode=Final, gby=[], aggr=[]", + " CoalescePartitionsExec", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=0", + " MemoryExec: partitions=0, partition_sizes=[]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan); + Ok(()) + } + + #[tokio::test] + async fn test_remove_unnecessary_sort4() -> 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 = vec![ + "SortPreservingMergeExec: [nullable_col@0 ASC]", + " SortExec: [nullable_col@0 ASC], global=true", + " SortPreservingMergeExec: [nullable_col@0 ASC]", + " SortExec: [nullable_col@0 ASC], global=true", + " MemoryExec: partitions=0, partition_sizes=[]", + ]; + let expected_optimized = vec![ + "SortPreservingMergeExec: [nullable_col@0 ASC]", + " SortExec: [nullable_col@0 ASC], global=true", " MemoryExec: partitions=0, partition_sizes=[]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan); Ok(()) } + #[tokio::test] + async fn test_remove_unnecessary_sort5() -> Result<()> { + let schema = create_test_schema()?; + let source = memory_exec(&schema); + + let input = sort_exec(vec![sort_expr("non_nullable_col", &schema)], source); + let input2 = sort_exec( + vec![ + sort_expr("nullable_col", &schema), + sort_expr("non_nullable_col", &schema), + ], + input, + ); + let physical_plan = sort_exec(vec![sort_expr("nullable_col", &schema)], input2); + + let expected_input = vec![ + "SortExec: [nullable_col@0 ASC], global=true", + " SortExec: [nullable_col@0 ASC,non_nullable_col@1 ASC], global=true", + " SortExec: [non_nullable_col@1 ASC], global=true", + " MemoryExec: partitions=0, partition_sizes=[]", + ]; + // Keep the middle SortExec + let expected_optimized = [ + "SortExec: [nullable_col@0 ASC,non_nullable_col@1 ASC], global=true", + " MemoryExec: partitions=0, partition_sizes=[]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan); + Ok(()) + } + #[tokio::test] async fn test_remove_unnecessary_spm1() -> Result<()> { let schema = create_test_schema()?; @@ -1643,6 +1680,52 @@ mod tests { 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 = vec![ + "SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", + " UnionExec", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, output_ordering=[nullable_col@0 ASC], projection=[nullable_col, non_nullable_col]", + " GlobalLimitExec: skip=0, fetch=100", + " LocalLimitExec: fetch=100", + " SortExec: [nullable_col@0 ASC,non_nullable_col@1 ASC], global=true", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + ]; + + // We should keep the bottom `SortExec`. + let expected_optimized = vec![ + "SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", + " SortExec: [nullable_col@0 ASC,non_nullable_col@1 ASC], global=false", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", + " UnionExec", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, output_ordering=[nullable_col@0 ASC], projection=[nullable_col, non_nullable_col]", + " GlobalLimitExec: skip=0, fetch=100", + " LocalLimitExec: fetch=100", + " SortExec: [nullable_col@0 ASC,non_nullable_col@1 ASC], global=true", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan); + Ok(()) + } + #[tokio::test] async fn test_change_wrong_sorting() -> Result<()> { let schema = create_test_schema()?; @@ -1799,32 +1882,40 @@ mod tests { let schema = create_test_schema()?; let source1 = parquet_exec(&schema); - let sort_exprs1 = vec![sort_expr("nullable_col", &schema)]; - let sort_exprs2 = vec![ + let sort_exprs1 = vec![ sort_expr("nullable_col", &schema), sort_expr("non_nullable_col", &schema), ]; - let sort = sort_exec(sort_exprs1.clone(), source1); + 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 source2 = parquet_exec_sorted(&schema, sort_exprs2.clone()); + 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![source2, sort]); - let physical_plan = sort_preserving_merge_exec(sort_exprs2, union); + let union = union_exec(vec![sort1, source2, sort2]); + let physical_plan = sort_preserving_merge_exec(parquet_sort_exprs, union); - // Input is an invalid plan. In this case rule should add required sorting in appropriate places. + // 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 = vec![ - "SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", + "SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", - " ParquetExec: limit=None, partitions={1 group: [[x]]}, output_ordering=[nullable_col@0 ASC, non_nullable_col@1 ASC], projection=[nullable_col, non_nullable_col]", + " SortExec: [nullable_col@0 ASC,non_nullable_col@1 ASC], global=true", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, output_ordering=[nullable_col@0 ASC], projection=[nullable_col, non_nullable_col]", " SortExec: [nullable_col@0 ASC], global=true", " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", ]; - // expect to replace the wrong SortExec with the correct one + // should adjust sorting in the first input of the union such that it is not unnecessarily fine let expected_optimized = vec![ - "SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", + "SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", - " ParquetExec: limit=None, partitions={1 group: [[x]]}, output_ordering=[nullable_col@0 ASC, non_nullable_col@1 ASC], projection=[nullable_col, non_nullable_col]", - " SortExec: [nullable_col@0 ASC,non_nullable_col@1 ASC], global=true", + " SortExec: [nullable_col@0 ASC], global=true", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, output_ordering=[nullable_col@0 ASC], projection=[nullable_col, non_nullable_col]", + " SortExec: [nullable_col@0 ASC], global=true", " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan); @@ -1840,36 +1931,36 @@ mod tests { 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 sort1 = sort_exec(sort_exprs2.clone(), source1.clone()); + let sort2 = sort_exec(sort_exprs2.clone(), source1); - let parquet_sort_exprs = vec![sort_expr("nullable_col", &schema)]; - let source2 = parquet_exec_sorted(&schema, parquet_sort_exprs.clone()); + let source2 = parquet_exec_sorted(&schema, sort_exprs2); let union = union_exec(vec![sort1, source2, sort2]); - let physical_plan = sort_preserving_merge_exec(parquet_sort_exprs, union); + let physical_plan = sort_preserving_merge_exec(sort_exprs1, 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). + // 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 = vec![ - "SortPreservingMergeExec: [nullable_col@0 ASC]", + "SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", " UnionExec", - " SortExec: [nullable_col@0 ASC,non_nullable_col@1 ASC], global=true", + " SortExec: [nullable_col@0 ASC], global=true", " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", " ParquetExec: limit=None, partitions={1 group: [[x]]}, output_ordering=[nullable_col@0 ASC], projection=[nullable_col, non_nullable_col]", " SortExec: [nullable_col@0 ASC], global=true", " ParquetExec: limit=None, partitions={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 = vec![ - "SortPreservingMergeExec: [nullable_col@0 ASC]", + "SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", " UnionExec", " SortExec: [nullable_col@0 ASC,non_nullable_col@1 ASC], global=true", " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " ParquetExec: limit=None, partitions={1 group: [[x]]}, output_ordering=[nullable_col@0 ASC], projection=[nullable_col, non_nullable_col]", - " SortExec: [nullable_col@0 ASC], global=true", + " SortExec: [nullable_col@0 ASC,non_nullable_col@1 ASC], global=true", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, output_ordering=[nullable_col@0 ASC], projection=[nullable_col, non_nullable_col]", + " SortExec: [nullable_col@0 ASC,non_nullable_col@1 ASC], global=true", " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan); @@ -1885,44 +1976,101 @@ mod tests { 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 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, source2, sort2]); - let physical_plan = sort_preserving_merge_exec(sort_exprs1, union); + let union = union_exec(vec![sort1, sort2]); + let physical_plan = sort_preserving_merge_exec(sort_exprs3, 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). + // 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 = vec![ - "SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", + "SortPreservingMergeExec: [nullable_col@0 ASC]", + " UnionExec", + " SortExec: [nullable_col@0 ASC,non_nullable_col@1 ASC], global=true", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " SortExec: [nullable_col@0 ASC,non_nullable_col@1 DESC NULLS LAST], global=true", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + ]; + let expected_optimized = vec![ + "SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", " SortExec: [nullable_col@0 ASC], global=true", " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " ParquetExec: limit=None, partitions={1 group: [[x]]}, output_ordering=[nullable_col@0 ASC], projection=[nullable_col, non_nullable_col]", " SortExec: [nullable_col@0 ASC], global=true", " ParquetExec: limit=None, partitions={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 + assert_optimized!(expected_input, expected_optimized, physical_plan); + 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 = vec![ + "SortPreservingMergeExec: [nullable_col@0 ASC]", + " UnionExec", + " SortExec: [nullable_col@0 ASC], global=true", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, output_ordering=[nullable_col@0 ASC], projection=[nullable_col, non_nullable_col]", + " SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " ParquetExec: limit=None, partitions={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 = vec![ - "SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", - " UnionExec", - " SortExec: [nullable_col@0 ASC,non_nullable_col@1 ASC], global=true", - " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " SortExec: [nullable_col@0 ASC,non_nullable_col@1 ASC], global=true", - " ParquetExec: limit=None, partitions={1 group: [[x]]}, output_ordering=[nullable_col@0 ASC], projection=[nullable_col, non_nullable_col]", - " SortExec: [nullable_col@0 ASC,non_nullable_col@1 ASC], global=true", - " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + "SortPreservingMergeExec: [nullable_col@0 ASC]", + " UnionExec", + " SortExec: [nullable_col@0 ASC], global=true", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, output_ordering=[nullable_col@0 ASC], projection=[nullable_col, non_nullable_col]", + " SortExec: [nullable_col@0 ASC], global=false", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan); Ok(()) } #[tokio::test] - async fn test_union_inputs_different_sorted6() -> Result<()> { + async fn test_union_inputs_different_sorted7() -> Result<()> { let schema = create_test_schema()?; let source1 = parquet_exec(&schema); @@ -1963,46 +2111,48 @@ mod tests { } #[tokio::test] - async fn test_do_not_remove_sort_with_limit() -> Result<()> { + async fn test_window_multi_path_sort() -> Result<()> { let schema = create_test_schema()?; - let source1 = parquet_exec(&schema); - let sort_exprs = vec![ + let sort_exprs1 = 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 sort_exprs2 = vec![sort_expr("nullable_col", &schema)]; + // reverse sorting of sort_exprs2 + let reversed_sort_exprs2 = 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(reversed_sort_exprs2.clone(), source1); + let sort2 = sort_exec(reversed_sort_exprs2.clone(), source2); - let union = union_exec(vec![source2, limit]); - let repartition = repartition_exec(union); - let physical_plan = sort_preserving_merge_exec(sort_exprs, repartition); + let union = union_exec(vec![sort1, sort2]); + let physical_plan = window_exec("nullable_col", reversed_sort_exprs2, union); + // 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 = vec![ - "SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", - " UnionExec", + "WindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow }]", + " UnionExec", + " SortExec: [nullable_col@0 DESC NULLS LAST], global=true", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, output_ordering=[nullable_col@0 ASC, non_nullable_col@1 ASC], projection=[nullable_col, non_nullable_col]", + " SortExec: [nullable_col@0 DESC NULLS LAST], global=true", " ParquetExec: limit=None, partitions={1 group: [[x]]}, output_ordering=[nullable_col@0 ASC], projection=[nullable_col, non_nullable_col]", - " GlobalLimitExec: skip=0, fetch=100", - " LocalLimitExec: fetch=100", - " SortExec: [nullable_col@0 ASC,non_nullable_col@1 ASC], global=true", - " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", ]; - - // expect to keep the bottom SortExec let expected_optimized = vec![ - "SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", - " SortExec: [nullable_col@0 ASC,non_nullable_col@1 ASC], global=false", - " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", - " UnionExec", - " ParquetExec: limit=None, partitions={1 group: [[x]]}, output_ordering=[nullable_col@0 ASC], projection=[nullable_col, non_nullable_col]", - " GlobalLimitExec: skip=0, fetch=100", - " LocalLimitExec: fetch=100", - " SortExec: [nullable_col@0 ASC,non_nullable_col@1 ASC], global=true", - " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + "WindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(NULL) }]", + " UnionExec", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, output_ordering=[nullable_col@0 ASC, non_nullable_col@1 ASC], projection=[nullable_col, non_nullable_col]", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, output_ordering=[nullable_col@0 ASC], projection=[nullable_col, non_nullable_col]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan); Ok(()) diff --git a/datafusion/core/src/physical_plan/windows/bounded_window_agg_exec.rs b/datafusion/core/src/physical_plan/windows/bounded_window_agg_exec.rs index 6222561bb1a27..2affa9f364109 100644 --- a/datafusion/core/src/physical_plan/windows/bounded_window_agg_exec.rs +++ b/datafusion/core/src/physical_plan/windows/bounded_window_agg_exec.rs @@ -124,7 +124,9 @@ impl BoundedWindowAggExec { let mut result = vec![]; // All window exprs have the same partition by, so we just use the first one: let partition_by = self.window_expr()[0].partition_by(); - let sort_keys = self.output_ordering().unwrap_or(self.sort_keys.as_deref().unwrap_or(&[])); + let sort_keys = self + .output_ordering() + .unwrap_or(self.sort_keys.as_deref().unwrap_or(&[])); for item in partition_by { if let Some(a) = sort_keys.iter().find(|&e| e.expr.eq(item)) { result.push(a.clone()); diff --git a/datafusion/core/src/physical_plan/windows/window_agg_exec.rs b/datafusion/core/src/physical_plan/windows/window_agg_exec.rs index 7ab9951ae107c..16404579233ca 100644 --- a/datafusion/core/src/physical_plan/windows/window_agg_exec.rs +++ b/datafusion/core/src/physical_plan/windows/window_agg_exec.rs @@ -115,7 +115,9 @@ impl WindowAggExec { let mut result = vec![]; // All window exprs have the same partition by, so we just use the first one: let partition_by = self.window_expr()[0].partition_by(); - let sort_keys = self.output_ordering().unwrap_or(self.sort_keys.as_deref().unwrap_or(&[])); + let sort_keys = self + .output_ordering() + .unwrap_or(self.sort_keys.as_deref().unwrap_or(&[])); for item in partition_by { if let Some(a) = sort_keys.iter().find(|&e| e.expr.eq(item)) { result.push(a.clone()); From 5285a3ef2ba51cfc8856fb6d1232368fdfbaa3eb Mon Sep 17 00:00:00 2001 From: "mingmwang@ebay.com" Date: Wed, 15 Feb 2023 16:11:51 +0800 Subject: [PATCH 05/13] refine codebase --- datafusion/core/src/dataframe.rs | 8 +- datafusion/core/src/execution/context.rs | 1 - .../physical_optimizer/dist_enforcement.rs | 19 +- .../physical_optimizer/sort_enforcement2.rs | 970 ++++++++++-------- datafusion/core/src/physical_plan/planner.rs | 1 - .../sorts/sort_preserving_merge.rs | 21 - .../windows/bounded_window_agg_exec.rs | 5 +- .../physical_plan/windows/window_agg_exec.rs | 4 +- datafusion/physical-expr/src/utils.rs | 13 +- datafusion/sql/src/statement.rs | 8 +- .../substrait/src/logical_plan/consumer.rs | 122 ++- 11 files changed, 618 insertions(+), 554 deletions(-) diff --git a/datafusion/core/src/dataframe.rs b/datafusion/core/src/dataframe.rs index 26fe5c0512049..883f203bbd2b3 100644 --- a/datafusion/core/src/dataframe.rs +++ b/datafusion/core/src/dataframe.rs @@ -459,9 +459,11 @@ impl DataFrame { .and_then(|r| r.columns().first()) .and_then(|c| c.as_any().downcast_ref::()) .and_then(|a| a.values().first()) - .ok_or(DataFusionError::Internal( - "Unexpected output when collecting for count()".to_string(), - ))? as usize; + .ok_or_else(|| { + DataFusionError::Internal( + "Unexpected output when collecting for count()".to_string(), + ) + })? as usize; Ok(len) } diff --git a/datafusion/core/src/execution/context.rs b/datafusion/core/src/execution/context.rs index 992b2291ced00..4fd55ea571828 100644 --- a/datafusion/core/src/execution/context.rs +++ b/datafusion/core/src/execution/context.rs @@ -1089,7 +1089,6 @@ impl QueryPlanner for DefaultQueryPlanner { session_state: &SessionState, ) -> Result> { let planner = DefaultPhysicalPlanner::default(); - println!("optimized logical plan {:?}", logical_plan); planner .create_physical_plan(logical_plan, session_state) .await diff --git a/datafusion/core/src/physical_optimizer/dist_enforcement.rs b/datafusion/core/src/physical_optimizer/dist_enforcement.rs index 3bc599c0cb0d2..222b0f5e1a925 100644 --- a/datafusion/core/src/physical_optimizer/dist_enforcement.rs +++ b/datafusion/core/src/physical_optimizer/dist_enforcement.rs @@ -856,14 +856,17 @@ fn ensure_distribution( Distribution::SinglePartition if child.output_partitioning().partition_count() > 1 => { - if repartition_sort && required_ordering.is_some() { - let new_physical_ordering = create_sort_expr_from_requirement( - required_ordering.unwrap().as_ref(), - ); - Ok(Arc::new(SortPreservingMergeExec::new_for_distribuion( - new_physical_ordering, - child.clone(), - ))) + if repartition_sort { + if let Some(ordering) = required_ordering { + let new_physical_ordering = + create_sort_expr_from_requirement(ordering.as_ref()); + Ok(Arc::new(SortPreservingMergeExec::new( + new_physical_ordering, + child.clone(), + ))) + } else { + Ok(Arc::new(CoalescePartitionsExec::new(child.clone()))) + } } else { Ok(Arc::new(CoalescePartitionsExec::new(child.clone()))) } diff --git a/datafusion/core/src/physical_optimizer/sort_enforcement2.rs b/datafusion/core/src/physical_optimizer/sort_enforcement2.rs index b624fd762dce0..25c407e2a7f72 100644 --- a/datafusion/core/src/physical_optimizer/sort_enforcement2.rs +++ b/datafusion/core/src/physical_optimizer/sort_enforcement2.rs @@ -31,13 +31,14 @@ //! SortExec: [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. +//! in the physical plan. The child sort is unnecessary since its result is overwritten +//! by the parent SortExec. Therefore, this rule removes it from the physical plan. use crate::config::ConfigOptions; use crate::error::Result; use crate::execution::context::TaskContext; use crate::physical_optimizer::utils::add_sort_above_child; use crate::physical_optimizer::PhysicalOptimizerRule; +use crate::physical_plan::coalesce_partitions::CoalescePartitionsExec; use crate::physical_plan::filter::FilterExec; use crate::physical_plan::joins::utils::JoinSide; use crate::physical_plan::joins::SortMergeJoinExec; @@ -50,7 +51,7 @@ 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::{ - displayable, with_new_children_if_necessary, DisplayFormatType, ExecutionPlan, + with_new_children_if_necessary, DisplayFormatType, Distribution, ExecutionPlan, Partitioning, SendableRecordBatchStream, }; use arrow::datatypes::SchemaRef; @@ -60,15 +61,16 @@ use datafusion_physical_expr::utils::{ create_sort_expr_from_requirement, map_requirement_before_projection, ordering_satisfy, ordering_satisfy_requirement, requirements_compatible, }; +use datafusion_physical_expr::window::WindowExpr; use datafusion_physical_expr::{ - EquivalenceProperties, PhysicalExpr, PhysicalSortExpr, PhysicalSortRequirements, + new_sort_requirements, EquivalenceProperties, PhysicalExpr, PhysicalSortExpr, + PhysicalSortRequirements, }; use itertools::izip; use std::any::Any; -use std::iter::zip; use std::sync::Arc; -/// This rule inspects SortExec's in the given physical plan and removes the +/// This rule implements a TOP-Downinspects SortExec's in the given physical plan and removes the /// ones it can prove unnecessary. #[derive(Default)] pub struct TopDownEnforceSorting {} @@ -87,6 +89,8 @@ struct PlanWithSortRequirements { plan: Arc, /// Whether the plan could impact the final result ordering impact_result_ordering: bool, + /// Parent has the SinglePartition requirement to children + satisfy_single_distribution: bool, /// Parent required sort ordering required_ordering: Option>, /// The adjusted request sort ordering to children. @@ -97,13 +101,14 @@ struct PlanWithSortRequirements { impl PlanWithSortRequirements { pub fn init(plan: Arc) -> Self { let impact_result_ordering = plan.output_ordering().is_some() - || plan.output_partitioning().partition_count() == 1 + || plan.output_partitioning().partition_count() <= 1 || plan.as_any().downcast_ref::().is_some() || plan.as_any().downcast_ref::().is_some(); let request_ordering = plan.required_input_ordering(); PlanWithSortRequirements { plan, impact_result_ordering, + satisfy_single_distribution: false, required_ordering: None, adjusted_request_ordering: request_ordering, } @@ -114,6 +119,7 @@ impl PlanWithSortRequirements { PlanWithSortRequirements { plan, impact_result_ordering: false, + satisfy_single_distribution: false, required_ordering: None, adjusted_request_ordering: request_ordering, } @@ -122,44 +128,43 @@ impl PlanWithSortRequirements { pub fn children(&self) -> Vec { let plan_children = self.plan.children(); assert_eq!(plan_children.len(), self.adjusted_request_ordering.len()); - let child_impact_result_ordering = if self - .plan - .as_any() - .downcast_ref::() - .is_some() - || self - .plan - .as_any() - .downcast_ref::() - .is_some() - { - true - } else if self.plan.as_any().downcast_ref::().is_some() { - false - } else if self.plan.as_any().downcast_ref::().is_some() { - self.plan.output_ordering().is_some() && self.impact_result_ordering - } else { - self.plan.maintains_input_order().iter().all(|o| *o) - && self.impact_result_ordering - }; - println!( - "child_impact_result_ordering {:?}", - child_impact_result_ordering - ); - plan_children - .into_iter() - .zip(self.adjusted_request_ordering.clone().into_iter()) - .map(|(child, required)| { - let from_parent = required; + + izip!( + plan_children.into_iter(), + self.adjusted_request_ordering.clone().into_iter(), + self.plan.maintains_input_order().into_iter(), + self.plan.required_input_distribution().into_iter(), + ) + .map( + |(child, from_parent, maintains_input_order, required_dist)| { + let child_satisfy_single_distribution = + matches!(required_dist, Distribution::SinglePartition); + let child_impact_result_ordering = if self + .plan + .as_any() + .downcast_ref::() + .is_some() + || self + .plan + .as_any() + .downcast_ref::() + .is_some() + { + true + } else { + maintains_input_order && self.impact_result_ordering + }; let child_request_ordering = child.required_input_ordering(); PlanWithSortRequirements { plan: child, impact_result_ordering: child_impact_result_ordering, + satisfy_single_distribution: child_satisfy_single_distribution, required_ordering: from_parent, adjusted_request_ordering: child_request_ordering, } - }) - .collect() + }, + ) + .collect() } } @@ -185,6 +190,7 @@ impl TreeNodeRewritable for PlanWithSortRequirements { Ok(PlanWithSortRequirements { plan, impact_result_ordering: self.impact_result_ordering, + satisfy_single_distribution: self.satisfy_single_distribution, required_ordering: self.required_ordering, adjusted_request_ordering: self.adjusted_request_ordering, }) @@ -242,40 +248,10 @@ impl PhysicalOptimizerRule for TopDownEnforceSorting { fn ensure_sorting( requirements: PlanWithSortRequirements, ) -> Result> { - println!( - "=== Current plan ===\n{}\n", - displayable(requirements.plan.as_ref()).indent() - ); - println!( - "impact_result_ordering: {:?}, parent required_ordering {:?}, adjusted request ordering {:?}", - requirements.impact_result_ordering, requirements.required_ordering, requirements.adjusted_request_ordering, - ); if let Some(sort_exec) = requirements.plan.as_any().downcast_ref::() { - // Remove unnecessary global SortExec - if !sort_exec.preserve_partitioning() { - if !requirements.impact_result_ordering - && requirements.required_ordering.is_none() - { - println!("remove sort_exec due to no need to keep ordering"); - return Ok(Some(PlanWithSortRequirements { - plan: Arc::new(TombStoneExec::new(sort_exec.input().clone())), - impact_result_ordering: false, - required_ordering: None, - adjusted_request_ordering: vec![None], - })); - } else if ordering_satisfy( - sort_exec.input().output_ordering(), - sort_exec.output_ordering(), - || sort_exec.input().equivalence_properties(), - ) { - println!("remove sort_exec due to child already satisfy"); - return Ok(Some(PlanWithSortRequirements { - plan: Arc::new(TombStoneExec::new(sort_exec.input().clone())), - impact_result_ordering: true, - required_ordering: None, - adjusted_request_ordering: vec![requirements.required_ordering], - })); - } + // Remove unnecessary SortExec(local/global) + if let Some(result) = analyze_immediate_sort_removal(&requirements, sort_exec) { + return Ok(Some(result)); } } else if let Some(sort_pres_exec) = requirements .plan @@ -288,96 +264,55 @@ fn ensure_sorting( sort_pres_exec.input().as_any().downcast_ref::() { if sort_pres_exec.expr() == child_sort_exec.expr() { - if !requirements.impact_result_ordering - && requirements.required_ordering.is_none() + if let Some(result) = + analyze_immediate_sort_removal(&requirements, child_sort_exec) { - println!("remove SortPreservingMergeExec + SortExec due to no need to keep ordering"); - return Ok(Some(PlanWithSortRequirements { - plan: Arc::new(TombStoneExec::new( - child_sort_exec.input().clone(), - )), - impact_result_ordering: false, - required_ordering: None, - adjusted_request_ordering: vec![None], - })); - } else if ordering_satisfy( - child_sort_exec.input().output_ordering(), - child_sort_exec.output_ordering(), - || child_sort_exec.input().equivalence_properties(), - ) && child_sort_exec - .input() - .output_partitioning() - .partition_count() - == 1 - { - println!("remove SortPreservingMergeExec + SortExec due to child already satisfy"); - return Ok(Some(PlanWithSortRequirements { - plan: Arc::new(TombStoneExec::new( - child_sort_exec.input().clone(), - )), - impact_result_ordering: true, - required_ordering: None, - adjusted_request_ordering: vec![requirements.required_ordering], - })); + return Ok(Some(result)); } } - } else { - if !sort_pres_exec.satisfy_distribution() { - // Remove unnecessary SortPreservingMergeExec only - if !requirements.impact_result_ordering { - println!( - "remove SortPreservingMergeExec due to no need to keep ordering" - ); - return Ok(Some(PlanWithSortRequirements { - plan: Arc::new(TombStoneExec::new( - sort_pres_exec.input().clone(), - )), - impact_result_ordering: false, - required_ordering: None, - adjusted_request_ordering: vec![requirements.required_ordering], - })); - } else if ordering_satisfy( - sort_pres_exec.input().output_ordering(), - Some(sort_pres_exec.expr()), - || sort_pres_exec.input().equivalence_properties(), - ) && sort_pres_exec - .input() - .output_partitioning() - .partition_count() - == 1 - { - println!( - "remove SortPreservingMergeExec due to child already satisfy" - ); - return Ok(Some(PlanWithSortRequirements { - plan: Arc::new(TombStoneExec::new( - sort_pres_exec.input().clone(), - )), - impact_result_ordering: true, - required_ordering: None, - adjusted_request_ordering: vec![requirements.required_ordering], - })); - } + } else if !requirements.satisfy_single_distribution + || sort_pres_exec + .input() + .output_partitioning() + .partition_count() + <= 1 + { + if let Some(result) = + analyze_immediate_spm_removal(&requirements, sort_pres_exec) + { + return Ok(Some(result)); } } } - println!("no removing"); let plan = &requirements.plan; let parent_required = requirements.required_ordering.as_deref(); if ordering_satisfy_requirement(plan.output_ordering(), parent_required, || { plan.equivalence_properties() }) { - // Can satisfy the parent requirements, clear the requirements - println!( - "Can satisfy the parent requirements, impact_result_ordering {:?}", - requirements.impact_result_ordering - ); - if plan.as_any().downcast_ref::().is_some() + // Can satisfy the parent requirements, change the adjusted_request_ordering for UnionExec and WindowAggExec(BoundedWindowAggExec) + if let Some(union_exec) = plan.as_any().downcast_ref::() { + // 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 adjusted = new_sort_requirements(union_exec.output_ordering()); + return Ok(Some(PlanWithSortRequirements { + plan: plan.clone(), + impact_result_ordering: requirements.impact_result_ordering, + satisfy_single_distribution: requirements.satisfy_single_distribution, + required_ordering: None, + adjusted_request_ordering: vec![ + adjusted; + requirements + .adjusted_request_ordering + .len() + ], + })); + } else if plan.as_any().downcast_ref::().is_some() || plan .as_any() .downcast_ref::() .is_some() { + // WindowAggExec(BoundedWindowAggExec) might reverse their sort requirements let request_child = requirements.adjusted_request_ordering[0].as_deref(); let reversed_request_child = reverse_window_sort_requirements(request_child); @@ -386,27 +321,12 @@ fn ensure_sorting( request_child, reversed_request_child.as_deref(), ) { - println!("Should reverse top window sort_requirements"); - let (window_expr, input_schema, partition_keys) = if let Some(exec) = - plan.as_any().downcast_ref::() - { - ( - exec.window_expr(), - exec.input_schema(), - exec.partition_keys.clone(), - ) - } else if let Some(exec) = plan.as_any().downcast_ref::() { - ( - exec.window_expr(), - exec.input_schema(), - exec.partition_keys.clone(), - ) - } else { - return Err(DataFusionError::Plan( - "Expects to receive either WindowAggExec of BoundedWindowAggExec" - .to_string(), - )); - }; + let WindowExecInfo { + window_expr, + input_schema, + partition_keys, + } = extract_window_info_from_plan(plan).unwrap(); + let new_window_expr = window_expr .iter() .map(|e| e.get_reverse_expr()) @@ -426,7 +346,7 @@ fn ensure_sorting( input_schema, partition_keys, Some(new_physical_ordering), - )?) as _ + )?) as Arc } else { Arc::new(WindowAggExec::try_new( window_expr, @@ -434,37 +354,28 @@ fn ensure_sorting( input_schema, partition_keys, Some(new_physical_ordering), - )?) as _ + )?) as Arc }; - println!("Reverse WindowAggExec expressions and push down the reversed requirements"); - return Ok(Some(PlanWithSortRequirements { plan: new_plan, impact_result_ordering: false, + satisfy_single_distribution: requirements + .satisfy_single_distribution, required_ordering: None, adjusted_request_ordering: vec![reversed_request_child], })); } - } else { - println!("Should not reverse top window sort_requirements"); } - } else if let Some(_) = plan.as_any().downcast_ref::() { - return Ok(Some(PlanWithSortRequirements { - plan: plan.clone(), - impact_result_ordering: false, - required_ordering: None, - adjusted_request_ordering: requirements.adjusted_request_ordering.clone(), - })); } - return Ok(Some(PlanWithSortRequirements { + Ok(Some(PlanWithSortRequirements { plan: plan.clone(), impact_result_ordering: requirements.impact_result_ordering, + satisfy_single_distribution: requirements.satisfy_single_distribution, required_ordering: None, adjusted_request_ordering: requirements.adjusted_request_ordering, - })); + })) } else if let Some(sort_exec) = plan.as_any().downcast_ref::() { - println!("Modify current SortExec to satisfy the parent requirements"); - // If the current plan is a SortExec, update the SortExec to satisfy the parent requirements + // If the current plan is a SortExec, modify current SortExec to satisfy the parent requirements let parent_required_expr = create_sort_expr_from_requirement(parent_required.unwrap()); let new_plan = add_sort_above_child( @@ -472,12 +383,11 @@ fn ensure_sorting( parent_required_expr, sort_exec.fetch(), )?; - return Ok(Some( + Ok(Some( PlanWithSortRequirements::new_without_impact_result_ordering(new_plan), - )); + )) } else { - println!("Can not satisfy the parent requirements, try to push down"); - // Can not satisfy the parent requirements, check whether should push down the requirements. Add new SortExec when the parent requirements can not be pushed down + // Can not satisfy the parent requirements, check whether the requirements can be pushed down. If not, add new SortExec. let parent_required_expr = create_sort_expr_from_requirement(parent_required.unwrap()); let maintains_input_order = plan.maintains_input_order(); @@ -493,158 +403,88 @@ fn ensure_sorting( && plan.as_any().downcast_ref::().is_none() { let new_plan = add_sort_above_child(plan, parent_required_expr, None)?; - return Ok(Some( + Ok(Some( PlanWithSortRequirements::new_without_impact_result_ordering(new_plan), - )); - } else if let Some(window_agg_exec) = - plan.as_any().downcast_ref::() + )) + } else if plan.as_any().downcast_ref::().is_some() + || plan + .as_any() + .downcast_ref::() + .is_some() { - let window_expr = window_agg_exec.window_expr(); let request_child = requirements.adjusted_request_ordering[0].as_deref(); if requirements_compatible(request_child, parent_required, || { plan.children()[0].equivalence_properties() }) { - println!("WindowAggExec child requirements are more specific, no need to add SortExec"); - return Ok(Some(PlanWithSortRequirements { - plan: plan.clone(), - impact_result_ordering: true, - required_ordering: None, - adjusted_request_ordering: requirements.adjusted_request_ordering, - })); + // request child requirements are more specific, no need to push down the parent requirements + Ok(None) } else if requirements_compatible(parent_required, request_child, || { plan.children()[0].equivalence_properties() }) { - println!("Parent requirements are more specific, adjust WindowAggExec child requirements and push down the requirements"); + // parent requirements are more specific, adjust the request child requirements and push down the new requirements let adjusted = parent_required.map(|r| r.to_vec()); - return Ok(Some(PlanWithSortRequirements { + Ok(Some(PlanWithSortRequirements { plan: plan.clone(), impact_result_ordering: true, + satisfy_single_distribution: requirements.satisfy_single_distribution, required_ordering: None, adjusted_request_ordering: vec![adjusted], - })); + })) } else { - let should_reverse = can_reverse_window_request( - window_expr[0].partition_by(), + let WindowExecInfo { + window_expr, + input_schema, + partition_keys, + } = extract_window_info_from_plan(plan).unwrap(); + if should_reverse_window_exec( parent_required, request_child, - &window_agg_exec.input().schema(), - ); - if should_reverse { - let new_window_expr = window_expr - .iter() - .map(|e| e.get_reverse_expr()) - .collect::>>(); - if let Some(window_expr) = new_window_expr { - let new_plan = Arc::new(WindowAggExec::try_new( - window_expr, - window_agg_exec.children()[0].clone(), - window_agg_exec.input_schema(), - window_agg_exec.partition_keys.clone(), - Some(parent_required_expr.to_vec()), - )?) as _; - println!("Reverse WindowAggExec expressions and push down the requirements"); - return Ok(Some( - PlanWithSortRequirements::new_without_impact_result_ordering( - new_plan, - ), - )); - } else { - println!("Can not push down, add new SortExec"); - let new_plan = - add_sort_above_child(plan, parent_required_expr, None)?; - return Ok(Some( - PlanWithSortRequirements::new_without_impact_result_ordering( - new_plan, - ), - )); - } - } else { - // Can not push down, add new SortExec - println!("Can not push down, add new SortExec"); - let new_plan = - add_sort_above_child(plan, parent_required_expr, None)?; - return Ok(Some( - PlanWithSortRequirements::new_without_impact_result_ordering( - new_plan, - ), - )); - } - } - } else if let Some(window_agg_exec) = - plan.as_any().downcast_ref::() - { - let window_expr = window_agg_exec.window_expr(); - let request_child = &plan.required_input_ordering()[0]; - if requirements_compatible(request_child.as_deref(), parent_required, || { - plan.children()[0].equivalence_properties() - }) { - println!("BoundedWindowAggExec child requirements are more specific, no need to add SortExec"); - return Ok(Some(PlanWithSortRequirements { - plan: plan.clone(), - impact_result_ordering: true, - required_ordering: None, - adjusted_request_ordering: requirements.adjusted_request_ordering, - })); - } else if requirements_compatible( - parent_required, - request_child.as_deref(), - || plan.children()[0].equivalence_properties(), - ) { - println!("Parent requirements are more specific, adjust BoundedWindowAggExec child requirements and push down the requirements"); - let adjusted = parent_required.map(|r| r.to_vec()); - return Ok(Some(PlanWithSortRequirements { - plan: plan.clone(), - impact_result_ordering: true, - required_ordering: None, - adjusted_request_ordering: vec![adjusted], - })); - } else { - let should_reverse = can_reverse_window_request( - window_expr[0].partition_by(), - parent_required, - request_child.as_deref(), - &window_agg_exec.input().schema(), - ); - if should_reverse { + &input_schema, + ) { + let new_physical_ordering = parent_required_expr.to_vec(); let new_window_expr = window_expr .iter() .map(|e| e.get_reverse_expr()) .collect::>>(); if let Some(window_expr) = new_window_expr { - let new_plan = Arc::new(BoundedWindowAggExec::try_new( - window_expr, - window_agg_exec.children()[0].clone(), - window_agg_exec.input_schema(), - window_agg_exec.partition_keys.clone(), - Some(parent_required_expr.to_vec()), - )?) as _; - println!("Reverse BoundedWindowAggExec expressions and push down the requirements"); - return Ok(Some( - PlanWithSortRequirements::new_without_impact_result_ordering( - new_plan, - ), - )); - } else { - println!("Can not push down, add new SortExec"); - let new_plan = - add_sort_above_child(plan, parent_required_expr, None)?; - return Ok(Some( - PlanWithSortRequirements::new_without_impact_result_ordering( - new_plan, - ), - )); + let uses_bounded_memory = + window_expr.iter().all(|e| e.uses_bounded_memory()); + let new_plan = if uses_bounded_memory { + Arc::new(BoundedWindowAggExec::try_new( + window_expr, + plan.children()[0].clone(), + input_schema, + partition_keys, + Some(new_physical_ordering), + )?) as Arc + } else { + Arc::new(WindowAggExec::try_new( + window_expr, + plan.children()[0].clone(), + input_schema, + partition_keys, + Some(new_physical_ordering), + )?) as Arc + }; + let adjusted_request_ordering = + new_plan.required_input_ordering(); + return Ok(Some(PlanWithSortRequirements { + plan: new_plan, + impact_result_ordering: false, + satisfy_single_distribution: requirements + .satisfy_single_distribution, + required_ordering: None, + adjusted_request_ordering, + })); } - } else { - // Can not push down, add new SortExec - println!("Can not push down, add new SortExec"); - let new_plan = - add_sort_above_child(plan, parent_required_expr, None)?; - return Ok(Some( - PlanWithSortRequirements::new_without_impact_result_ordering( - new_plan, - ), - )); } + // Can not push down requirements, add new SortExec + let new_plan = add_sort_above_child(plan, parent_required_expr, None)?; + Ok(Some( + PlanWithSortRequirements::new_without_impact_result_ordering( + new_plan, + ), + )) } } else if let Some(smj) = plan.as_any().downcast_ref::() { // If the current plan is SortMergeJoinExec @@ -659,23 +499,25 @@ fn ensure_sorting( || plan.children()[0].equivalence_properties(), ) { println!("Requirements are compatible with SMJ"); - return Ok(Some(PlanWithSortRequirements { + Ok(Some(PlanWithSortRequirements { plan: plan.clone(), impact_result_ordering: true, + satisfy_single_distribution: requirements + .satisfy_single_distribution, required_ordering: None, adjusted_request_ordering: requirements .adjusted_request_ordering, - })); + })) } else { // Can not push down, add new SortExec println!("Can not push down, add new SortExec"); let new_plan = add_sort_above_child(plan, parent_required_expr, None)?; - return Ok(Some( + Ok(Some( PlanWithSortRequirements::new_without_impact_result_ordering( new_plan, ), - )); + )) } } Some(JoinSide::Right) if maintains_input_order[1] => { @@ -687,34 +529,36 @@ fn ensure_sorting( || plan.children()[1].equivalence_properties(), ) { println!("Requirements are compatible with SMJ"); - return Ok(Some(PlanWithSortRequirements { + Ok(Some(PlanWithSortRequirements { plan: plan.clone(), impact_result_ordering: true, + satisfy_single_distribution: requirements + .satisfy_single_distribution, required_ordering: None, adjusted_request_ordering: requirements .adjusted_request_ordering, - })); + })) } else { // Can not push down, add new SortExec println!("Can not push down, add new SortExec"); let new_plan = add_sort_above_child(plan, parent_required_expr, None)?; - return Ok(Some( + Ok(Some( PlanWithSortRequirements::new_without_impact_result_ordering( new_plan, ), - )); + )) } } _ => { println!("Can not decide the expr side for SortMergeJoinExec, can not push down, add SortExec"); let new_plan = add_sort_above_child(plan, parent_required_expr, None)?; - return Ok(Some( + Ok(Some( PlanWithSortRequirements::new_without_impact_result_ordering( new_plan, ), - )); + )) } } } else if plan.required_input_ordering().iter().any(Option::is_some) { @@ -740,20 +584,15 @@ fn ensure_sorting( ) }) .collect::>(); - println!( - "plan.equivalence_properties() {:?}", - plan.equivalence_properties() - ); - println!("compatible_with_children {:?}", compatible_with_children); if compatible_with_children.iter().all(|a| *a) { // Requirements are compatible, not need to push down. - println!("Requirements are compatible, no need to push down"); - return Ok(Some(PlanWithSortRequirements { + Ok(Some(PlanWithSortRequirements { plan: plan.clone(), impact_result_ordering: true, + satisfy_single_distribution: requirements.satisfy_single_distribution, required_ordering: None, adjusted_request_ordering: requirements.adjusted_request_ordering, - })); + })) } else { let can_adjust_child_requirements = plan .required_input_ordering() @@ -771,26 +610,28 @@ fn ensure_sorting( // Adjust child requirements and push down the requirements println!("Adjust child requirements and push down the requirements"); let adjusted = parent_required.map(|r| r.to_vec()); - return Ok(Some(PlanWithSortRequirements { + Ok(Some(PlanWithSortRequirements { plan: plan.clone(), impact_result_ordering: true, + satisfy_single_distribution: requirements + .satisfy_single_distribution, required_ordering: None, adjusted_request_ordering: vec![ adjusted; can_adjust_child_requirements .len() ], - })); + })) } else { // Can not push down, add new SortExec println!("Can not push down, add new SortExec"); let new_plan = add_sort_above_child(plan, parent_required_expr, None)?; - return Ok(Some( + Ok(Some( PlanWithSortRequirements::new_without_impact_result_ordering( new_plan, ), - )); + )) } } } else { @@ -807,6 +648,8 @@ fn ensure_sorting( Ok(Some(PlanWithSortRequirements { plan: plan.clone(), impact_result_ordering: true, + satisfy_single_distribution: requirements + .satisfy_single_distribution, required_ordering: None, adjusted_request_ordering: vec![new_requirement], })) @@ -817,30 +660,113 @@ fn ensure_sorting( ); let new_plan = add_sort_above_child(plan, parent_required_expr, None)?; - return Ok(Some( + Ok(Some( PlanWithSortRequirements::new_without_impact_result_ordering( new_plan, ), - )); + )) } } else { println!("Push down requirements."); - return Ok(Some(PlanWithSortRequirements { + Ok(Some(PlanWithSortRequirements { plan: plan.clone(), impact_result_ordering: requirements.impact_result_ordering, required_ordering: None, + satisfy_single_distribution: requirements.satisfy_single_distribution, adjusted_request_ordering: vec![ requirements.required_ordering; requirements .adjusted_request_ordering .len() ], - })); + })) } } } } +/// Analyzes a given `Sort` (`plan`) to determine whether the Sort can be removed: +/// 1) The input already has a finer ordering than this `Sort` enforces. +/// 2) The `Sort` does not impact the final result ordering. +fn analyze_immediate_sort_removal( + requirements: &PlanWithSortRequirements, + sort_exec: &SortExec, +) -> Option { + if ordering_satisfy( + sort_exec.input().output_ordering(), + sort_exec.output_ordering(), + || sort_exec.input().equivalence_properties(), + ) { + Some(PlanWithSortRequirements { + plan: Arc::new(TombStoneExec::new(sort_exec.input().clone())), + impact_result_ordering: requirements.impact_result_ordering, + satisfy_single_distribution: requirements.satisfy_single_distribution, + required_ordering: None, + adjusted_request_ordering: vec![requirements.required_ordering.clone()], + }) + } + // Remove unnecessary SortExec + else if !requirements.impact_result_ordering { + if requirements.satisfy_single_distribution + && !sort_exec.preserve_partitioning() + && sort_exec.input().output_partitioning().partition_count() > 1 + { + Some(PlanWithSortRequirements { + plan: Arc::new(CoalescePartitionsExec::new(sort_exec.input().clone())), + impact_result_ordering: false, + satisfy_single_distribution: false, + required_ordering: None, + adjusted_request_ordering: vec![requirements.required_ordering.clone()], + }) + } else { + Some(PlanWithSortRequirements { + plan: Arc::new(TombStoneExec::new(sort_exec.input().clone())), + impact_result_ordering: false, + satisfy_single_distribution: false, + required_ordering: None, + adjusted_request_ordering: vec![requirements.required_ordering.clone()], + }) + } + } else { + None + } +} + +/// Analyzes a given `SortPreservingMergeExec` (`plan`) to determine whether the SortPreservingMergeExec can be removed: +/// 1) The input already has a finer ordering than this `SortPreservingMergeExec` enforces. +/// 2) The `SortPreservingMergeExec` does not impact the final result ordering. +fn analyze_immediate_spm_removal( + requirements: &PlanWithSortRequirements, + spm_exec: &SortPreservingMergeExec, +) -> Option { + if ordering_satisfy( + spm_exec.input().output_ordering(), + Some(spm_exec.expr()), + || spm_exec.input().equivalence_properties(), + ) && spm_exec.input().output_partitioning().partition_count() <= 1 + { + Some(PlanWithSortRequirements { + plan: Arc::new(TombStoneExec::new(spm_exec.input().clone())), + impact_result_ordering: true, + satisfy_single_distribution: false, + required_ordering: None, + adjusted_request_ordering: vec![requirements.required_ordering.clone()], + }) + } + // Remove unnecessary SortPreservingMergeExec only + else if !requirements.impact_result_ordering { + Some(PlanWithSortRequirements { + plan: Arc::new(TombStoneExec::new(spm_exec.input().clone())), + impact_result_ordering: false, + satisfy_single_distribution: false, + required_ordering: None, + adjusted_request_ordering: vec![requirements.required_ordering.clone()], + }) + } else { + None + } +} + fn expr_source_sides( required_exprs: &[PhysicalSortExpr], left_columns_len: usize, @@ -892,7 +818,7 @@ fn shift_right_required( col.name(), col.index() - left_columns_len, )) as Arc, - sort_options: r.sort_options.clone(), + sort_options: r.sort_options, }) } else { None @@ -911,51 +837,6 @@ fn shift_right_required( } } -#[derive(Debug)] -/// This structure stores extra column information required to remove unnecessary sorts. -pub struct ColumnInfo { - reverse: bool, - is_partition: bool, -} - -fn can_reverse_window_request( - partition_keys: &[Arc], - required: Option<&[PhysicalSortRequirements]>, - request_ordering: Option<&[PhysicalSortRequirements]>, - input_schema: &SchemaRef, -) -> bool { - match (required, request_ordering) { - (_, None) => false, - (None, Some(_)) => false, - (Some(required), Some(request_ordering)) => { - if required.len() > request_ordering.len() { - return false; - } - let mut col_infos = vec![]; - for (required_expr, request_expr) in zip(required, request_ordering) { - let column = required_expr.expr.clone(); - let is_partition = partition_keys.iter().any(|e| e.eq(&column)); - let reverse = check_alignment(input_schema, request_expr, required_expr); - col_infos.push(ColumnInfo { - reverse, - is_partition, - }); - } - let order_by_sections = col_infos - .iter() - .filter(|elem| !elem.is_partition) - .collect::>(); - let should_reverse_order_bys = if order_by_sections.is_empty() { - false - } else { - let first_reverse = order_by_sections[0].reverse; - first_reverse - }; - should_reverse_order_bys - } - } -} - /// Compares window expression's `window_request` and `parent_required_expr` ordering, returns /// whether we should reverse the window expression's ordering in order to meet parent's requirements. fn check_alignment( @@ -970,13 +851,12 @@ fn check_alignment( let nullable = parent_required_expr.expr.nullable(input_schema).unwrap(); let window_request_opts = window_request.sort_options.unwrap(); let parent_required_opts = parent_required_expr.sort_options.unwrap(); - let is_reversed = if nullable { + if nullable { window_request_opts == reverse_sort_options(parent_required_opts) } else { // If the column is not nullable, NULLS FIRST/LAST is not important. window_request_opts.descending != parent_required_opts.descending - }; - is_reversed + } } else { false } @@ -985,7 +865,7 @@ fn check_alignment( fn reverse_window_sort_requirements( request_child: Option<&[PhysicalSortRequirements]>, ) -> Option> { - let reversed_request = request_child.map(|request| { + request_child.map(|request| { request .iter() .map(|req| match req.sort_options { @@ -996,10 +876,12 @@ fn reverse_window_sort_requirements( }, }) .collect::>() - }); - reversed_request + }) } +/// Whether to reverse the top WindowExec's sort requirements. +/// Considering the requirements of the descendants WindowExecs and leaf nodes' output ordering. +/// TODO!considering all the cases fn should_reverse_window_sort_requirements( window_plan: Arc, top_requirement: Option<&[PhysicalSortRequirements]>, @@ -1059,7 +941,7 @@ fn should_reverse_window_sort_requirements( top_reversed_requirement, ) } else { - if requirements_compatible( + requirements_compatible( top_reversed_requirement, window_plan.required_input_ordering()[0].as_deref(), || window_plan.equivalence_properties(), @@ -1067,24 +949,18 @@ fn should_reverse_window_sort_requirements( window_plan.required_input_ordering()[0].as_deref(), top_reversed_requirement, || window_plan.equivalence_properties(), - ) { - true - } else { - false - } + ) } - } else if requirements_compatible( - top_reversed_requirement, - window_plan.required_input_ordering()[0].as_deref(), - || window_plan.equivalence_properties(), - ) || requirements_compatible( - window_plan.required_input_ordering()[0].as_deref(), - top_reversed_requirement, - || window_plan.equivalence_properties(), - ) { - true } else { - false + requirements_compatible( + top_reversed_requirement, + window_plan.required_input_ordering()[0].as_deref(), + || window_plan.equivalence_properties(), + ) || requirements_compatible( + window_plan.required_input_ordering()[0].as_deref(), + top_reversed_requirement, + || window_plan.equivalence_properties(), + ) } }) .collect::>(); @@ -1092,6 +968,71 @@ fn should_reverse_window_sort_requirements( flags.iter().all(|o| *o) } +fn should_reverse_window_exec( + required: Option<&[PhysicalSortRequirements]>, + request_ordering: Option<&[PhysicalSortRequirements]>, + input_schema: &SchemaRef, +) -> bool { + match (required, request_ordering) { + (_, None) => false, + (None, Some(_)) => false, + (Some(required), Some(request_ordering)) => { + if required.len() > request_ordering.len() { + return false; + } + let alignment_flags = required + .iter() + .zip(request_ordering.iter()) + .filter_map(|(required_expr, request_expr)| { + // Only check the alignment of non-partition columns + if request_expr.sort_options.is_some() + && required_expr.sort_options.is_some() + { + Some(check_alignment(input_schema, request_expr, required_expr)) + } else if request_expr.expr.eq(&required_expr.expr) { + None + } else { + Some(false) + } + }) + .collect::>(); + if alignment_flags.is_empty() { + false + } else { + alignment_flags.iter().all(|o| *o) + } + } + } +} + +fn extract_window_info_from_plan( + plan: &Arc, +) -> Option { + if let Some(exec) = plan.as_any().downcast_ref::() { + Some(WindowExecInfo { + window_expr: exec.window_expr().to_vec(), + input_schema: exec.input_schema(), + partition_keys: exec.partition_keys.clone(), + }) + } else { + plan.as_any() + .downcast_ref::() + .map(|exec| WindowExecInfo { + window_expr: exec.window_expr().to_vec(), + input_schema: exec.input_schema(), + partition_keys: exec.partition_keys.clone(), + }) + } +} + +#[derive(Debug)] +/// This structure stores extra Window information required to create a new WindowExec +pub struct WindowExecInfo { + window_expr: Vec>, + input_schema: SchemaRef, + partition_keys: Vec>, +} + /// A TombStoneExec execution plan generated during optimization process, should be removed finally #[derive(Debug)] struct TombStoneExec { @@ -1146,9 +1087,9 @@ impl ExecutionPlan for TombStoneExec { _partition: usize, _context: Arc, ) -> Result { - Err(DataFusionError::Internal(format!( - "TombStoneExec, invalid plan" - ))) + Err(DataFusionError::Internal( + "TombStoneExec, invalid plan".to_string(), + )) } fn fmt_as( @@ -1190,6 +1131,7 @@ mod tests { use datafusion_expr::{AggregateFunction, WindowFrame, WindowFunction}; use datafusion_physical_expr::expressions::{col, NotExpr}; use datafusion_physical_expr::PhysicalSortExpr; + use std::ops::Deref; use std::sync::Arc; fn create_test_schema() -> Result { @@ -1286,6 +1228,126 @@ mod tests { Ok(()) } + #[tokio::test] + async fn test_should_reverse_window() -> Result<()> { + let schema = create_test_schema()?; + + // partition by nullable_col order by non_nullable_col + let window_request_ordering1 = vec![ + PhysicalSortRequirements { + expr: col("nullable_col", &schema)?, + sort_options: None, + }, + PhysicalSortRequirements { + expr: col("non_nullable_col", &schema)?, + sort_options: Some(SortOptions { + descending: true, + nulls_first: true, + }), + }, + ]; + let required_ordering1 = vec![ + PhysicalSortRequirements { + expr: col("nullable_col", &schema)?, + sort_options: None, + }, + PhysicalSortRequirements { + expr: col("non_nullable_col", &schema)?, + sort_options: Some(SortOptions { + descending: false, + nulls_first: false, + }), + }, + ]; + + let reverse = should_reverse_window_exec( + Some(required_ordering1.deref()), + Some(window_request_ordering1.deref()), + &schema, + ); + assert!(reverse); + + // order by nullable_col, non_nullable_col + let window_request_ordering2 = vec![ + PhysicalSortRequirements { + expr: col("nullable_col", &schema)?, + sort_options: Some(SortOptions { + descending: true, + nulls_first: true, + }), + }, + PhysicalSortRequirements { + expr: col("non_nullable_col", &schema)?, + sort_options: Some(SortOptions { + descending: true, + nulls_first: true, + }), + }, + ]; + + let required_ordering2 = vec![ + PhysicalSortRequirements { + expr: col("nullable_col", &schema)?, + sort_options: None, + }, + PhysicalSortRequirements { + expr: col("non_nullable_col", &schema)?, + sort_options: Some(SortOptions { + descending: false, + nulls_first: false, + }), + }, + ]; + + let reverse = should_reverse_window_exec( + Some(required_ordering2.deref()), + Some(window_request_ordering2.deref()), + &schema, + ); + assert!(reverse); + + // wrong partition columns + let window_request_ordering3 = vec![ + PhysicalSortRequirements { + expr: col("nullable_col", &schema)?, + sort_options: Some(SortOptions { + descending: true, + nulls_first: true, + }), + }, + PhysicalSortRequirements { + expr: col("non_nullable_col", &schema)?, + sort_options: Some(SortOptions { + descending: true, + nulls_first: true, + }), + }, + ]; + + let required_ordering3 = vec![ + PhysicalSortRequirements { + expr: col("non_nullable_col", &schema)?, + sort_options: None, + }, + PhysicalSortRequirements { + expr: col("non_nullable_col", &schema)?, + sort_options: Some(SortOptions { + descending: false, + nulls_first: false, + }), + }, + ]; + + let reverse = should_reverse_window_exec( + Some(required_ordering3.deref()), + Some(window_request_ordering3.deref()), + &schema, + ); + assert!(!reverse); + + Ok(()) + } + /// Runs the sort enforcement optimizer and asserts the plan /// against the original and expected plans /// @@ -1347,7 +1409,7 @@ mod tests { } #[tokio::test] - async fn test_remove_unnecessary_sort_window_multilayer() -> Result<()> { + async fn test_not_remove_top_sort_window_multilayer() -> Result<()> { let schema = create_test_schema()?; let source = memory_exec(&schema); @@ -1374,7 +1436,7 @@ mod tests { 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 + // Add dummy layer propagating Sort above, the top Sort should not be removed let filter = filter_exec( Arc::new(NotExpr::new( col("non_nullable_col", schema.as_ref()).unwrap(), @@ -1395,11 +1457,12 @@ mod tests { ]; let expected_optimized = vec![ - "WindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(NULL) }]", + "WindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow }]", " FilterExec: NOT non_nullable_col@1", - " WindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow }]", - " SortExec: [non_nullable_col@1 DESC]", - " MemoryExec: partitions=0, partition_sizes=[]", + " SortExec: [non_nullable_col@1 ASC NULLS LAST], global=true", + " WindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow }]", + " SortExec: [non_nullable_col@1 DESC], global=true", + " MemoryExec: partitions=0, partition_sizes=[]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan); Ok(()) @@ -1855,8 +1918,7 @@ mod tests { // 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 required ordering - // of SortPreservingMergeExec. Hence rule should remove unnecessary sort for second child of the UnionExec - // and put a sort above Union to satisfy required ordering. + // of SortPreservingMergeExec. let expected_input = vec![ "SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", " UnionExec", @@ -2078,32 +2140,20 @@ mod tests { 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 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 doesn't preserve any of the inputs ordering. However, we should be able to change unnecessarily fine - // SortExecs under UnionExec with required SortExecs that are absolutely necessary. + // Union preserves the inputs ordering and we should not change any of the SortExecs under UnionExec let expected_input = vec![ "SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", " SortExec: [nullable_col@0 ASC,non_nullable_col@1 ASC], global=true", " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " SortExec: [nullable_col@0 ASC,non_nullable_col@1 DESC NULLS LAST], global=true", + " SortExec: [nullable_col@0 ASC,non_nullable_col@1 ASC], global=true", " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", ]; assert_optimized!(expected_input, expected_input, physical_plan); @@ -2137,9 +2187,47 @@ mod tests { let physical_plan = window_exec("nullable_col", reversed_sort_exprs2, union); // 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. + // The SortExecs should be kept to ensure the final result ordering + let expected_input = vec![ + "WindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow }]", + " UnionExec", + " SortExec: [nullable_col@0 DESC NULLS LAST], global=true", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, output_ordering=[nullable_col@0 ASC, non_nullable_col@1 ASC], projection=[nullable_col, non_nullable_col]", + " SortExec: [nullable_col@0 DESC NULLS LAST], global=true", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, output_ordering=[nullable_col@0 ASC], projection=[nullable_col, non_nullable_col]", + ]; + assert_optimized!(expected_input, expected_input, physical_plan); + Ok(()) + } + + #[tokio::test] + async fn test_window_multi_path_sort2() -> 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 reversed_sort_exprs2 = 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.clone()); + let sort1 = sort_exec(reversed_sort_exprs2.clone(), source1); + let sort2 = sort_exec(reversed_sort_exprs2, source2); + + let union = union_exec(vec![sort1, sort2]); + let physical_plan = window_exec("nullable_col", sort_exprs2, union); + + // The `WindowAggExec` gets its sorting from multiple children jointly. + // The SortExecs should be kept to ensure the final result ordering let expected_input = vec![ "WindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow }]", " UnionExec", @@ -2149,7 +2237,7 @@ mod tests { " ParquetExec: limit=None, partitions={1 group: [[x]]}, output_ordering=[nullable_col@0 ASC], projection=[nullable_col, non_nullable_col]", ]; let expected_optimized = vec![ - "WindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: CurrentRow, end_bound: Following(NULL) }]", + "WindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow }]", " UnionExec", " ParquetExec: limit=None, partitions={1 group: [[x]]}, output_ordering=[nullable_col@0 ASC, non_nullable_col@1 ASC], projection=[nullable_col, non_nullable_col]", " ParquetExec: limit=None, partitions={1 group: [[x]]}, output_ordering=[nullable_col@0 ASC], projection=[nullable_col, non_nullable_col]", diff --git a/datafusion/core/src/physical_plan/planner.rs b/datafusion/core/src/physical_plan/planner.rs index 2214d0168ed87..b6269a560386c 100644 --- a/datafusion/core/src/physical_plan/planner.rs +++ b/datafusion/core/src/physical_plan/planner.rs @@ -1900,7 +1900,6 @@ mod tests { let session_state = make_session_state(); // optimize the logical plan let logical_plan = session_state.optimize(logical_plan)?; - println!("optimized logical plan {:?}", logical_plan); let planner = DefaultPhysicalPlanner::default(); planner .create_physical_plan(&logical_plan, &session_state) diff --git a/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs b/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs index 9af412a9585bb..2d7018f38a335 100644 --- a/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs +++ b/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs @@ -85,8 +85,6 @@ pub struct SortPreservingMergeExec { expr: Vec, /// Execution metrics metrics: ExecutionPlanMetricsSet, - /// use SortPreservingMergeExec to satisfy the Sort Distribution - satisfy_distribution: bool, } impl SortPreservingMergeExec { @@ -96,20 +94,6 @@ impl SortPreservingMergeExec { input, expr, metrics: ExecutionPlanMetricsSet::new(), - satisfy_distribution: false, - } - } - - /// Create a new SortPreservingMergeExec to satisfy the Sort Distribution - pub fn new_for_distribuion( - expr: Vec, - input: Arc, - ) -> Self { - Self { - input, - expr, - metrics: ExecutionPlanMetricsSet::new(), - satisfy_distribution: true, } } @@ -122,11 +106,6 @@ impl SortPreservingMergeExec { pub fn expr(&self) -> &[PhysicalSortExpr] { &self.expr } - - /// satisfy the Sort Distribution requirements - pub fn satisfy_distribution(&self) -> bool { - self.satisfy_distribution - } } impl ExecutionPlan for SortPreservingMergeExec { diff --git a/datafusion/core/src/physical_plan/windows/bounded_window_agg_exec.rs b/datafusion/core/src/physical_plan/windows/bounded_window_agg_exec.rs index 2affa9f364109..aee8d3edf93f4 100644 --- a/datafusion/core/src/physical_plan/windows/bounded_window_agg_exec.rs +++ b/datafusion/core/src/physical_plan/windows/bounded_window_agg_exec.rs @@ -126,7 +126,7 @@ impl BoundedWindowAggExec { let partition_by = self.window_expr()[0].partition_by(); let sort_keys = self .output_ordering() - .unwrap_or(self.sort_keys.as_deref().unwrap_or(&[])); + .unwrap_or_else(|| self.sort_keys.as_deref().unwrap_or(&[])); for item in partition_by { if let Some(a) = sort_keys.iter().find(|&e| e.expr.eq(item)) { result.push(a.clone()); @@ -188,7 +188,7 @@ impl ExecutionPlan for BoundedWindowAggExec { } else { PhysicalSortRequirements { expr: o.expr.clone(), - sort_options: Some(o.options.clone()), + sort_options: Some(o.options), } } }) @@ -460,7 +460,6 @@ impl SortedPartitionByBoundedWindowStream { ) -> Self { let state = window_expr.iter().map(|_| IndexMap::new()).collect(); let empty_batch = RecordBatch::new_empty(schema.clone()); - println!("partition_by_sort_keys {:?}", partition_by_sort_keys); Self { schema, input, diff --git a/datafusion/core/src/physical_plan/windows/window_agg_exec.rs b/datafusion/core/src/physical_plan/windows/window_agg_exec.rs index 16404579233ca..43614e34adc82 100644 --- a/datafusion/core/src/physical_plan/windows/window_agg_exec.rs +++ b/datafusion/core/src/physical_plan/windows/window_agg_exec.rs @@ -117,7 +117,7 @@ impl WindowAggExec { let partition_by = self.window_expr()[0].partition_by(); let sort_keys = self .output_ordering() - .unwrap_or(self.sort_keys.as_deref().unwrap_or(&[])); + .unwrap_or_else(|| self.sort_keys.as_deref().unwrap_or(&[])); for item in partition_by { if let Some(a) = sort_keys.iter().find(|&e| e.expr.eq(item)) { result.push(a.clone()); @@ -193,7 +193,7 @@ impl ExecutionPlan for WindowAggExec { } else { PhysicalSortRequirements { expr: o.expr.clone(), - sort_options: Some(o.options.clone()), + sort_options: Some(o.options), } } }) diff --git a/datafusion/physical-expr/src/utils.rs b/datafusion/physical-expr/src/utils.rs index 44a33b2d45fb3..fa9b8d134efde 100644 --- a/datafusion/physical-expr/src/utils.rs +++ b/datafusion/physical-expr/src/utils.rs @@ -172,16 +172,15 @@ pub fn normalize_expr_with_equivalence_properties( pub fn new_sort_requirements( sort_keys: Option<&[PhysicalSortExpr]>, ) -> Option> { - let ordering_requirements = sort_keys.map(|ordering| { + sort_keys.map(|ordering| { ordering .iter() .map(|o| PhysicalSortRequirements { expr: o.expr.clone(), - sort_options: Some(o.options.clone()), + sort_options: Some(o.options), }) .collect::>() - }); - ordering_requirements + }) } pub fn normalize_sort_expr_with_equivalence_properties( @@ -212,7 +211,7 @@ pub fn normalize_sort_requirement_with_equivalence_properties( if sort_requirement.expr.ne(&normalized_expr) { PhysicalSortRequirements { expr: normalized_expr, - sort_options: sort_requirement.sort_options.clone(), + sort_options: sort_requirement.sort_options, } } else { sort_requirement @@ -410,7 +409,7 @@ pub fn map_requirement_before_projection( .zip(requirement.iter()) .map(|(new, old)| PhysicalSortRequirements { expr: new.clone(), - sort_options: old.sort_options.clone(), + sort_options: old.sort_options, }) .collect::>(); Some(new_request) @@ -431,7 +430,7 @@ pub fn create_sort_expr_from_requirement( if prop.sort_options.is_some() { PhysicalSortExpr { expr: prop.expr.clone(), - options: prop.sort_options.unwrap().clone(), + options: prop.sort_options.unwrap(), } } else { PhysicalSortExpr { diff --git a/datafusion/sql/src/statement.rs b/datafusion/sql/src/statement.rs index 201cf7a852432..4c3260415f4be 100644 --- a/datafusion/sql/src/statement.rs +++ b/datafusion/sql/src/statement.rs @@ -700,11 +700,9 @@ impl<'a, S: ContextProvider> SqlToRel<'a, S> { let mut assign_map = assignments .iter() .map(|assign| { - let col_name: &Ident = assign - .id - .iter() - .last() - .ok_or(DataFusionError::Plan("Empty column id".to_string()))?; + let col_name: &Ident = assign.id.iter().last().ok_or_else(|| { + DataFusionError::Plan("Empty column id".to_string()) + })?; // Validate that the assignment target column exists table_schema.field_with_unqualified_name(&col_name.value)?; Ok((col_name.value.clone(), assign.value.clone())) diff --git a/datafusion/substrait/src/logical_plan/consumer.rs b/datafusion/substrait/src/logical_plan/consumer.rs index afb83058a4023..f57b56b9b004f 100644 --- a/datafusion/substrait/src/logical_plan/consumer.rs +++ b/datafusion/substrait/src/logical_plan/consumer.rs @@ -612,69 +612,67 @@ pub async fn from_substrait_rex( ))), } } - Some(RexType::Literal(lit)) => { - match &lit.literal_type { - Some(LiteralType::I8(n)) => { - Ok(Arc::new(Expr::Literal(ScalarValue::Int8(Some(*n as i8))))) - } - Some(LiteralType::I16(n)) => { - Ok(Arc::new(Expr::Literal(ScalarValue::Int16(Some(*n as i16))))) - } - Some(LiteralType::I32(n)) => { - Ok(Arc::new(Expr::Literal(ScalarValue::Int32(Some(*n))))) - } - Some(LiteralType::I64(n)) => { - Ok(Arc::new(Expr::Literal(ScalarValue::Int64(Some(*n))))) - } - Some(LiteralType::Boolean(b)) => { - Ok(Arc::new(Expr::Literal(ScalarValue::Boolean(Some(*b))))) - } - Some(LiteralType::Date(d)) => { - Ok(Arc::new(Expr::Literal(ScalarValue::Date32(Some(*d))))) - } - Some(LiteralType::Fp32(f)) => { - Ok(Arc::new(Expr::Literal(ScalarValue::Float32(Some(*f))))) - } - Some(LiteralType::Fp64(f)) => { - Ok(Arc::new(Expr::Literal(ScalarValue::Float64(Some(*f))))) - } - Some(LiteralType::Decimal(d)) => { - let value: [u8; 16] = d.value.clone().try_into().or(Err( - DataFusionError::Substrait( - "Failed to parse decimal value".to_string(), - ), - ))?; - let p = d.precision.try_into().map_err(|e| { - DataFusionError::Substrait(format!( - "Failed to parse decimal precision: {e}" - )) - })?; - let s = d.scale.try_into().map_err(|e| { - DataFusionError::Substrait(format!( - "Failed to parse decimal scale: {e}" - )) - })?; - Ok(Arc::new(Expr::Literal(ScalarValue::Decimal128( - Some(std::primitive::i128::from_le_bytes(value)), - p, - s, - )))) - } - Some(LiteralType::String(s)) => { - Ok(Arc::new(Expr::Literal(ScalarValue::Utf8(Some(s.clone()))))) - } - Some(LiteralType::Binary(b)) => Ok(Arc::new(Expr::Literal( - ScalarValue::Binary(Some(b.clone())), - ))), - Some(LiteralType::Null(ntype)) => { - Ok(Arc::new(Expr::Literal(from_substrait_null(ntype)?))) - } - _ => Err(DataFusionError::NotImplemented(format!( - "Unsupported literal_type: {:?}", - lit.literal_type - ))), + Some(RexType::Literal(lit)) => match &lit.literal_type { + Some(LiteralType::I8(n)) => { + Ok(Arc::new(Expr::Literal(ScalarValue::Int8(Some(*n as i8))))) } - } + Some(LiteralType::I16(n)) => { + Ok(Arc::new(Expr::Literal(ScalarValue::Int16(Some(*n as i16))))) + } + Some(LiteralType::I32(n)) => { + Ok(Arc::new(Expr::Literal(ScalarValue::Int32(Some(*n))))) + } + Some(LiteralType::I64(n)) => { + Ok(Arc::new(Expr::Literal(ScalarValue::Int64(Some(*n))))) + } + Some(LiteralType::Boolean(b)) => { + Ok(Arc::new(Expr::Literal(ScalarValue::Boolean(Some(*b))))) + } + Some(LiteralType::Date(d)) => { + Ok(Arc::new(Expr::Literal(ScalarValue::Date32(Some(*d))))) + } + Some(LiteralType::Fp32(f)) => { + Ok(Arc::new(Expr::Literal(ScalarValue::Float32(Some(*f))))) + } + Some(LiteralType::Fp64(f)) => { + Ok(Arc::new(Expr::Literal(ScalarValue::Float64(Some(*f))))) + } + Some(LiteralType::Decimal(d)) => { + let value: [u8; 16] = d.value.clone().try_into().map_err(|_| { + DataFusionError::Substrait( + "Failed to parse decimal value".to_string(), + ) + })?; + let p = d.precision.try_into().map_err(|e| { + DataFusionError::Substrait(format!( + "Failed to parse decimal precision: {e}" + )) + })?; + let s = d.scale.try_into().map_err(|e| { + DataFusionError::Substrait(format!( + "Failed to parse decimal scale: {e}" + )) + })?; + Ok(Arc::new(Expr::Literal(ScalarValue::Decimal128( + Some(std::primitive::i128::from_le_bytes(value)), + p, + s, + )))) + } + Some(LiteralType::String(s)) => { + Ok(Arc::new(Expr::Literal(ScalarValue::Utf8(Some(s.clone()))))) + } + Some(LiteralType::Binary(b)) => Ok(Arc::new(Expr::Literal( + ScalarValue::Binary(Some(b.clone())), + ))), + Some(LiteralType::Null(ntype)) => { + Ok(Arc::new(Expr::Literal(from_substrait_null(ntype)?))) + } + _ => Err(DataFusionError::NotImplemented(format!( + "Unsupported literal_type: {:?}", + lit.literal_type + ))), + }, _ => Err(DataFusionError::NotImplemented( "unsupported rex_type".to_string(), )), From 245428dd5e46262f44075d77586cdfb27f16b69a Mon Sep 17 00:00:00 2001 From: "mingmwang@ebay.com" Date: Thu, 16 Feb 2023 00:10:35 +0800 Subject: [PATCH 06/13] Fix SortMergeJoin case --- .../physical_optimizer/sort_enforcement2.rs | 733 ++++++++++++++---- .../core/src/physical_optimizer/utils.rs | 21 - 2 files changed, 569 insertions(+), 185 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/sort_enforcement2.rs b/datafusion/core/src/physical_optimizer/sort_enforcement2.rs index bba088339e6ff..5d57285262d5b 100644 --- a/datafusion/core/src/physical_optimizer/sort_enforcement2.rs +++ b/datafusion/core/src/physical_optimizer/sort_enforcement2.rs @@ -36,7 +36,7 @@ use crate::config::ConfigOptions; use crate::error::Result; use crate::execution::context::TaskContext; -use crate::physical_optimizer::utils::add_sort_above_child; +use crate::physical_optimizer::utils::add_sort_above; use crate::physical_optimizer::PhysicalOptimizerRule; use crate::physical_plan::coalesce_partitions::CoalescePartitionsExec; use crate::physical_plan::filter::FilterExec; @@ -56,6 +56,7 @@ use crate::physical_plan::{ }; use arrow::datatypes::SchemaRef; use datafusion_common::{reverse_sort_options, DataFusionError, Statistics}; +use datafusion_expr::JoinType; use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::utils::{ create_sort_expr_from_requirement, map_requirement_before_projection, @@ -68,9 +69,10 @@ use datafusion_physical_expr::{ }; use itertools::izip; use std::any::Any; +use std::ops::Deref; use std::sync::Arc; -/// This rule implements a TOP-Downinspects SortExec's in the given physical plan and removes the +/// This rule implements a Top-Down approach to inspects SortExec's in the given physical plan and removes the /// ones it can prove unnecessary. #[derive(Default)] pub struct TopDownEnforceSorting {} @@ -82,7 +84,7 @@ impl TopDownEnforceSorting { } } -/// This is a "data class" we use within the [EnforceSorting] rule +/// This is a "data class" we use within the [TopDownEnforceSorting] rule #[derive(Debug, Clone)] struct PlanWithSortRequirements { /// Current plan @@ -237,7 +239,7 @@ impl PhysicalOptimizerRule for TopDownEnforceSorting { } fn name(&self) -> &str { - "EnforceSorting2" + "TopDownEnforceSorting" } fn schema_check(&self) -> bool { @@ -378,11 +380,8 @@ fn ensure_sorting( // If the current plan is a SortExec, modify current SortExec to satisfy the parent requirements let parent_required_expr = create_sort_expr_from_requirement(parent_required.unwrap()); - let new_plan = add_sort_above_child( - &sort_exec.input, - parent_required_expr, - sort_exec.fetch(), - )?; + let mut new_plan = sort_exec.input.clone(); + add_sort_above(&mut new_plan, parent_required_expr)?; Ok(Some( PlanWithSortRequirements::new_without_impact_result_ordering(new_plan), )) @@ -402,7 +401,8 @@ fn ensure_sorting( || plan.as_any().downcast_ref::().is_some()) && plan.as_any().downcast_ref::().is_none() { - let new_plan = add_sort_above_child(plan, parent_required_expr, None)?; + let mut new_plan = plan.clone(); + add_sort_above(&mut new_plan, parent_required_expr)?; Ok(Some( PlanWithSortRequirements::new_without_impact_result_ordering(new_plan), )) @@ -425,7 +425,7 @@ fn ensure_sorting( let adjusted = parent_required.map(|r| r.to_vec()); Ok(Some(PlanWithSortRequirements { plan: plan.clone(), - impact_result_ordering: true, + impact_result_ordering: requirements.impact_result_ordering, satisfy_single_distribution: requirements.satisfy_single_distribution, required_ordering: None, adjusted_request_ordering: vec![adjusted], @@ -479,7 +479,8 @@ fn ensure_sorting( } } // Can not push down requirements, add new SortExec - let new_plan = add_sort_above_child(plan, parent_required_expr, None)?; + let mut new_plan = plan.clone(); + add_sort_above(&mut new_plan, parent_required_expr)?; Ok(Some( PlanWithSortRequirements::new_without_impact_result_ordering( new_plan, @@ -490,70 +491,42 @@ fn ensure_sorting( // If the current plan is SortMergeJoinExec let left_columns_len = smj.left.schema().fields().len(); let expr_source_side = - expr_source_sides(&parent_required_expr, left_columns_len); + expr_source_sides(&parent_required_expr, smj.join_type, left_columns_len); match expr_source_side { Some(JoinSide::Left) if maintains_input_order[0] => { - if requirements_compatible( - plan.required_input_ordering()[0].as_deref(), + try_pushdown_requirements_to_join( + &requirements, + plan, parent_required, - || plan.children()[0].equivalence_properties(), - ) { - println!("Requirements are compatible with SMJ"); - Ok(Some(PlanWithSortRequirements { - plan: plan.clone(), - impact_result_ordering: true, - satisfy_single_distribution: requirements - .satisfy_single_distribution, - required_ordering: None, - adjusted_request_ordering: requirements - .adjusted_request_ordering, - })) - } else { - // Can not push down, add new SortExec - println!("Can not push down, add new SortExec"); - let new_plan = - add_sort_above_child(plan, parent_required_expr, None)?; - Ok(Some( - PlanWithSortRequirements::new_without_impact_result_ordering( - new_plan, - ), - )) - } + parent_required_expr, + JoinSide::Left, + ) } Some(JoinSide::Right) if maintains_input_order[1] => { - let shift_right_required = - shift_right_required(parent_required.unwrap(), left_columns_len); - if requirements_compatible( - plan.required_input_ordering()[1].as_deref(), - shift_right_required.as_deref(), - || plan.children()[1].equivalence_properties(), - ) { - println!("Requirements are compatible with SMJ"); - Ok(Some(PlanWithSortRequirements { - plan: plan.clone(), - impact_result_ordering: true, - satisfy_single_distribution: requirements - .satisfy_single_distribution, - required_ordering: None, - adjusted_request_ordering: requirements - .adjusted_request_ordering, - })) - } else { - // Can not push down, add new SortExec - println!("Can not push down, add new SortExec"); - let new_plan = - add_sort_above_child(plan, parent_required_expr, None)?; - Ok(Some( - PlanWithSortRequirements::new_without_impact_result_ordering( - new_plan, - ), - )) - } + let new_right_required = match smj.join_type { + JoinType::Inner | JoinType::Right => shift_right_required( + parent_required.unwrap(), + left_columns_len, + )?, + JoinType::RightSemi | JoinType::RightAnti => { + parent_required.unwrap().to_vec() + } + _ => Err(DataFusionError::Plan( + "Unexpected SortMergeJoin type here".to_string(), + ))?, + }; + try_pushdown_requirements_to_join( + &requirements, + plan, + Some(new_right_required.deref()), + parent_required_expr, + JoinSide::Right, + ) } _ => { - println!("Can not decide the expr side for SortMergeJoinExec, can not push down, add SortExec"); - let new_plan = - add_sort_above_child(plan, parent_required_expr, None)?; + // Can not decide the expr side for SortMergeJoinExec, can not push down, add SortExec; + let mut new_plan = plan.clone(); + add_sort_above(&mut new_plan, parent_required_expr)?; Ok(Some( PlanWithSortRequirements::new_without_impact_result_ordering( new_plan, @@ -564,11 +537,6 @@ fn ensure_sorting( } else if plan.required_input_ordering().iter().any(Option::is_some) { // If the current plan has its own ordering requirements to its children, check whether the requirements // are compatible with the parent requirements. - println!( - "the current plan has its own ordering requirements, {:?}", - plan.required_input_ordering() - ); - let plan_children = plan.children(); let compatible_with_children = izip!( maintains_input_order.iter(), @@ -586,13 +554,7 @@ fn ensure_sorting( .collect::>(); if compatible_with_children.iter().all(|a| *a) { // Requirements are compatible, not need to push down. - Ok(Some(PlanWithSortRequirements { - plan: plan.clone(), - impact_result_ordering: true, - satisfy_single_distribution: requirements.satisfy_single_distribution, - required_ordering: None, - adjusted_request_ordering: requirements.adjusted_request_ordering, - })) + Ok(None) } else { let can_adjust_child_requirements = plan .required_input_ordering() @@ -608,11 +570,10 @@ fn ensure_sorting( .collect::>(); if can_adjust_child_requirements.iter().all(|a| *a) { // Adjust child requirements and push down the requirements - println!("Adjust child requirements and push down the requirements"); let adjusted = parent_required.map(|r| r.to_vec()); Ok(Some(PlanWithSortRequirements { plan: plan.clone(), - impact_result_ordering: true, + impact_result_ordering: requirements.impact_result_ordering, satisfy_single_distribution: requirements .satisfy_single_distribution, required_ordering: None, @@ -624,9 +585,8 @@ fn ensure_sorting( })) } else { // Can not push down, add new SortExec - println!("Can not push down, add new SortExec"); - let new_plan = - add_sort_above_child(plan, parent_required_expr, None)?; + let mut new_plan = plan.clone(); + add_sort_above(&mut new_plan, parent_required_expr)?; Ok(Some( PlanWithSortRequirements::new_without_impact_result_ordering( new_plan, @@ -644,10 +604,9 @@ fn ensure_sorting( let new_requirement = map_requirement_before_projection(parent_required, expr); if new_requirement.is_some() { - println!("Push requirements down to Projection"); Ok(Some(PlanWithSortRequirements { plan: plan.clone(), - impact_result_ordering: true, + impact_result_ordering: requirements.impact_result_ordering, satisfy_single_distribution: requirements .satisfy_single_distribution, required_ordering: None, @@ -655,11 +614,8 @@ fn ensure_sorting( })) } else { // Can not push down, add new SortExec - println!( - "Can not push requirements down to Projection, add SortExec" - ); - let new_plan = - add_sort_above_child(plan, parent_required_expr, None)?; + let mut new_plan = plan.clone(); + add_sort_above(&mut new_plan, parent_required_expr)?; Ok(Some( PlanWithSortRequirements::new_without_impact_result_ordering( new_plan, @@ -667,7 +623,6 @@ fn ensure_sorting( )) } } else { - println!("Push down requirements."); Ok(Some(PlanWithSortRequirements { plan: plan.clone(), impact_result_ordering: requirements.impact_result_ordering, @@ -767,76 +722,6 @@ fn analyze_immediate_spm_removal( } } -fn expr_source_sides( - required_exprs: &[PhysicalSortExpr], - left_columns_len: usize, -) -> Option { - let all_column_sides = required_exprs - .iter() - .filter_map(|r| { - if let Some(col) = r.expr.as_any().downcast_ref::() { - if col.index() < left_columns_len { - Some(JoinSide::Left) - } else { - Some(JoinSide::Right) - } - } else { - None - } - }) - .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 - } -} - -fn shift_right_required( - parent_required: &[PhysicalSortRequirements], - left_columns_len: usize, -) -> Option> { - let new_right_required: Vec = parent_required - .iter() - .filter_map(|r| { - if let Some(col) = r.expr.as_any().downcast_ref::() { - if col.index() >= left_columns_len { - Some(PhysicalSortRequirements { - expr: Arc::new(Column::new( - col.name(), - col.index() - left_columns_len, - )) as Arc, - sort_options: r.sort_options, - }) - } else { - None - } - } else { - None - } - }) - .collect::>(); - - // if the parent required are all comming from the right side, the requirements can be pushdown - if new_right_required.len() != parent_required.len() { - None - } else { - Some(new_right_required) - } -} - /// Compares window expression's `window_request` and `parent_required_expr` ordering, returns /// whether we should reverse the window expression's ordering in order to meet parent's requirements. fn check_alignment( @@ -1025,6 +910,169 @@ fn extract_window_info_from_plan( } } +fn try_pushdown_requirements_to_join( + requirements: &PlanWithSortRequirements, + plan: &Arc, + parent_required: Option<&[PhysicalSortRequirements]>, + parent_required_expr: Vec, + push_side: JoinSide, +) -> Result> { + let child_idx = match push_side { + JoinSide::Left => 0, + JoinSide::Right => 1, + }; + if requirements_compatible( + plan.required_input_ordering()[child_idx].as_deref(), + parent_required, + || plan.children()[child_idx].equivalence_properties(), + ) { + // parent requirements are compatible with the SortMergeJoinExec + Ok(None) + } else if requirements_compatible( + parent_required, + plan.required_input_ordering()[child_idx].as_deref(), + || plan.children()[child_idx].equivalence_properties(), + ) { + // parent requirements are more specific, adjust the SortMergeJoinExec child requirements and push down the new requirements + let new_adjusted = match push_side { + JoinSide::Left => vec![ + parent_required.map(|r| r.to_vec()), + requirements.adjusted_request_ordering[1].clone(), + ], + JoinSide::Right => vec![ + requirements.adjusted_request_ordering[0].clone(), + parent_required.map(|r| r.to_vec()), + ], + }; + Ok(Some(PlanWithSortRequirements { + plan: plan.clone(), + impact_result_ordering: requirements.impact_result_ordering, + satisfy_single_distribution: requirements.satisfy_single_distribution, + required_ordering: None, + adjusted_request_ordering: new_adjusted, + })) + } else { + // Can not push down, add new SortExec + let mut new_plan = plan.clone(); + add_sort_above(&mut new_plan, parent_required_expr)?; + Ok(Some( + PlanWithSortRequirements::new_without_impact_result_ordering(new_plan), + )) + } +} + +fn expr_source_sides( + required_exprs: &[PhysicalSortExpr], + join_type: JoinType, + left_columns_len: usize, +) -> Option { + match join_type { + JoinType::Inner | JoinType::Left | JoinType::Right | JoinType::Full => { + let all_column_sides = required_exprs + .iter() + .filter_map(|r| { + if let Some(col) = r.expr.as_any().downcast_ref::() { + if col.index() < left_columns_len { + Some(JoinSide::Left) + } else { + Some(JoinSide::Right) + } + } else { + None + } + }) + .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 => { + if required_exprs + .iter() + .filter_map(|r| { + if r.expr.as_any().downcast_ref::().is_some() { + Some(JoinSide::Left) + } else { + None + } + }) + .count() + != required_exprs.len() + { + None + } else { + Some(JoinSide::Left) + } + } + JoinType::RightSemi | JoinType::RightAnti => { + if required_exprs + .iter() + .filter_map(|r| { + if r.expr.as_any().downcast_ref::().is_some() { + Some(JoinSide::Right) + } else { + None + } + }) + .count() + != required_exprs.len() + { + None + } else { + Some(JoinSide::Right) + } + } + } +} + +fn shift_right_required( + parent_required: &[PhysicalSortRequirements], + left_columns_len: usize, +) -> Result> { + let new_right_required: Vec = parent_required + .iter() + .filter_map(|r| { + if let Some(col) = r.expr.as_any().downcast_ref::() { + if col.index() >= left_columns_len { + Some(PhysicalSortRequirements { + expr: Arc::new(Column::new( + col.name(), + col.index() - left_columns_len, + )) as Arc, + sort_options: r.sort_options, + }) + } else { + None + } + } else { + None + } + }) + .collect::>(); + if new_right_required.len() != parent_required.len() { + Err(DataFusionError::Plan( + "Expect to shift all the parent required column indexes for SortMergeJoin" + .to_string(), + )) + } else { + Ok(new_right_required) + } +} + #[derive(Debug)] /// This structure stores extra Window information required to create a new WindowExec pub struct WindowExecInfo { @@ -1119,6 +1167,7 @@ mod tests { use crate::physical_plan::displayable; use crate::physical_plan::file_format::{FileScanConfig, ParquetExec}; use crate::physical_plan::filter::FilterExec; + use crate::physical_plan::joins::utils::JoinOn; use crate::physical_plan::memory::MemoryExec; use crate::physical_plan::repartition::RepartitionExec; use crate::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; @@ -1128,6 +1177,7 @@ mod tests { use arrow::compute::SortOptions; use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use datafusion_common::{Result, Statistics}; + use datafusion_expr::logical_plan::JoinType; use datafusion_expr::{AggregateFunction, WindowFrame, WindowFunction}; use datafusion_physical_expr::expressions::{col, NotExpr}; use datafusion_physical_expr::PhysicalSortExpr; @@ -1142,6 +1192,13 @@ mod tests { 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) + } + // Util function to get string representation of a physical plan fn get_plan_string(plan: &Arc) -> Vec { let formatted = displayable(plan.as_ref()).indent().to_string(); @@ -2160,6 +2217,111 @@ mod tests { 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 = vec![ + "UnionExec", + " SortExec: expr=[nullable_col@0 ASC,non_nullable_col@1 ASC], global=true", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " SortExec: expr=[nullable_col@0 DESC NULLS LAST,non_nullable_col@1 DESC NULLS LAST], global=true", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + ]; + let expected_optimized = vec![ + "UnionExec", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan); + 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 = vec![ + "SortPreservingMergeExec: [nullable_col@0 ASC]", + " UnionExec", + " SortExec: expr=[nullable_col@0 ASC,non_nullable_col@1 ASC], global=true", + " ParquetExec: limit=None, partitions={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], global=true", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + ]; + let expected_optimized = vec![ + "SortPreservingMergeExec: [nullable_col@0 ASC]", + " UnionExec", + " SortExec: expr=[nullable_col@0 ASC], global=true", + " ParquetExec: limit=None, partitions={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], global=true", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan); + Ok(()) + } + #[tokio::test] async fn test_window_multi_path_sort() -> Result<()> { let schema = create_test_schema()?; @@ -2246,6 +2408,230 @@ mod tests { 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![( + Column::new_with_schema("nullable_col", &left.schema()).unwrap(), + Column::new_with_schema("col_a", &right.schema()).unwrap(), + )]; + + 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=[(Column {{ name: \"nullable_col\", index: 0 }}, Column {{ name: \"col_a\", index: 0 }})]"); + let join_plan2 = + format!(" SortMergeJoin: join_type={join_type}, on=[(Column {{ name: \"nullable_col\", index: 0 }}, Column {{ name: \"col_a\", index: 0 }})]"); + + let expected_input = vec![ + "SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", + join_plan.as_str(), + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " ParquetExec: limit=None, partitions={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![ + "SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", + join_plan.as_str(), + " SortExec: expr=[nullable_col@0 ASC,non_nullable_col@1 ASC], global=true", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " SortExec: expr=[col_a@0 ASC], global=true", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[col_a, col_b]", + ] + } + _ => { + // can not push down the sort requirements + vec![ + "SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", + " SortExec: expr=[nullable_col@0 ASC,non_nullable_col@1 ASC], global=true", + join_plan2.as_str(), + " SortExec: expr=[nullable_col@0 ASC], global=true", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " SortExec: expr=[col_a@0 ASC], global=true", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[col_a, col_b]", + ] + } + }; + assert_optimized!(expected_input, expected_optimized, physical_plan); + } + 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![( + Column::new_with_schema("nullable_col", &left.schema()).unwrap(), + Column::new_with_schema("col_a", &right.schema()).unwrap(), + )]; + + 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=[(Column {{ name: \"nullable_col\", index: 0 }}, Column {{ name: \"col_a\", index: 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=[(Column {{ name: \"nullable_col\", index: 0 }}, Column {{ name: \"col_a\", index: 0 }})]"); + + let expected_input = vec![ + spm_plan, + join_plan.as_str(), + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " ParquetExec: limit=None, partitions={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![ + spm_plan, + join_plan.as_str(), + " SortExec: expr=[nullable_col@0 ASC], global=true", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " SortExec: expr=[col_a@0 ASC,col_b@1 ASC], global=true", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[col_a, col_b]", + ] + } + _ => { + // can not push down the sort requirements for Left and Full join. + vec![ + spm_plan, + " SortExec: expr=[col_a@2 ASC,col_b@3 ASC], global=true", + join_plan2.as_str(), + " SortExec: expr=[nullable_col@0 ASC], global=true", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " SortExec: expr=[col_a@0 ASC], global=true", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[col_a, col_b]", + ] + } + }; + assert_optimized!(expected_input, expected_optimized, physical_plan); + } + 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![( + Column::new_with_schema("nullable_col", &left.schema()).unwrap(), + Column::new_with_schema("col_a", &right.schema()).unwrap(), + )]; + + 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 = vec![ + "SortPreservingMergeExec: [col_b@3 ASC,col_a@2 ASC]", + " SortMergeJoin: join_type=Inner, on=[(Column { name: \"nullable_col\", index: 0 }, Column { name: \"col_a\", index: 0 })]", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[col_a, col_b]", + ]; + + // can not push down the sort requirements, need to add SortExec + let expected_optimized = vec![ + "SortPreservingMergeExec: [col_b@3 ASC,col_a@2 ASC]", + " SortExec: expr=[col_b@3 ASC,col_a@2 ASC], global=true", + " SortMergeJoin: join_type=Inner, on=[(Column { name: \"nullable_col\", index: 0 }, Column { name: \"col_a\", index: 0 })]", + " SortExec: expr=[nullable_col@0 ASC], global=true", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " SortExec: expr=[col_a@0 ASC], global=true", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[col_a, col_b]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan); + + // 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 = vec![ + "SortPreservingMergeExec: [nullable_col@0 ASC,col_b@3 ASC,col_a@2 ASC]", + " SortMergeJoin: join_type=Inner, on=[(Column { name: \"nullable_col\", index: 0 }, Column { name: \"col_a\", index: 0 })]", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[col_a, col_b]", + ]; + + // can not push down the sort requirements, need to add SortExec + let expected_optimized = vec![ + "SortPreservingMergeExec: [nullable_col@0 ASC,col_b@3 ASC,col_a@2 ASC]", + " SortExec: expr=[nullable_col@0 ASC,col_b@3 ASC,col_a@2 ASC], global=true", + " SortMergeJoin: join_type=Inner, on=[(Column { name: \"nullable_col\", index: 0 }, Column { name: \"col_a\", index: 0 })]", + " SortExec: expr=[nullable_col@0 ASC], global=true", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " SortExec: expr=[col_a@0 ASC], global=true", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[col_a, col_b]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan); + + Ok(()) + } + /// make PhysicalSortExpr with default options fn sort_expr(name: &str, schema: &Schema) -> PhysicalSortExpr { sort_expr_options(name, schema, SortOptions::default()) @@ -2397,4 +2783,23 @@ mod tests { .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(), + *join_type, + vec![SortOptions::default(); join_on.len()], + false, + ) + .unwrap(), + ) + } } diff --git a/datafusion/core/src/physical_optimizer/utils.rs b/datafusion/core/src/physical_optimizer/utils.rs index 31c47bae03ad6..b6666fbefae1e 100644 --- a/datafusion/core/src/physical_optimizer/utils.rs +++ b/datafusion/core/src/physical_optimizer/utils.rs @@ -67,24 +67,3 @@ pub fn add_sort_above( } Ok(()) } - -/// Util function to add SortExec above child -/// preserving the original partitioning -pub fn add_sort_above_child( - child: &Arc, - sort_expr: Vec, - fetch: Option, -) -> Result> { - let new_child = if child.output_partitioning().partition_count() > 1 { - Arc::new(SortExec::new_with_partitioning( - sort_expr, - child.clone(), - true, - fetch, - )) as Arc - } else { - Arc::new(SortExec::try_new(sort_expr, child.clone(), fetch)?) - as Arc - }; - Ok(new_child) -} From abad84c4209a8e6a9e8a9452740c472ead0dfcdb Mon Sep 17 00:00:00 2001 From: "mingmwang@ebay.com" Date: Thu, 16 Feb 2023 16:30:47 +0800 Subject: [PATCH 07/13] Fix reverse window sort requirements --- .../physical_optimizer/sort_enforcement2.rs | 9 +++++++ datafusion/core/tests/sql/window.rs | 27 ++++++++++--------- 2 files changed, 23 insertions(+), 13 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/sort_enforcement2.rs b/datafusion/core/src/physical_optimizer/sort_enforcement2.rs index 5d57285262d5b..7d0d431a16d9f 100644 --- a/datafusion/core/src/physical_optimizer/sort_enforcement2.rs +++ b/datafusion/core/src/physical_optimizer/sort_enforcement2.rs @@ -775,6 +775,15 @@ fn should_reverse_window_sort_requirements( if top_requirement.is_none() { return false; } + let WindowExecInfo { window_expr, .. } = + extract_window_info_from_plan(&window_plan).unwrap(); + let reverse_window_expr = window_expr + .iter() + .map(|e| e.get_reverse_expr()) + .collect::>>(); + if reverse_window_expr.is_none() { + return false; + } let flags = window_plan .children() .into_iter() diff --git a/datafusion/core/tests/sql/window.rs b/datafusion/core/tests/sql/window.rs index 578f75360350c..116f37ace11be 100644 --- a/datafusion/core/tests/sql/window.rs +++ b/datafusion/core/tests/sql/window.rs @@ -1601,10 +1601,11 @@ async fn test_window_agg_sort_multi_layer_non_reversed_plan() -> Result<()> { vec![ "ProjectionExec: expr=[c9@2 as c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c1 ASC NULLS LAST, aggregate_test_100.c2 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@5 as sum1, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST, aggregate_test_100.c1 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@3 as sum2, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@4 as rn2]", " GlobalLimitExec: skip=0, fetch=5", - " BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9): Ok(Field { name: \"SUM(aggregate_test_100.c9)\", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: Following(UInt64(1)) }]", - " BoundedWindowAggExec: wdw=[ROW_NUMBER(): Ok(Field { name: \"ROW_NUMBER()\", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)) }]", - " BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9): Ok(Field { name: \"SUM(aggregate_test_100.c9)\", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)) }]", - " SortExec: expr=[c9@2 DESC,c1@0 DESC,c2@1 DESC], global=true", + " BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9): Ok(Field { name: \"SUM(aggregate_test_100.c9)\", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)) }]", + " SortExec: expr=[c9@2 ASC NULLS LAST,c1@0 ASC NULLS LAST,c2@1 ASC NULLS LAST], global=true", + " BoundedWindowAggExec: wdw=[ROW_NUMBER(): Ok(Field { name: \"ROW_NUMBER()\", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)) }]", + " BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9): Ok(Field { name: \"SUM(aggregate_test_100.c9)\", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)) }]", + " SortExec: expr=[c9@2 DESC,c1@0 DESC], global=true", ] }; @@ -1618,15 +1619,15 @@ async fn test_window_agg_sort_multi_layer_non_reversed_plan() -> Result<()> { let actual = execute_to_batches(&ctx, sql).await; let expected = vec![ - "+------------+-------------+-------------+-----+", - "| c9 | sum1 | sum2 | rn2 |", - "+------------+-------------+-------------+-----+", - "| 4268716378 | 8498370520 | 24997484146 | 1 |", - "| 4229654142 | 12714811027 | 29012926487 | 2 |", - "| 4216440507 | 16858984380 | 28743001064 | 3 |", - "| 4144173353 | 20935849039 | 28472563256 | 4 |", - "| 4076864659 | 24997484146 | 28118515915 | 5 |", - "+------------+-------------+-------------+-----+", + "+-----------+------------+-----------+-----+", + "| c9 | sum1 | sum2 | rn2 |", + "+-----------+------------+-----------+-----+", + "| 28774375 | 745354217 | 91818943 | 100 |", + "| 63044568 | 988558066 | 232866360 | 99 |", + "| 141047417 | 1285934966 | 374546521 | 98 |", + "| 141680161 | 1654839259 | 519841132 | 97 |", + "| 145294611 | 1980231675 | 745354217 | 96 |", + "+-----------+------------+-----------+-----+", ]; assert_batches_eq!(expected, &actual); From 6a53df0a65af636126946a35e02724028879b642 Mon Sep 17 00:00:00 2001 From: "mingmwang@ebay.com" Date: Thu, 16 Feb 2023 16:50:28 +0800 Subject: [PATCH 08/13] fix test comments --- datafusion/core/src/physical_optimizer/sort_enforcement2.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/sort_enforcement2.rs b/datafusion/core/src/physical_optimizer/sort_enforcement2.rs index 7d0d431a16d9f..7cd83ee7e7ec0 100644 --- a/datafusion/core/src/physical_optimizer/sort_enforcement2.rs +++ b/datafusion/core/src/physical_optimizer/sort_enforcement2.rs @@ -2397,8 +2397,8 @@ mod tests { let union = union_exec(vec![sort1, sort2]); let physical_plan = window_exec("nullable_col", sort_exprs2, union); - // The `WindowAggExec` gets its sorting from multiple children jointly. - // The SortExecs should be kept to ensure the final result ordering + // The `WindowAggExec` can get its required sorting from the leaf nodes directly. + // The unnecessary SortExecs should be removed let expected_input = vec![ "WindowAggExec: wdw=[count: Ok(Field { name: \"count\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(NULL), end_bound: CurrentRow }]", " UnionExec", From c5c5bc8cb62fbdb02b455be704952c8054478357 Mon Sep 17 00:00:00 2001 From: "mingmwang@ebay.com" Date: Thu, 16 Feb 2023 22:11:53 +0800 Subject: [PATCH 09/13] add determine_children_requirement() method --- .../physical_optimizer/sort_enforcement2.rs | 349 +++++++++--------- 1 file changed, 174 insertions(+), 175 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/sort_enforcement2.rs b/datafusion/core/src/physical_optimizer/sort_enforcement2.rs index 7cd83ee7e7ec0..e2d5e1d8c7251 100644 --- a/datafusion/core/src/physical_optimizer/sort_enforcement2.rs +++ b/datafusion/core/src/physical_optimizer/sort_enforcement2.rs @@ -297,9 +297,6 @@ fn ensure_sorting( // propagate the sort requirements down to correct the unnecessary descendant SortExec under the UnionExec let adjusted = new_sort_requirements(union_exec.output_ordering()); return Ok(Some(PlanWithSortRequirements { - plan: plan.clone(), - impact_result_ordering: requirements.impact_result_ordering, - satisfy_single_distribution: requirements.satisfy_single_distribution, required_ordering: None, adjusted_request_ordering: vec![ adjusted; @@ -307,6 +304,7 @@ fn ensure_sorting( .adjusted_request_ordering .len() ], + ..requirements })); } else if plan.as_any().downcast_ref::().is_some() || plan @@ -370,11 +368,8 @@ fn ensure_sorting( } } Ok(Some(PlanWithSortRequirements { - plan: plan.clone(), - impact_result_ordering: requirements.impact_result_ordering, - satisfy_single_distribution: requirements.satisfy_single_distribution, required_ordering: None, - adjusted_request_ordering: requirements.adjusted_request_ordering, + ..requirements })) } else if let Some(sort_exec) = plan.as_any().downcast_ref::() { // If the current plan is a SortExec, modify current SortExec to satisfy the parent requirements @@ -413,79 +408,79 @@ fn ensure_sorting( .is_some() { let request_child = requirements.adjusted_request_ordering[0].as_deref(); - if requirements_compatible(request_child, parent_required, || { - plan.children()[0].equivalence_properties() - }) { - // request child requirements are more specific, no need to push down the parent requirements - Ok(None) - } else if requirements_compatible(parent_required, request_child, || { - plan.children()[0].equivalence_properties() - }) { - // parent requirements are more specific, adjust the request child requirements and push down the new requirements - let adjusted = parent_required.map(|r| r.to_vec()); - Ok(Some(PlanWithSortRequirements { - plan: plan.clone(), - impact_result_ordering: requirements.impact_result_ordering, - satisfy_single_distribution: requirements.satisfy_single_distribution, - required_ordering: None, - adjusted_request_ordering: vec![adjusted], - })) - } else { - let WindowExecInfo { - window_expr, - input_schema, - partition_keys, - } = extract_window_info_from_plan(plan).unwrap(); - if should_reverse_window_exec( - parent_required, - request_child, - &input_schema, - ) { - let new_physical_ordering = parent_required_expr.to_vec(); - let new_window_expr = window_expr - .iter() - .map(|e| e.get_reverse_expr()) - .collect::>>(); - if let Some(window_expr) = new_window_expr { - let uses_bounded_memory = - window_expr.iter().all(|e| e.uses_bounded_memory()); - let new_plan = if uses_bounded_memory { - Arc::new(BoundedWindowAggExec::try_new( - window_expr, - plan.children()[0].clone(), - input_schema, - partition_keys, - Some(new_physical_ordering), - )?) as Arc - } else { - Arc::new(WindowAggExec::try_new( - window_expr, - plan.children()[0].clone(), - input_schema, - partition_keys, - Some(new_physical_ordering), - )?) as Arc - }; - let adjusted_request_ordering = - new_plan.required_input_ordering(); - return Ok(Some(PlanWithSortRequirements { - plan: new_plan, - impact_result_ordering: false, - satisfy_single_distribution: requirements - .satisfy_single_distribution, - required_ordering: None, - adjusted_request_ordering, - })); + let child_plan = plan.children()[0].clone(); + match determine_children_requirement( + parent_required, + request_child, + child_plan, + ) { + RequirementsCompatibility::Satisfy => Ok(None), + RequirementsCompatibility::Compatible(adjusted) => { + Ok(Some(PlanWithSortRequirements { + required_ordering: None, + adjusted_request_ordering: vec![adjusted], + ..requirements + })) + } + RequirementsCompatibility::NonCompatible => { + let WindowExecInfo { + window_expr, + input_schema, + partition_keys, + } = extract_window_info_from_plan(plan).unwrap(); + if should_reverse_window_exec( + parent_required, + request_child, + &input_schema, + ) { + let new_physical_ordering = parent_required_expr.to_vec(); + let new_window_expr = window_expr + .iter() + .map(|e| e.get_reverse_expr()) + .collect::>>(); + if let Some(window_expr) = new_window_expr { + let uses_bounded_memory = + window_expr.iter().all(|e| e.uses_bounded_memory()); + let new_plan = if uses_bounded_memory { + Arc::new(BoundedWindowAggExec::try_new( + window_expr, + plan.children()[0].clone(), + input_schema, + partition_keys, + Some(new_physical_ordering), + )?) + as Arc + } else { + Arc::new(WindowAggExec::try_new( + window_expr, + plan.children()[0].clone(), + input_schema, + partition_keys, + Some(new_physical_ordering), + )?) + as Arc + }; + let adjusted_request_ordering = + new_plan.required_input_ordering(); + return Ok(Some(PlanWithSortRequirements { + plan: new_plan, + impact_result_ordering: false, + satisfy_single_distribution: requirements + .satisfy_single_distribution, + required_ordering: None, + adjusted_request_ordering, + })); + } } + // Can not push down requirements, add new SortExec + let mut new_plan = plan.clone(); + add_sort_above(&mut new_plan, parent_required_expr)?; + Ok(Some( + PlanWithSortRequirements::new_without_impact_result_ordering( + new_plan, + ), + )) } - // Can not push down requirements, add new SortExec - let mut new_plan = plan.clone(); - add_sort_above(&mut new_plan, parent_required_expr)?; - Ok(Some( - PlanWithSortRequirements::new_without_impact_result_ordering( - new_plan, - ), - )) } } else if let Some(smj) = plan.as_any().downcast_ref::() { // If the current plan is SortMergeJoinExec @@ -496,7 +491,6 @@ fn ensure_sorting( Some(JoinSide::Left) if maintains_input_order[0] => { try_pushdown_requirements_to_join( &requirements, - plan, parent_required, parent_required_expr, JoinSide::Left, @@ -517,7 +511,6 @@ fn ensure_sorting( }; try_pushdown_requirements_to_join( &requirements, - plan, Some(new_right_required.deref()), parent_required_expr, JoinSide::Right, @@ -535,8 +528,6 @@ fn ensure_sorting( } } } else if plan.required_input_ordering().iter().any(Option::is_some) { - // If the current plan has its own ordering requirements to its children, check whether the requirements - // are compatible with the parent requirements. let plan_children = plan.children(); let compatible_with_children = izip!( maintains_input_order.iter(), @@ -544,55 +535,43 @@ fn ensure_sorting( plan_children.iter() ) .map(|(can_push_down, request_child, child)| { - *can_push_down - && requirements_compatible( - request_child.as_deref(), + if *can_push_down { + determine_children_requirement( parent_required, - || child.equivalence_properties(), + request_child.as_deref(), + child.clone(), ) + } else { + RequirementsCompatibility::NonCompatible + } }) .collect::>(); - if compatible_with_children.iter().all(|a| *a) { - // Requirements are compatible, not need to push down. + if compatible_with_children + .iter() + .all(|a| matches!(a, RequirementsCompatibility::Satisfy)) + { + // Requirements are satisfied, not need to push down. Ok(None) + } else if compatible_with_children + .iter() + .all(|a| matches!(a, RequirementsCompatibility::Compatible(_))) + { + // Adjust child requirements and push down the requirements + let adjusted = parent_required.map(|r| r.to_vec()); + Ok(Some(PlanWithSortRequirements { + required_ordering: None, + adjusted_request_ordering: vec![adjusted; plan_children.len()], + ..requirements + })) } else { - let can_adjust_child_requirements = plan - .required_input_ordering() - .into_iter() - .zip(plan_children.iter()) - .map(|(request_child, child)| { - requirements_compatible( - parent_required, - request_child.as_deref(), - || child.equivalence_properties(), - ) - }) - .collect::>(); - if can_adjust_child_requirements.iter().all(|a| *a) { - // Adjust child requirements and push down the requirements - let adjusted = parent_required.map(|r| r.to_vec()); - Ok(Some(PlanWithSortRequirements { - plan: plan.clone(), - impact_result_ordering: requirements.impact_result_ordering, - satisfy_single_distribution: requirements - .satisfy_single_distribution, - required_ordering: None, - adjusted_request_ordering: vec![ - adjusted; - can_adjust_child_requirements - .len() - ], - })) - } else { - // Can not push down, add new SortExec - let mut new_plan = plan.clone(); - add_sort_above(&mut new_plan, parent_required_expr)?; - Ok(Some( - PlanWithSortRequirements::new_without_impact_result_ordering( - new_plan, - ), - )) - } + // Can not push down, add new SortExec + let mut new_plan = plan.clone(); + add_sort_above(&mut new_plan, parent_required_expr)?; + Ok(Some( + PlanWithSortRequirements::new_without_impact_result_ordering( + new_plan, + ), + )) } } else { // The current plan does not have its own ordering requirements to its children, consider push down the requirements @@ -601,16 +580,13 @@ fn ensure_sorting( { // For Projection, we need to transform the requirements to the columns before the Projection // And then to push down the requirements - let new_requirement = + let new_adjusted = map_requirement_before_projection(parent_required, expr); - if new_requirement.is_some() { + if new_adjusted.is_some() { Ok(Some(PlanWithSortRequirements { - plan: plan.clone(), - impact_result_ordering: requirements.impact_result_ordering, - satisfy_single_distribution: requirements - .satisfy_single_distribution, required_ordering: None, - adjusted_request_ordering: vec![new_requirement], + adjusted_request_ordering: vec![new_adjusted], + ..requirements })) } else { // Can not push down, add new SortExec @@ -624,16 +600,14 @@ fn ensure_sorting( } } else { Ok(Some(PlanWithSortRequirements { - plan: plan.clone(), - impact_result_ordering: requirements.impact_result_ordering, required_ordering: None, - satisfy_single_distribution: requirements.satisfy_single_distribution, adjusted_request_ordering: vec![ requirements.required_ordering; requirements .adjusted_request_ordering .len() ], + ..requirements })) } } @@ -722,6 +696,31 @@ fn analyze_immediate_spm_removal( } } +/// Determine the children requirements +/// If the children requirements are more specific, do not push down the parent requirements +/// If the the parent requirements are more specific, push down the parent requirements +/// If they are not compatible, need to add Sort. +fn determine_children_requirement( + parent_required: Option<&[PhysicalSortRequirements]>, + request_child: Option<&[PhysicalSortRequirements]>, + child_plan: Arc, +) -> RequirementsCompatibility { + if requirements_compatible(request_child, parent_required, || { + child_plan.equivalence_properties() + }) { + // request child requirements are more specific, no need to push down the parent requirements + RequirementsCompatibility::Satisfy + } else if requirements_compatible(parent_required, request_child, || { + child_plan.equivalence_properties() + }) { + // parent requirements are more specific, adjust the request child requirements and push down the new requirements + let adjusted = parent_required.map(|r| r.to_vec()); + RequirementsCompatibility::Compatible(adjusted) + } else { + RequirementsCompatibility::NonCompatible + } +} + /// Compares window expression's `window_request` and `parent_required_expr` ordering, returns /// whether we should reverse the window expression's ordering in order to meet parent's requirements. fn check_alignment( @@ -921,52 +920,41 @@ fn extract_window_info_from_plan( fn try_pushdown_requirements_to_join( requirements: &PlanWithSortRequirements, - plan: &Arc, parent_required: Option<&[PhysicalSortRequirements]>, - parent_required_expr: Vec, + sort_expr: Vec, push_side: JoinSide, ) -> Result> { let child_idx = match push_side { JoinSide::Left => 0, JoinSide::Right => 1, }; - if requirements_compatible( - plan.required_input_ordering()[child_idx].as_deref(), - parent_required, - || plan.children()[child_idx].equivalence_properties(), - ) { - // parent requirements are compatible with the SortMergeJoinExec - Ok(None) - } else if requirements_compatible( - parent_required, - plan.required_input_ordering()[child_idx].as_deref(), - || plan.children()[child_idx].equivalence_properties(), - ) { - // parent requirements are more specific, adjust the SortMergeJoinExec child requirements and push down the new requirements - let new_adjusted = match push_side { - JoinSide::Left => vec![ - parent_required.map(|r| r.to_vec()), - requirements.adjusted_request_ordering[1].clone(), - ], - JoinSide::Right => vec![ - requirements.adjusted_request_ordering[0].clone(), - parent_required.map(|r| r.to_vec()), - ], - }; - Ok(Some(PlanWithSortRequirements { - plan: plan.clone(), - impact_result_ordering: requirements.impact_result_ordering, - satisfy_single_distribution: requirements.satisfy_single_distribution, - required_ordering: None, - adjusted_request_ordering: new_adjusted, - })) - } else { - // Can not push down, add new SortExec - let mut new_plan = plan.clone(); - add_sort_above(&mut new_plan, parent_required_expr)?; - Ok(Some( - PlanWithSortRequirements::new_without_impact_result_ordering(new_plan), - )) + let request_child = requirements.adjusted_request_ordering[child_idx].as_deref(); + let child_plan = requirements.plan.children()[child_idx].clone(); + match determine_children_requirement(parent_required, request_child, child_plan) { + RequirementsCompatibility::Satisfy => Ok(None), + RequirementsCompatibility::Compatible(adjusted) => { + let new_adjusted = match push_side { + JoinSide::Left => { + vec![adjusted, requirements.adjusted_request_ordering[1].clone()] + } + JoinSide::Right => { + vec![requirements.adjusted_request_ordering[0].clone(), adjusted] + } + }; + Ok(Some(PlanWithSortRequirements { + required_ordering: None, + adjusted_request_ordering: new_adjusted, + ..requirements.clone() + })) + } + RequirementsCompatibility::NonCompatible => { + // Can not push down, add new SortExec + let mut new_plan = requirements.plan.clone(); + add_sort_above(&mut new_plan, sort_expr)?; + Ok(Some( + PlanWithSortRequirements::new_without_impact_result_ordering(new_plan), + )) + } } } @@ -1082,14 +1070,25 @@ fn shift_right_required( } } -#[derive(Debug)] /// This structure stores extra Window information required to create a new WindowExec +#[derive(Debug)] pub struct WindowExecInfo { window_expr: Vec>, input_schema: SchemaRef, partition_keys: Vec>, } +/// Define the Requirements Compatibility +#[derive(Debug)] +pub enum RequirementsCompatibility { + /// Requirements satisfy + Satisfy, + /// Requirements compatible + Compatible(Option>), + /// Requirements not compatible + NonCompatible, +} + /// A TombStoneExec execution plan generated during optimization process, should be removed finally #[derive(Debug)] struct TombStoneExec { From 56a855da72fe495e7e79f10b225d4faed2b92061 Mon Sep 17 00:00:00 2001 From: "mingmwang@ebay.com" Date: Mon, 20 Feb 2023 15:06:07 +0800 Subject: [PATCH 10/13] fix removing SortPreservingMergeExec+SortExec --- .../physical_optimizer/sort_enforcement2.rs | 63 ++++++++++++++++++- 1 file changed, 61 insertions(+), 2 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/sort_enforcement2.rs b/datafusion/core/src/physical_optimizer/sort_enforcement2.rs index e2d5e1d8c7251..7c81f6316f18a 100644 --- a/datafusion/core/src/physical_optimizer/sort_enforcement2.rs +++ b/datafusion/core/src/physical_optimizer/sort_enforcement2.rs @@ -140,7 +140,13 @@ impl PlanWithSortRequirements { .map( |(child, from_parent, maintains_input_order, required_dist)| { let child_satisfy_single_distribution = - matches!(required_dist, Distribution::SinglePartition); + matches!(required_dist, Distribution::SinglePartition) + || (self.satisfy_single_distribution + && self + .plan + .as_any() + .downcast_ref::() + .is_none()); let child_impact_result_ordering = if self .plan .as_any() @@ -637,7 +643,6 @@ fn analyze_immediate_sort_removal( // Remove unnecessary SortExec else if !requirements.impact_result_ordering { if requirements.satisfy_single_distribution - && !sort_exec.preserve_partitioning() && sort_exec.input().output_partitioning().partition_count() > 1 { Some(PlanWithSortRequirements { @@ -1755,6 +1760,60 @@ mod tests { Ok(()) } + #[tokio::test] + async fn test_remove_unnecessary_sort6() -> 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 = vec![sort_expr("non_nullable_col", &schema)]; + let sort = sort_exec(sort_exprs.clone(), union); + 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 = vec![ + "SortExec: expr=[nullable_col@0 ASC,non_nullable_col@1 ASC], global=true", + " FilterExec: NOT non_nullable_col@1", + " SortPreservingMergeExec: [non_nullable_col@1 ASC]", + " SortExec: expr=[non_nullable_col@1 ASC], global=true", + " UnionExec", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=0", + " MemoryExec: partitions=0, partition_sizes=[]", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=0", + " MemoryExec: partitions=0, partition_sizes=[]", + ]; + + let expected_optimized = vec![ + "SortExec: expr=[nullable_col@0 ASC,non_nullable_col@1 ASC], global=true", + " FilterExec: NOT non_nullable_col@1", + " CoalescePartitionsExec", + " UnionExec", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=0", + " MemoryExec: partitions=0, partition_sizes=[]", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=0", + " MemoryExec: partitions=0, partition_sizes=[]", + ]; + assert_optimized!(expected_input, expected_optimized, physical_plan); + Ok(()) + } + #[tokio::test] async fn test_remove_unnecessary_spm1() -> Result<()> { let schema = create_test_schema()?; From 6ce1ebb473bb509c4a5b4161273c75c3bfc3610f Mon Sep 17 00:00:00 2001 From: "mingmwang@ebay.com" Date: Thu, 23 Feb 2023 13:12:08 +0800 Subject: [PATCH 11/13] resolve review comments --- .../physical_optimizer/sort_enforcement.rs | 4 +- .../physical_optimizer/sort_enforcement2.rs | 267 ++++++++---------- .../core/src/physical_optimizer/utils.rs | 2 +- datafusion/physical-expr/src/utils.rs | 60 ++-- 4 files changed, 158 insertions(+), 175 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/sort_enforcement.rs b/datafusion/core/src/physical_optimizer/sort_enforcement.rs index 06b17b4507bd8..914e663299d40 100644 --- a/datafusion/core/src/physical_optimizer/sort_enforcement.rs +++ b/datafusion/core/src/physical_optimizer/sort_enforcement.rs @@ -445,7 +445,7 @@ fn ensure_sorting( let is_ordering_satisfied = ordering_satisfy_requirement_concrete( physical_ordering, &required_ordering, - || child.equivalence_properties(), + &|| child.equivalence_properties(), ); if !is_ordering_satisfied { // Make sure we preserve the ordering requirements: @@ -509,7 +509,7 @@ fn analyze_immediate_sort_removal( if ordering_satisfy( sort_input.output_ordering(), sort_exec.output_ordering(), - || sort_input.equivalence_properties(), + &|| sort_input.equivalence_properties(), ) { // Since we know that a `SortExec` has exactly one child, // we can use the zero index safely: diff --git a/datafusion/core/src/physical_optimizer/sort_enforcement2.rs b/datafusion/core/src/physical_optimizer/sort_enforcement2.rs index 7c81f6316f18a..68d372e0af3cb 100644 --- a/datafusion/core/src/physical_optimizer/sort_enforcement2.rs +++ b/datafusion/core/src/physical_optimizer/sort_enforcement2.rs @@ -60,7 +60,8 @@ use datafusion_expr::JoinType; use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::utils::{ create_sort_expr_from_requirement, map_requirement_before_projection, - ordering_satisfy, ordering_satisfy_requirement, requirements_compatible, + ordering_satisfy, ordering_satisfy_requirement, required_provided_either_compatible, + requirements_compatible, }; use datafusion_physical_expr::window::WindowExpr; use datafusion_physical_expr::{ @@ -104,8 +105,7 @@ impl PlanWithSortRequirements { pub fn init(plan: Arc) -> Self { let impact_result_ordering = plan.output_ordering().is_some() || plan.output_partitioning().partition_count() <= 1 - || plan.as_any().downcast_ref::().is_some() - || plan.as_any().downcast_ref::().is_some(); + || is_limit(&plan); let request_ordering = plan.required_input_ordering(); PlanWithSortRequirements { plan, @@ -147,17 +147,7 @@ impl PlanWithSortRequirements { .as_any() .downcast_ref::() .is_none()); - let child_impact_result_ordering = if self - .plan - .as_any() - .downcast_ref::() - .is_some() - || self - .plan - .as_any() - .downcast_ref::() - .is_some() - { + let child_impact_result_ordering = if is_limit(&self.plan) { true } else { maintains_input_order && self.impact_result_ordering @@ -221,7 +211,7 @@ impl PhysicalOptimizerRule for TopDownEnforceSorting { if ordering_satisfy( sort_exec.input().output_ordering(), sort_exec.output_ordering(), - || sort_exec.input().equivalence_properties(), + &|| sort_exec.input().equivalence_properties(), ) { Ok(Some(Arc::new(TombStoneExec::new( sort_exec.input().clone(), @@ -253,6 +243,18 @@ impl PhysicalOptimizerRule for TopDownEnforceSorting { } } +/// Checks whether the given plan is a limit plan; +/// i.e. either a `LocalLimitExec` or a `GlobalLimitExec`. +fn is_limit(plan: &Arc) -> bool { + plan.as_any().is::() || plan.as_any().is::() +} + +/// Checks whether the given plan is a window plan; +/// i.e. either a `WindowAggExec` or a `BoundedWindowAggExec`. +fn is_window(plan: &Arc) -> bool { + plan.as_any().is::() || plan.as_any().is::() +} + fn ensure_sorting( requirements: PlanWithSortRequirements, ) -> Result> { @@ -266,7 +268,7 @@ fn ensure_sorting( .as_any() .downcast_ref::() { - // SortPreservingMergeExec + SortExec(local/global) is the same as the global SortExec + // SortExec(local/global) -> SortPreservingMergeExe is the same as the global SortExec // Remove unnecessary SortPreservingMergeExec + SortExec(local/global) if let Some(child_sort_exec) = sort_pres_exec.input().as_any().downcast_ref::() @@ -294,7 +296,7 @@ fn ensure_sorting( } let plan = &requirements.plan; let parent_required = requirements.required_ordering.as_deref(); - if ordering_satisfy_requirement(plan.output_ordering(), parent_required, || { + if ordering_satisfy_requirement(plan.output_ordering(), parent_required, &|| { plan.equivalence_properties() }) { // Can satisfy the parent requirements, change the adjusted_request_ordering for UnionExec and WindowAggExec(BoundedWindowAggExec) @@ -312,12 +314,7 @@ fn ensure_sorting( ], ..requirements })); - } else if plan.as_any().downcast_ref::().is_some() - || plan - .as_any() - .downcast_ref::() - .is_some() - { + } else if is_window(plan) { // WindowAggExec(BoundedWindowAggExec) might reverse their sort requirements let request_child = requirements.adjusted_request_ordering[0].as_deref(); let reversed_request_child = reverse_window_sort_requirements(request_child); @@ -331,7 +328,7 @@ fn ensure_sorting( window_expr, input_schema, partition_keys, - } = extract_window_info_from_plan(plan).unwrap(); + } = extract_window_info_from_plan(plan)?; let new_window_expr = window_expr .iter() @@ -396,10 +393,9 @@ fn ensure_sorting( // For UnionExec, we can always push down if (maintains_input_order.is_empty() || !maintains_input_order.iter().any(|o| *o) - || plan.as_any().downcast_ref::().is_some() - || plan.as_any().downcast_ref::().is_some() - || plan.as_any().downcast_ref::().is_some() - || plan.as_any().downcast_ref::().is_some()) + || plan.as_any().is::() + || plan.as_any().is::() + || is_limit(plan)) && plan.as_any().downcast_ref::().is_none() { let mut new_plan = plan.clone(); @@ -407,12 +403,7 @@ fn ensure_sorting( Ok(Some( PlanWithSortRequirements::new_without_impact_result_ordering(new_plan), )) - } else if plan.as_any().downcast_ref::().is_some() - || plan - .as_any() - .downcast_ref::() - .is_some() - { + } else if is_window(plan) { let request_child = requirements.adjusted_request_ordering[0].as_deref(); let child_plan = plan.children()[0].clone(); match determine_children_requirement( @@ -433,12 +424,12 @@ fn ensure_sorting( window_expr, input_schema, partition_keys, - } = extract_window_info_from_plan(plan).unwrap(); + } = extract_window_info_from_plan(plan)?; if should_reverse_window_exec( parent_required, request_child, &input_schema, - ) { + )? { let new_physical_ordering = parent_required_expr.to_vec(); let new_window_expr = window_expr .iter() @@ -630,7 +621,7 @@ fn analyze_immediate_sort_removal( if ordering_satisfy( sort_exec.input().output_ordering(), sort_exec.output_ordering(), - || sort_exec.input().equivalence_properties(), + &|| sort_exec.input().equivalence_properties(), ) { Some(PlanWithSortRequirements { plan: Arc::new(TombStoneExec::new(sort_exec.input().clone())), @@ -676,7 +667,7 @@ fn analyze_immediate_spm_removal( if ordering_satisfy( spm_exec.input().output_ordering(), Some(spm_exec.expr()), - || spm_exec.input().equivalence_properties(), + &|| spm_exec.input().equivalence_properties(), ) && spm_exec.input().output_partitioning().partition_count() <= 1 { Some(PlanWithSortRequirements { @@ -710,12 +701,12 @@ fn determine_children_requirement( request_child: Option<&[PhysicalSortRequirements]>, child_plan: Arc, ) -> RequirementsCompatibility { - if requirements_compatible(request_child, parent_required, || { + if requirements_compatible(request_child, parent_required, &|| { child_plan.equivalence_properties() }) { // request child requirements are more specific, no need to push down the parent requirements RequirementsCompatibility::Satisfy - } else if requirements_compatible(parent_required, request_child, || { + } else if requirements_compatible(parent_required, request_child, &|| { child_plan.equivalence_properties() }) { // parent requirements are more specific, adjust the request child requirements and push down the new requirements @@ -732,22 +723,25 @@ fn check_alignment( input_schema: &SchemaRef, window_request: &PhysicalSortRequirements, parent_required_expr: &PhysicalSortRequirements, -) -> bool { - if parent_required_expr.expr.eq(&window_request.expr) - && window_request.sort_options.is_some() - && parent_required_expr.sort_options.is_some() - { - let nullable = parent_required_expr.expr.nullable(input_schema).unwrap(); - let window_request_opts = window_request.sort_options.unwrap(); - let parent_required_opts = parent_required_expr.sort_options.unwrap(); - if nullable { - window_request_opts == reverse_sort_options(parent_required_opts) +) -> Result { + if parent_required_expr.expr.eq(&window_request.expr) { + let nullable = parent_required_expr.expr.nullable(input_schema)?; + if let Some(window_request_opts) = window_request.sort_options { + if let Some(parent_required_opts) = parent_required_expr.sort_options { + if nullable { + Ok(window_request_opts == reverse_sort_options(parent_required_opts)) + } else { + // If the column is not nullable, NULLS FIRST/LAST is not important. + Ok(window_request_opts.descending != parent_required_opts.descending) + } + } else { + Ok(false) + } } else { - // If the column is not nullable, NULLS FIRST/LAST is not important. - window_request_opts.descending != parent_required_opts.descending + Ok(false) } } else { - false + Ok(false) } } @@ -788,97 +782,70 @@ fn should_reverse_window_sort_requirements( if reverse_window_expr.is_none() { return false; } - let flags = window_plan - .children() - .into_iter() - .map(|child| { - // If the child is leaf node, check the output ordering - if child.children().is_empty() - && ordering_satisfy_requirement( - child.output_ordering(), - top_requirement, - || child.equivalence_properties(), - ) - { - false - } else if child.children().is_empty() - && ordering_satisfy_requirement( - child.output_ordering(), - top_reversed_requirement, - || child.equivalence_properties(), - ) - { - true - } else if child.as_any().downcast_ref::().is_some() - || child - .as_any() - .downcast_ref::() - .is_some() - { - // If the child is WindowExec, check the child requirements - if requirements_compatible( - top_requirement, - child.required_input_ordering()[0].as_deref(), - || child.equivalence_properties(), - ) || requirements_compatible( - child.required_input_ordering()[0].as_deref(), - top_requirement, - || child.equivalence_properties(), - ) || requirements_compatible( - top_reversed_requirement, - child.required_input_ordering()[0].as_deref(), - || child.equivalence_properties(), - ) || requirements_compatible( - child.required_input_ordering()[0].as_deref(), - top_reversed_requirement, - || child.equivalence_properties(), - ) { - should_reverse_window_sort_requirements( - child, - top_requirement, - top_reversed_requirement, - ) - } else { - requirements_compatible( - top_reversed_requirement, - window_plan.required_input_ordering()[0].as_deref(), - || window_plan.equivalence_properties(), - ) || requirements_compatible( - window_plan.required_input_ordering()[0].as_deref(), - top_reversed_requirement, - || window_plan.equivalence_properties(), - ) - } - } else { - requirements_compatible( - top_reversed_requirement, - window_plan.required_input_ordering()[0].as_deref(), - || window_plan.equivalence_properties(), - ) || requirements_compatible( - window_plan.required_input_ordering()[0].as_deref(), - top_reversed_requirement, - || window_plan.equivalence_properties(), - ) - } + // Since we know that a window plan has exactly one child, + // we can use the zero index safely: + let window_child = window_plan.children()[0].clone(); + let window_child_output = window_child.output_ordering(); + // If the child is leaf node, check the output ordering + if window_child.children().is_empty() + && ordering_satisfy_requirement(window_child_output, top_requirement, &|| { + window_child.equivalence_properties() }) - .collect::>(); - - flags.iter().all(|o| *o) + { + false + } else if window_child.children().is_empty() + && ordering_satisfy_requirement( + window_child_output, + top_reversed_requirement, + &|| window_child.equivalence_properties(), + ) + { + true + } else if is_window(&window_child) { + // If the child is WindowExec, check the child requirements + if required_provided_either_compatible( + top_requirement, + window_child.required_input_ordering()[0].as_deref(), + &|| window_child.equivalence_properties(), + ) || required_provided_either_compatible( + top_reversed_requirement, + window_child.required_input_ordering()[0].as_deref(), + &|| window_child.equivalence_properties(), + ) { + should_reverse_window_sort_requirements( + window_child, + top_requirement, + top_reversed_requirement, + ) + } else { + required_provided_either_compatible( + top_reversed_requirement, + window_plan.required_input_ordering()[0].as_deref(), + &|| window_plan.equivalence_properties(), + ) + } + } else { + required_provided_either_compatible( + top_reversed_requirement, + window_plan.required_input_ordering()[0].as_deref(), + &|| window_plan.equivalence_properties(), + ) + } } fn should_reverse_window_exec( required: Option<&[PhysicalSortRequirements]>, request_ordering: Option<&[PhysicalSortRequirements]>, input_schema: &SchemaRef, -) -> bool { +) -> Result { match (required, request_ordering) { - (_, None) => false, - (None, Some(_)) => false, + (_, None) => Ok(false), + (None, Some(_)) => Ok(false), (Some(required), Some(request_ordering)) => { if required.len() > request_ordering.len() { - return false; + return Ok(false); } - let alignment_flags = required + let alignment_flags_rs: Result> = required .iter() .zip(request_ordering.iter()) .filter_map(|(required_expr, request_expr)| { @@ -890,14 +857,15 @@ fn should_reverse_window_exec( } else if request_expr.expr.eq(&required_expr.expr) { None } else { - Some(false) + Some(Ok(false)) } }) - .collect::>(); + .collect(); + let alignment_flags = alignment_flags_rs?; if alignment_flags.is_empty() { - false + Ok(false) } else { - alignment_flags.iter().all(|o| *o) + Ok(alignment_flags.iter().all(|o| *o)) } } } @@ -905,9 +873,9 @@ fn should_reverse_window_exec( fn extract_window_info_from_plan( plan: &Arc, -) -> Option { +) -> Result { if let Some(exec) = plan.as_any().downcast_ref::() { - Some(WindowExecInfo { + Ok(WindowExecInfo { window_expr: exec.window_expr().to_vec(), input_schema: exec.input_schema(), partition_keys: exec.partition_keys.clone(), @@ -920,6 +888,12 @@ fn extract_window_info_from_plan( input_schema: exec.input_schema(), partition_keys: exec.partition_keys.clone(), }) + .ok_or_else(|| { + DataFusionError::Plan( + "Expects to receive either WindowAggExec of BoundedWindowAggExec" + .to_string(), + ) + }) } } @@ -1006,7 +980,7 @@ fn expr_source_sides( if required_exprs .iter() .filter_map(|r| { - if r.expr.as_any().downcast_ref::().is_some() { + if r.expr.as_any().is::() { Some(JoinSide::Left) } else { None @@ -1024,7 +998,7 @@ fn expr_source_sides( if required_exprs .iter() .filter_map(|r| { - if r.expr.as_any().downcast_ref::().is_some() { + if r.expr.as_any().is::() { Some(JoinSide::Right) } else { None @@ -1251,7 +1225,7 @@ mod tests { }), }; let reverse = - check_alignment(&schema, &physical_ordering, &required_ordering); + check_alignment(&schema, &physical_ordering, &required_ordering)?; assert_eq!(reverse, reverse_expected); } @@ -1291,7 +1265,7 @@ mod tests { }), }; let reverse = - check_alignment(&schema, &physical_ordering, &required_ordering); + check_alignment(&schema, &physical_ordering, &required_ordering)?; assert_eq!(reverse, reverse_expected); } @@ -1334,7 +1308,7 @@ mod tests { Some(required_ordering1.deref()), Some(window_request_ordering1.deref()), &schema, - ); + )?; assert!(reverse); // order by nullable_col, non_nullable_col @@ -1373,7 +1347,7 @@ mod tests { Some(required_ordering2.deref()), Some(window_request_ordering2.deref()), &schema, - ); + )?; assert!(reverse); // wrong partition columns @@ -1412,7 +1386,7 @@ mod tests { Some(required_ordering3.deref()), Some(window_request_ordering3.deref()), &schema, - ); + )?; assert!(!reverse); Ok(()) @@ -1552,7 +1526,6 @@ mod tests { let sort1 = sort_exec(sort_exprs1.clone(), source); let window_agg1 = window_exec("non_nullable_col", sort_exprs1.clone(), sort1); let window_agg2 = window_exec("non_nullable_col", sort_exprs2, window_agg1); - // let filter_exec = sort_exec; let physical_plan = window_exec("non_nullable_col", sort_exprs1, window_agg2); let expected_input = vec![ diff --git a/datafusion/core/src/physical_optimizer/utils.rs b/datafusion/core/src/physical_optimizer/utils.rs index b6666fbefae1e..c9a9cc2d9cbb8 100644 --- a/datafusion/core/src/physical_optimizer/utils.rs +++ b/datafusion/core/src/physical_optimizer/utils.rs @@ -56,7 +56,7 @@ pub fn add_sort_above( sort_expr: Vec, ) -> Result<()> { // If the ordering requirement is already satisfied, do not add a sort. - if !ordering_satisfy(node.output_ordering(), Some(&sort_expr), || { + if !ordering_satisfy(node.output_ordering(), Some(&sort_expr), &|| { node.equivalence_properties() }) { *node = Arc::new(if node.output_partitioning().partition_count() > 1 { diff --git a/datafusion/physical-expr/src/utils.rs b/datafusion/physical-expr/src/utils.rs index fa9b8d134efde..3e14b8cc524f6 100644 --- a/datafusion/physical-expr/src/utils.rs +++ b/datafusion/physical-expr/src/utils.rs @@ -28,6 +28,7 @@ use arrow::datatypes::SchemaRef; use arrow_schema::SortOptions; use std::collections::HashMap; +use std::ops::Deref; use std::sync::Arc; /// Compare the two expr lists are equal no matter the order. @@ -219,10 +220,10 @@ pub fn normalize_sort_requirement_with_equivalence_properties( } /// Checks whether the required [PhysicalSortExpr]s are satisfied by the provided [PhysicalSortExpr]s. -pub fn ordering_satisfy EquivalenceProperties>( +pub fn ordering_satisfy EquivalenceProperties>( provided: Option<&[PhysicalSortExpr]>, required: Option<&[PhysicalSortExpr]>, - equal_properties: F, + equal_properties: &F, ) -> bool { match (provided, required) { (_, None) => true, @@ -233,10 +234,10 @@ pub fn ordering_satisfy EquivalenceProperties>( } } -fn ordering_satisfy_concrete EquivalenceProperties>( +fn ordering_satisfy_concrete EquivalenceProperties>( provided: &[PhysicalSortExpr], required: &[PhysicalSortExpr], - equal_properties: F, + equal_properties: &F, ) -> bool { if required.len() > provided.len() { false @@ -246,7 +247,9 @@ fn ordering_satisfy_concrete EquivalenceProperties>( .all(|(order1, order2)| order1.eq(order2)) { true - } else if let eq_classes @ [_, ..] = equal_properties().classes() { + } else { + let eq_props = equal_properties(); + let eq_classes = eq_props.classes(); let normalized_required_exprs = required .iter() .map(|e| { @@ -263,16 +266,14 @@ fn ordering_satisfy_concrete EquivalenceProperties>( .iter() .zip(normalized_provided_exprs.iter()) .all(|(order1, order2)| order1.eq(order2)) - } else { - false } } /// Checks whether the required ordering requirements are satisfied by the provided [PhysicalSortExpr]s. -pub fn ordering_satisfy_requirement EquivalenceProperties>( +pub fn ordering_satisfy_requirement EquivalenceProperties>( provided: Option<&[PhysicalSortExpr]>, required: Option<&[PhysicalSortRequirements]>, - equal_properties: F, + equal_properties: &F, ) -> bool { match (provided, required) { (_, None) => true, @@ -283,10 +284,10 @@ pub fn ordering_satisfy_requirement EquivalenceProperties>( } } -pub fn ordering_satisfy_requirement_concrete EquivalenceProperties>( +pub fn ordering_satisfy_requirement_concrete EquivalenceProperties>( provided: &[PhysicalSortExpr], required: &[PhysicalSortRequirements], - equal_properties: F, + equal_properties: &F, ) -> bool { if required.len() > provided.len() { false @@ -296,7 +297,9 @@ pub fn ordering_satisfy_requirement_concrete EquivalencePropertie .all(|(order1, order2)| order2.satisfy(order1)) { true - } else if let eq_classes @ [_, ..] = equal_properties().classes() { + } else { + let eq_props = equal_properties(); + let eq_classes = eq_props.classes(); let normalized_requirements = required .iter() .map(|e| { @@ -316,16 +319,14 @@ pub fn ordering_satisfy_requirement_concrete EquivalencePropertie .iter() .zip(normalized_provided_exprs.iter()) .all(|(order1, order2)| order2.satisfy(order1)) - } else { - false } } /// Provided requirements are compatible with the required, which means the provided requirements are equal or more specific than the required -pub fn requirements_compatible EquivalenceProperties>( +pub fn requirements_compatible EquivalenceProperties>( provided: Option<&[PhysicalSortRequirements]>, required: Option<&[PhysicalSortRequirements]>, - equal_properties: F, + equal_properties: &F, ) -> bool { match (provided, required) { (_, None) => true, @@ -339,13 +340,15 @@ pub fn requirements_compatible EquivalenceProperties>( .all(|(req, pro)| pro.compatible(req)) { true - } else if let eq_classes @ [_, ..] = equal_properties().classes() { + } else { + let eq_props = equal_properties(); + let eq_classes = eq_props.classes(); let normalized_required = required .iter() .map(|e| { normalize_sort_requirement_with_equivalence_properties( e.clone(), - eq_classes, + eq_classes.deref(), ) }) .collect::>(); @@ -354,7 +357,7 @@ pub fn requirements_compatible EquivalenceProperties>( .map(|e| { normalize_sort_requirement_with_equivalence_properties( e.clone(), - eq_classes, + eq_classes.deref(), ) }) .collect::>(); @@ -362,13 +365,20 @@ pub fn requirements_compatible EquivalenceProperties>( .iter() .zip(normalized_provided.iter()) .all(|(req, pro)| pro.compatible(req)) - } else { - false } } } } +pub fn required_provided_either_compatible EquivalenceProperties>( + provided: Option<&[PhysicalSortRequirements]>, + required: Option<&[PhysicalSortRequirements]>, + equal_properties: &F, +) -> bool { + requirements_compatible(provided, required, equal_properties) + || requirements_compatible(required, provided, equal_properties) +} + pub fn map_columns_before_projection( parent_required: &[Arc], proj_exprs: &[(Arc, String)], @@ -427,10 +437,10 @@ pub fn create_sort_expr_from_requirement( let parent_required_expr = required .iter() .map(|prop| { - if prop.sort_options.is_some() { + if let Some(sort_options) = prop.sort_options { PhysicalSortExpr { expr: prop.expr.clone(), - options: prop.sort_options.unwrap(), + options: sort_options, } } else { PhysicalSortExpr { @@ -629,10 +639,10 @@ mod tests { fields: vec![], metadata: Default::default(), }); - assert!(ordering_satisfy(finer, crude, || { + assert!(ordering_satisfy(finer, crude, &|| { EquivalenceProperties::new(empty_schema.clone()) })); - assert!(!ordering_satisfy(crude, finer, || { + assert!(!ordering_satisfy(crude, finer, &|| { EquivalenceProperties::new(empty_schema.clone()) })); Ok(()) From c9e3e0ed0afd994b695e482d6cf917b1b6357ad7 Mon Sep 17 00:00:00 2001 From: "mingmwang@ebay.com" Date: Thu, 23 Feb 2023 13:15:04 +0800 Subject: [PATCH 12/13] resolve review comments --- datafusion/physical-expr/src/sort_expr.rs | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/datafusion/physical-expr/src/sort_expr.rs b/datafusion/physical-expr/src/sort_expr.rs index a25802e9980ab..8e0b3fb98e784 100644 --- a/datafusion/physical-expr/src/sort_expr.rs +++ b/datafusion/physical-expr/src/sort_expr.rs @@ -71,9 +71,8 @@ impl PhysicalSortExpr { } pub fn satisfy(&self, requirement: &PhysicalSortRequirements) -> bool { - if requirement.sort_options.is_some() { - self.options == requirement.sort_options.unwrap() - && self.expr.eq(&requirement.expr) + if let Some(sort_options) = requirement.sort_options { + self.options == sort_options && self.expr.eq(&requirement.expr) } else { self.expr.eq(&requirement.expr) } From 60ace65df0b692e846ea431d30a10b79f0368852 Mon Sep 17 00:00:00 2001 From: "mingmwang@ebay.com" Date: Mon, 27 Feb 2023 22:55:47 +0800 Subject: [PATCH 13/13] remove unnecessary final SortPreservingMergeExec --- .../physical_optimizer/sort_enforcement2.rs | 123 +++++++++--------- datafusion/core/tests/sql/window.rs | 21 ++- 2 files changed, 73 insertions(+), 71 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/sort_enforcement2.rs b/datafusion/core/src/physical_optimizer/sort_enforcement2.rs index 68d372e0af3cb..2bf505a6ad985 100644 --- a/datafusion/core/src/physical_optimizer/sort_enforcement2.rs +++ b/datafusion/core/src/physical_optimizer/sort_enforcement2.rs @@ -219,6 +219,21 @@ impl PhysicalOptimizerRule for TopDownEnforceSorting { } else { Ok(None) } + } else if let Some(sort_pres_exec) = + plan.as_any().downcast_ref::() + { + if sort_pres_exec + .input() + .output_partitioning() + .partition_count() + <= 1 + { + Ok(Some(Arc::new(TombStoneExec::new( + sort_pres_exec.input().clone(), + )))) + } else { + Ok(None) + } } else { Ok(None) } @@ -1561,9 +1576,8 @@ mod tests { " MemoryExec: partitions=0, partition_sizes=[]", ]; let expected_optimized = vec![ - "SortPreservingMergeExec: [nullable_col@0 ASC]", - " SortExec: expr=[nullable_col@0 ASC], global=true", - " MemoryExec: partitions=0, partition_sizes=[]", + "SortExec: expr=[nullable_col@0 ASC], global=true", + " MemoryExec: partitions=0, partition_sizes=[]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan); Ok(()) @@ -1588,9 +1602,8 @@ mod tests { " MemoryExec: partitions=0, partition_sizes=[]", ]; let expected_optimized = vec![ - "SortPreservingMergeExec: [nullable_col@0 ASC]", - " SortExec: expr=[nullable_col@0 ASC], global=true", - " MemoryExec: partitions=0, partition_sizes=[]", + "SortExec: expr=[nullable_col@0 ASC], global=true", + " MemoryExec: partitions=0, partition_sizes=[]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan); Ok(()) @@ -1695,9 +1708,8 @@ mod tests { " MemoryExec: partitions=0, partition_sizes=[]", ]; let expected_optimized = vec![ - "SortPreservingMergeExec: [nullable_col@0 ASC]", - " SortExec: expr=[nullable_col@0 ASC], global=true", - " MemoryExec: partitions=0, partition_sizes=[]", + "SortExec: expr=[nullable_col@0 ASC], global=true", + " MemoryExec: partitions=0, partition_sizes=[]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan); Ok(()) @@ -1832,9 +1844,8 @@ mod tests { " MemoryExec: partitions=0, partition_sizes=[]", ]; let expected_optimized = vec![ - "SortPreservingMergeExec: [nullable_col@0 ASC]", - " SortExec: expr=[nullable_col@0 ASC], global=true", - " MemoryExec: partitions=0, partition_sizes=[]", + "SortExec: expr=[nullable_col@0 ASC], global=true", + " MemoryExec: partitions=0, partition_sizes=[]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan); Ok(()) @@ -1902,9 +1913,8 @@ mod tests { " MemoryExec: partitions=0, partition_sizes=[]", ]; let expected_optimized = vec![ - "SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", - " SortExec: expr=[nullable_col@0 ASC,non_nullable_col@1 ASC], global=true", - " MemoryExec: partitions=0, partition_sizes=[]", + "SortExec: expr=[nullable_col@0 ASC,non_nullable_col@1 ASC], global=true", + " MemoryExec: partitions=0, partition_sizes=[]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan); Ok(()) @@ -1930,9 +1940,8 @@ mod tests { " MemoryExec: partitions=0, partition_sizes=[]", ]; let expected_optimized = vec![ - "SortPreservingMergeExec: [non_nullable_col@1 ASC]", - " SortExec: expr=[non_nullable_col@1 ASC], global=true", - " MemoryExec: partitions=0, partition_sizes=[]", + "SortExec: expr=[non_nullable_col@1 ASC], global=true", + " MemoryExec: partitions=0, partition_sizes=[]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan); Ok(()) @@ -2482,7 +2491,7 @@ mod tests { let join_plan = format!(" SortMergeJoin: join_type={join_type}, on=[(Column {{ name: \"nullable_col\", index: 0 }}, Column {{ name: \"col_a\", index: 0 }})]"); let join_plan2 = - format!(" SortMergeJoin: join_type={join_type}, on=[(Column {{ name: \"nullable_col\", index: 0 }}, Column {{ name: \"col_a\", index: 0 }})]"); + format!("SortMergeJoin: join_type={join_type}, on=[(Column {{ name: \"nullable_col\", index: 0 }}, Column {{ name: \"col_a\", index: 0 }})]"); let expected_input = vec![ "SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", @@ -2497,24 +2506,22 @@ mod tests { | JoinType::LeftAnti => { // can push down the sort requirements and save 1 SortExec vec![ - "SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", - join_plan.as_str(), - " SortExec: expr=[nullable_col@0 ASC,non_nullable_col@1 ASC], global=true", - " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " SortExec: expr=[col_a@0 ASC], global=true", - " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[col_a, col_b]", + join_plan2.as_str(), + " SortExec: expr=[nullable_col@0 ASC,non_nullable_col@1 ASC], global=true", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " SortExec: expr=[col_a@0 ASC], global=true", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[col_a, col_b]", ] } _ => { // can not push down the sort requirements vec![ - "SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", - " SortExec: expr=[nullable_col@0 ASC,non_nullable_col@1 ASC], global=true", - join_plan2.as_str(), - " SortExec: expr=[nullable_col@0 ASC], global=true", - " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " SortExec: expr=[col_a@0 ASC], global=true", - " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[col_a, col_b]", + "SortExec: expr=[nullable_col@0 ASC,non_nullable_col@1 ASC], global=true", + join_plan.as_str(), + " SortExec: expr=[nullable_col@0 ASC], global=true", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " SortExec: expr=[col_a@0 ASC], global=true", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[col_a, col_b]", ] } }; @@ -2562,7 +2569,7 @@ mod tests { _ => "SortPreservingMergeExec: [col_a@2 ASC,col_b@3 ASC]", }; let join_plan2 = - format!(" SortMergeJoin: join_type={join_type}, on=[(Column {{ name: \"nullable_col\", index: 0 }}, Column {{ name: \"col_a\", index: 0 }})]"); + format!("SortMergeJoin: join_type={join_type}, on=[(Column {{ name: \"nullable_col\", index: 0 }}, Column {{ name: \"col_a\", index: 0 }})]"); let expected_input = vec![ spm_plan, @@ -2574,24 +2581,22 @@ mod tests { JoinType::Inner | JoinType::Right | JoinType::RightAnti => { // can push down the sort requirements and save 1 SortExec vec![ - spm_plan, - join_plan.as_str(), - " SortExec: expr=[nullable_col@0 ASC], global=true", - " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " SortExec: expr=[col_a@0 ASC,col_b@1 ASC], global=true", - " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[col_a, col_b]", + join_plan2.as_str(), + " SortExec: expr=[nullable_col@0 ASC], global=true", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " SortExec: expr=[col_a@0 ASC,col_b@1 ASC], global=true", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[col_a, col_b]", ] } _ => { // can not push down the sort requirements for Left and Full join. vec![ - spm_plan, - " SortExec: expr=[col_a@2 ASC,col_b@3 ASC], global=true", - join_plan2.as_str(), - " SortExec: expr=[nullable_col@0 ASC], global=true", - " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " SortExec: expr=[col_a@0 ASC], global=true", - " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[col_a, col_b]", + "SortExec: expr=[col_a@2 ASC,col_b@3 ASC], global=true", + join_plan.as_str(), + " SortExec: expr=[nullable_col@0 ASC], global=true", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " SortExec: expr=[col_a@0 ASC], global=true", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[col_a, col_b]", ] } }; @@ -2632,13 +2637,12 @@ mod tests { // can not push down the sort requirements, need to add SortExec let expected_optimized = vec![ - "SortPreservingMergeExec: [col_b@3 ASC,col_a@2 ASC]", - " SortExec: expr=[col_b@3 ASC,col_a@2 ASC], global=true", - " SortMergeJoin: join_type=Inner, on=[(Column { name: \"nullable_col\", index: 0 }, Column { name: \"col_a\", index: 0 })]", - " SortExec: expr=[nullable_col@0 ASC], global=true", - " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " SortExec: expr=[col_a@0 ASC], global=true", - " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[col_a, col_b]", + "SortExec: expr=[col_b@3 ASC,col_a@2 ASC], global=true", + " SortMergeJoin: join_type=Inner, on=[(Column { name: \"nullable_col\", index: 0 }, Column { name: \"col_a\", index: 0 })]", + " SortExec: expr=[nullable_col@0 ASC], global=true", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " SortExec: expr=[col_a@0 ASC], global=true", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[col_a, col_b]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan); @@ -2659,13 +2663,12 @@ mod tests { // can not push down the sort requirements, need to add SortExec let expected_optimized = vec![ - "SortPreservingMergeExec: [nullable_col@0 ASC,col_b@3 ASC,col_a@2 ASC]", - " SortExec: expr=[nullable_col@0 ASC,col_b@3 ASC,col_a@2 ASC], global=true", - " SortMergeJoin: join_type=Inner, on=[(Column { name: \"nullable_col\", index: 0 }, Column { name: \"col_a\", index: 0 })]", - " SortExec: expr=[nullable_col@0 ASC], global=true", - " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " SortExec: expr=[col_a@0 ASC], global=true", - " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[col_a, col_b]", + "SortExec: expr=[nullable_col@0 ASC,col_b@3 ASC,col_a@2 ASC], global=true", + " SortMergeJoin: join_type=Inner, on=[(Column { name: \"nullable_col\", index: 0 }, Column { name: \"col_a\", index: 0 })]", + " SortExec: expr=[nullable_col@0 ASC], global=true", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", + " SortExec: expr=[col_a@0 ASC], global=true", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[col_a, col_b]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan); diff --git a/datafusion/core/tests/sql/window.rs b/datafusion/core/tests/sql/window.rs index 684d6a6c313da..8d7039442da75 100644 --- a/datafusion/core/tests/sql/window.rs +++ b/datafusion/core/tests/sql/window.rs @@ -2131,17 +2131,16 @@ async fn test_window_agg_global_sort_intermediate_parallel_sort() -> Result<()> // Only 1 SortExec was added let expected = { vec![ - "SortPreservingMergeExec: [c1@0 ASC NULLS LAST]", - " SortExec: expr=[c1@0 ASC NULLS LAST], global=false", - " ProjectionExec: expr=[c1@0 as c1, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 3 FOLLOWING@2 as sum1, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@3 as sum2]", - " BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9): Ok(Field { name: \"SUM(aggregate_test_100.c9)\", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)) }]", - " SortPreservingMergeExec: [c9@1 ASC NULLS LAST]", - " SortExec: expr=[c9@1 ASC NULLS LAST], global=false", - " BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9): Ok(Field { name: \"SUM(aggregate_test_100.c9)\", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(3)) }]", - " SortExec: expr=[c1@0 ASC NULLS LAST,c9@1 ASC NULLS LAST], global=false", - " CoalesceBatchesExec: target_batch_size=8192", - " RepartitionExec: partitioning=Hash([Column { name: \"c1\", index: 0 }], 2), input_partitions=2", - " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", + "SortExec: expr=[c1@0 ASC NULLS LAST], global=false", + " ProjectionExec: expr=[c1@0 as c1, SUM(aggregate_test_100.c9) PARTITION BY [aggregate_test_100.c1] ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 3 FOLLOWING@2 as sum1, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@3 as sum2]", + " BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9): Ok(Field { name: \"SUM(aggregate_test_100.c9)\", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(5)) }]", + " SortPreservingMergeExec: [c9@1 ASC NULLS LAST]", + " SortExec: expr=[c9@1 ASC NULLS LAST], global=false", + " BoundedWindowAggExec: wdw=[SUM(aggregate_test_100.c9): Ok(Field { name: \"SUM(aggregate_test_100.c9)\", data_type: UInt64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(1)), end_bound: Following(UInt64(3)) }]", + " SortExec: expr=[c1@0 ASC NULLS LAST,c9@1 ASC NULLS LAST], global=false", + " CoalesceBatchesExec: target_batch_size=8192", + " RepartitionExec: partitioning=Hash([Column { name: \"c1\", index: 0 }], 2), input_partitions=2", + " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", ] };