diff --git a/datafusion/core/src/dataframe.rs b/datafusion/core/src/dataframe.rs index 36135bd1eb365..883db0563607f 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 8944b4f905034..f63b6df58fb13 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; @@ -1541,7 +1541,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 ae56b70bae4bb..7917339841c52 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::{ + 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 std::collections::HashMap; +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, + )?)) } }) } @@ -492,30 +502,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, @@ -843,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); @@ -853,31 +840,46 @@ 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 { + 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()))) } - _ => { - 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?) } @@ -969,7 +971,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 +1017,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 +1157,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::new(); + let optimizer = TopDownEnforceSorting {}; let optimized = optimizer.optimize(optimized, &config)?; // Now format correctly @@ -1657,6 +1680,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) @@ -1687,6 +1711,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) @@ -1776,6 +1801,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) @@ -1806,6 +1832,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) @@ -1873,7 +1900,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 +1908,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 +1939,7 @@ mod tests { )]; let top_join = sort_merge_join_exec( join.clone(), - parquet_exec(), + parquet_multiple_exec(), &top_join_on, &join_type, ); @@ -1925,33 +1952,33 @@ mod tests { vec![ top_join_plan.as_str(), join_plan.as_str(), - "SortExec: expr=[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: expr=[b1@1 ASC]", - "RepartitionExec: partitioning=Hash([Column { name: \"b1\", index: 1 }], 10), input_partitions=1", + "SortExec: expr=[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: expr=[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: expr=[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: expr=[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: expr=[a@0 ASC]", + "SortExec: expr=[a@0 ASC], global=false", "RepartitionExec: partitioning=Hash([Column { name: \"a\", index: 0 }], 10), input_partitions=10", join_plan.as_str(), - "SortExec: expr=[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: expr=[b1@1 ASC]", - "RepartitionExec: partitioning=Hash([Column { name: \"b1\", index: 1 }], 10), input_partitions=1", + "SortExec: expr=[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: expr=[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: expr=[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: expr=[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 +1993,7 @@ mod tests { )]; let top_join = sort_merge_join_exec( join, - parquet_exec(), + parquet_multiple_exec(), &top_join_on, &join_type, ); @@ -1978,33 +2005,33 @@ mod tests { JoinType::Inner | JoinType::Right => vec![ top_join_plan.as_str(), join_plan.as_str(), - "SortExec: expr=[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: expr=[b1@1 ASC]", - "RepartitionExec: partitioning=Hash([Column { name: \"b1\", index: 1 }], 10), input_partitions=1", + "SortExec: expr=[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: expr=[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: expr=[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: expr=[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: expr=[b1@6 ASC]", + "SortExec: expr=[b1@6 ASC], global=false", "RepartitionExec: partitioning=Hash([Column { name: \"b1\", index: 6 }], 10), input_partitions=10", join_plan.as_str(), - "SortExec: expr=[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: expr=[b1@1 ASC]", - "RepartitionExec: partitioning=Hash([Column { name: \"b1\", index: 1 }], 10), input_partitions=1", + "SortExec: expr=[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: expr=[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: expr=[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: expr=[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 +2092,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: expr=[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: expr=[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: expr=[b2@1 ASC,a2@0 ASC]", "ProjectionExec: expr=[a@1 as a2, b@0 as b2]", + "SortExec: expr=[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 +2130,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/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/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 b43b4f2088aeb..d1723d93a363d 100644 --- a/datafusion/core/src/physical_optimizer/repartition.rs +++ b/datafusion/core/src/physical_optimizer/repartition.rs @@ -330,6 +330,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)])) @@ -400,6 +401,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 { @@ -607,7 +635,7 @@ mod tests { "GlobalLimitExec: skip=0, fetch=100", "LocalLimitExec: fetch=100", // data is sorted so can't repartition here - "SortExec: expr=[c1@0 ASC]", + "SortExec: expr=[c1@0 ASC], global=true", "ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[c1]", ]; @@ -625,7 +653,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: expr=[c1@0 ASC]", + "SortExec: expr=[c1@0 ASC], global=true", "ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[c1]", ]; @@ -713,7 +741,7 @@ mod tests { // need repartiton and resort as the data was not sorted correctly let expected = &[ "SortPreservingMergeExec: [c1@0 ASC]", - "SortExec: expr=[c1@0 ASC]", + "SortExec: expr=[c1@0 ASC], global=false", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[c1]", ]; @@ -725,12 +753,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); @@ -813,7 +841,7 @@ mod tests { // needs to repartition / sort as the data was not sorted correctly let expected = &[ "SortPreservingMergeExec: [c1@0 ASC]", - "SortExec: expr=[c1@0 ASC]", + "SortExec: expr=[c1@0 ASC], global=false", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "ProjectionExec: expr=[c1@0 as c1]", "ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[c1]", @@ -826,13 +854,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); @@ -846,7 +875,8 @@ mod tests { let expected = &[ "SortPreservingMergeExec: [c1@0 ASC]", - "SortExec: expr=[c1@0 ASC]", + // Expect repartition on the input to the sort (as it can benefit from additional parallelism) + "SortExec: expr=[c1@0 ASC], global=false", "ProjectionExec: expr=[c1@0 as c1]", "ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[c1]", ]; @@ -863,7 +893,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: expr=[c1@0 ASC]", + "SortExec: expr=[c1@0 ASC], global=false", "FilterExec: c1@0", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", "ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[c1]", @@ -883,7 +913,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: expr=[c1@0 ASC]", + "SortExec: expr=[c1@0 ASC], global=false", "ProjectionExec: expr=[c1@0 as c1]", "FilterExec: c1@0", // repartition is lowest down @@ -950,7 +980,7 @@ mod tests { "GlobalLimitExec: skip=0, fetch=100", "LocalLimitExec: fetch=100", // data is sorted so can't repartition here - "SortExec: expr=[c1@0 ASC]", + "SortExec: expr=[c1@0 ASC], global=true", // Doesn't parallelize for SortExec without preserve_partitioning "ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[c1]", ]; @@ -969,7 +999,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: expr=[c1@0 ASC]", + "SortExec: expr=[c1@0 ASC], global=true", // SortExec doesn't benefit from input partitioning "ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[c1]", ]; @@ -1122,8 +1152,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 70880b7505526..914e663299d40 100644 --- a/datafusion/core/src/physical_optimizer/sort_enforcement.rs +++ b/datafusion/core/src/physical_optimizer/sort_enforcement.rs @@ -47,7 +47,10 @@ use crate::physical_plan::windows::{BoundedWindowAggExec, WindowAggExec}; use crate::physical_plan::{with_new_children_if_necessary, Distribution, 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::{PhysicalExpr, PhysicalSortExpr}; use itertools::{concat, izip}; use std::iter::zip; @@ -439,15 +442,15 @@ 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, - || child.equivalence_properties(), + &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, &plan)?; - let sort_expr = required_ordering.to_vec(); + let sort_expr = create_sort_expr_from_requirement(&required_ordering); add_sort_above(child, sort_expr)?; *sort_onwards = Some(ExecTree::new(child.clone(), idx, vec![])); } @@ -465,7 +468,8 @@ fn ensure_sorting( } (Some(required), None) => { // Ordering requirement is not met, we should add a `SortExec` to the plan. - add_sort_above(child, required.to_vec())?; + let sort_expr = create_sort_expr_from_requirement(&required); + add_sort_above(child, sort_expr)?; *sort_onwards = Some(ExecTree::new(child.clone(), idx, vec![])); } (None, Some(_)) => { @@ -505,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: @@ -1053,12 +1057,12 @@ mod tests { let physical_plan = sort_exec(vec![sort_expr("nullable_col", &schema)], input); let expected_input = vec![ - "SortExec: expr=[nullable_col@0 ASC]", - " SortExec: expr=[non_nullable_col@1 ASC]", + "SortExec: expr=[nullable_col@0 ASC], global=true", + " SortExec: expr=[non_nullable_col@1 ASC], global=true", " MemoryExec: partitions=0, partition_sizes=[]", ]; let expected_optimized = vec![ - "SortExec: expr=[nullable_col@0 ASC]", + "SortExec: expr=[nullable_col@0 ASC], global=true", " MemoryExec: partitions=0, partition_sizes=[]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan); @@ -1107,9 +1111,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: expr=[non_nullable_col@1 ASC NULLS LAST]", + " SortExec: expr=[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: expr=[non_nullable_col@1 DESC]", + " SortExec: expr=[non_nullable_col@1 DESC], global=true", " MemoryExec: partitions=0, partition_sizes=[]", ]; @@ -1117,7 +1121,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: expr=[non_nullable_col@1 DESC]", + " SortExec: expr=[non_nullable_col@1 DESC], global=true", " MemoryExec: partitions=0, partition_sizes=[]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan); @@ -1139,7 +1143,7 @@ mod tests { ]; let expected_optimized = vec![ "SortPreservingMergeExec: [nullable_col@0 ASC]", - " SortExec: expr=[nullable_col@0 ASC]", + " SortExec: expr=[nullable_col@0 ASC], global=true", " MemoryExec: partitions=0, partition_sizes=[]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan); @@ -1159,15 +1163,15 @@ mod tests { let physical_plan = sort_preserving_merge_exec(sort_exprs, sort); let expected_input = vec![ "SortPreservingMergeExec: [nullable_col@0 ASC]", - " SortExec: expr=[nullable_col@0 ASC]", + " SortExec: expr=[nullable_col@0 ASC], global=true", " SortPreservingMergeExec: [nullable_col@0 ASC]", - " SortExec: expr=[nullable_col@0 ASC]", + " SortExec: expr=[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: expr=[nullable_col@0 ASC]", + " SortExec: expr=[nullable_col@0 ASC], global=true", " MemoryExec: partitions=0, partition_sizes=[]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan); @@ -1196,11 +1200,11 @@ mod tests { let expected_input = vec![ "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", - " SortExec: expr=[nullable_col@0 ASC]", + " SortExec: expr=[nullable_col@0 ASC], global=true", " SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", - " SortExec: expr=[nullable_col@0 ASC,non_nullable_col@1 ASC]", + " SortExec: expr=[nullable_col@0 ASC,non_nullable_col@1 ASC], global=true", " SortPreservingMergeExec: [non_nullable_col@1 ASC]", - " SortExec: expr=[non_nullable_col@1 ASC]", + " SortExec: expr=[non_nullable_col@1 ASC], global=true", " MemoryExec: partitions=0, partition_sizes=[]", ]; @@ -1237,10 +1241,10 @@ mod tests { let expected_input = vec![ "AggregateExec: mode=Final, gby=[], aggr=[]", " SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", - " SortExec: expr=[nullable_col@0 ASC,non_nullable_col@1 ASC]", + " SortExec: expr=[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: expr=[non_nullable_col@1 ASC]", + " SortExec: expr=[non_nullable_col@1 ASC], global=true", " MemoryExec: partitions=0, partition_sizes=[]", ]; @@ -1254,6 +1258,34 @@ mod tests { 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: expr=[nullable_col@0 ASC], global=true", + " SortPreservingMergeExec: [nullable_col@0 ASC]", + " SortExec: expr=[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: expr=[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_do_not_remove_sort_with_limit() -> Result<()> { let schema = create_test_schema()?; @@ -1281,20 +1313,20 @@ mod tests { " 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: expr=[nullable_col@0 ASC,non_nullable_col@1 ASC]", + " 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]", ]; // We should keep the bottom `SortExec`. 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]", + " SortExec: expr=[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: expr=[nullable_col@0 ASC,non_nullable_col@1 ASC]", + " 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]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan); @@ -1313,12 +1345,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: expr=[nullable_col@0 ASC]", + " SortExec: expr=[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: expr=[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=[]", ]; assert_optimized!(expected_input, expected_optimized, physical_plan); @@ -1343,7 +1375,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: expr=[nullable_col@0 ASC]", + " SortExec: expr=[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 @@ -1374,7 +1406,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: expr=[nullable_col@0 ASC]", + " SortExec: expr=[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 @@ -1408,13 +1440,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: expr=[nullable_col@0 ASC,non_nullable_col@1 ASC]", + " 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]", ]; // 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: expr=[nullable_col@0 ASC,non_nullable_col@1 ASC]", + " SortExec: expr=[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]", @@ -1448,20 +1480,20 @@ mod tests { let expected_input = vec![ "SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", - " SortExec: expr=[nullable_col@0 ASC,non_nullable_col@1 ASC]", + " 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]", " ParquetExec: limit=None, partitions={1 group: [[x]]}, output_ordering=[nullable_col@0 ASC], projection=[nullable_col, non_nullable_col]", - " SortExec: expr=[nullable_col@0 ASC]", + " SortExec: expr=[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: expr=[nullable_col@0 ASC]", + " SortExec: expr=[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: expr=[nullable_col@0 ASC]", + " SortExec: expr=[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); @@ -1493,15 +1525,15 @@ mod tests { let expected_input = vec![ "SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", " UnionExec", - " SortExec: expr=[nullable_col@0 ASC]", + " SortExec: expr=[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: expr=[nullable_col@0 ASC]", + " SortExec: expr=[nullable_col@0 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]", - " SortExec: expr=[nullable_col@0 ASC,non_nullable_col@1 ASC]", + " SortExec: expr=[nullable_col@0 ASC,non_nullable_col@1 ASC], global=false", " UnionExec", " 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]", @@ -1544,17 +1576,17 @@ mod tests { let expected_input = vec![ "SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", - " SortExec: expr=[nullable_col@0 ASC,non_nullable_col@1 ASC]", + " 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 ASC,non_nullable_col@1 DESC NULLS LAST]", + " 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]", + " SortExec: expr=[nullable_col@0 ASC], global=true", " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", - " SortExec: expr=[nullable_col@0 ASC]", + " SortExec: expr=[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); @@ -1590,7 +1622,7 @@ mod tests { let expected_input = vec![ "SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", - " SortExec: expr=[nullable_col@0 ASC]", + " SortExec: expr=[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]", @@ -1602,11 +1634,11 @@ mod tests { let expected_optimized = vec![ "SortPreservingMergeExec: [nullable_col@0 ASC]", " UnionExec", - " SortExec: expr=[nullable_col@0 ASC]", + " SortExec: expr=[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]", - " SortExec: expr=[nullable_col@0 ASC]", + " SortExec: expr=[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]", ]; @@ -1647,9 +1679,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 }]", " UnionExec", - " SortExec: expr=[nullable_col@0 DESC NULLS LAST]", + " SortExec: expr=[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: expr=[nullable_col@0 DESC NULLS LAST]", + " SortExec: expr=[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]", ]; let expected_optimized = vec![ @@ -1683,7 +1715,7 @@ mod tests { // we should be able to parallelize Sorting also (given that executors in between don't require) // single partition. let expected_input = vec![ - "SortExec: expr=[nullable_col@0 ASC]", + "SortExec: expr=[nullable_col@0 ASC], global=true", " FilterExec: NOT non_nullable_col@1", " CoalescePartitionsExec", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", @@ -1691,7 +1723,7 @@ mod tests { ]; let expected_optimized = vec![ "SortPreservingMergeExec: [nullable_col@0 ASC]", - " SortExec: expr=[nullable_col@0 ASC]", + " SortExec: expr=[nullable_col@0 ASC], global=false", " FilterExec: NOT non_nullable_col@1", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", " ParquetExec: limit=None, partitions={1 group: [[x]]}, projection=[nullable_col, non_nullable_col]", @@ -1771,9 +1803,9 @@ mod tests { // Sort Parallelize rule should end Coalesce + Sort linkage when Sort is Global Sort // Also input plan is not valid as it is. We need to add SortExec before SortPreservingMergeExec. let expected_input = vec![ - "SortExec: expr=[nullable_col@0 ASC]", + "SortExec: expr=[nullable_col@0 ASC], global=true", " SortPreservingMergeExec: [nullable_col@0 ASC]", - " SortExec: expr=[nullable_col@0 ASC]", + " SortExec: expr=[nullable_col@0 ASC], global=false", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", " CoalescePartitionsExec", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=0", @@ -1781,7 +1813,7 @@ mod tests { ]; let expected_optimized = vec![ "SortPreservingMergeExec: [nullable_col@0 ASC]", - " SortExec: expr=[nullable_col@0 ASC]", + " SortExec: expr=[nullable_col@0 ASC], global=false", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10", " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=0", " MemoryExec: partitions=0, partition_sizes=[]", 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..2bf505a6ad985 --- /dev/null +++ b/datafusion/core/src/physical_optimizer/sort_enforcement2.rs @@ -0,0 +1,2848 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +//! EnforceSorting optimizer rule inspects the physical plan with respect +//! to local sorting requirements and does the following: +//! - Adds a [SortExec] when a requirement is not met, +//! - Removes an already-existing [SortExec] if it is possible to prove +//! that this sort is unnecessary +//! The rule can work on valid *and* invalid physical plans with respect to +//! sorting requirements, but always produces a valid physical plan in this sense. +//! +//! A non-realistic but easy to follow example for sort removals: Assume that we +//! somehow get the fragment +//! +//! ```text +//! SortExec: expr=[nullable_col@0 ASC] +//! SortExec: expr=[non_nullable_col@1 ASC] +//! ``` +//! +//! in the physical plan. The 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; +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; +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::{ + with_new_children_if_necessary, DisplayFormatType, Distribution, ExecutionPlan, + Partitioning, SendableRecordBatchStream, +}; +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, + ordering_satisfy, ordering_satisfy_requirement, required_provided_either_compatible, + requirements_compatible, +}; +use datafusion_physical_expr::window::WindowExpr; +use datafusion_physical_expr::{ + new_sort_requirements, EquivalenceProperties, PhysicalExpr, PhysicalSortExpr, + PhysicalSortRequirements, +}; +use itertools::izip; +use std::any::Any; +use std::ops::Deref; +use std::sync::Arc; + +/// 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 {} + +impl TopDownEnforceSorting { + #[allow(missing_docs)] + pub fn new() -> Self { + Self {} + } +} + +/// This is a "data class" we use within the [TopDownEnforceSorting] rule +#[derive(Debug, Clone)] +struct PlanWithSortRequirements { + /// Current plan + 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. + /// 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 + || is_limit(&plan); + let request_ordering = plan.required_input_ordering(); + PlanWithSortRequirements { + plan, + impact_result_ordering, + satisfy_single_distribution: false, + 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, + satisfy_single_distribution: 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()); + + 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) + || (self.satisfy_single_distribution + && self + .plan + .as_any() + .downcast_ref::() + .is_none()); + let child_impact_result_ordering = if is_limit(&self.plan) { + 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() + } +} + +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, + satisfy_single_distribution: self.satisfy_single_distribution, + 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)?; + // 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 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) + } + })?; + // 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 { + "TopDownEnforceSorting" + } + + fn schema_check(&self) -> bool { + true + } +} + +/// 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> { + if let Some(sort_exec) = requirements.plan.as_any().downcast_ref::() { + // 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 + .as_any() + .downcast_ref::() + { + // 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::() + { + if sort_pres_exec.expr() == child_sort_exec.expr() { + if let Some(result) = + analyze_immediate_sort_removal(&requirements, child_sort_exec) + { + return Ok(Some(result)); + } + } + } 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)); + } + } + } + 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, 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 { + required_ordering: None, + adjusted_request_ordering: vec![ + adjusted; + requirements + .adjusted_request_ordering + .len() + ], + ..requirements + })); + } 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); + + if should_reverse_window_sort_requirements( + plan.clone(), + request_child, + reversed_request_child.as_deref(), + ) { + let WindowExecInfo { + window_expr, + input_schema, + partition_keys, + } = extract_window_info_from_plan(plan)?; + + 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 Arc + } else { + Arc::new(WindowAggExec::try_new( + window_expr, + plan.children()[0].clone(), + input_schema, + partition_keys, + Some(new_physical_ordering), + )?) as Arc + }; + 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], + })); + } + } + } + Ok(Some(PlanWithSortRequirements { + required_ordering: None, + ..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 + let parent_required_expr = + create_sort_expr_from_requirement(parent_required.unwrap()); + 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), + )) + } else { + // 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(); + // 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().is::() + || plan.as_any().is::() + || is_limit(plan)) + && plan.as_any().downcast_ref::().is_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), + )) + } 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( + 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)?; + 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, + ), + )) + } + } + } 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, smj.join_type, left_columns_len); + match expr_source_side { + Some(JoinSide::Left) if maintains_input_order[0] => { + try_pushdown_requirements_to_join( + &requirements, + parent_required, + parent_required_expr, + JoinSide::Left, + ) + } + Some(JoinSide::Right) if maintains_input_order[1] => { + 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, + Some(new_right_required.deref()), + parent_required_expr, + JoinSide::Right, + ) + } + _ => { + // 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, + ), + )) + } + } + } else if plan.required_input_ordering().iter().any(Option::is_some) { + 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)| { + if *can_push_down { + determine_children_requirement( + parent_required, + request_child.as_deref(), + child.clone(), + ) + } else { + RequirementsCompatibility::NonCompatible + } + }) + .collect::>(); + 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 { + // 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 + 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_adjusted = + map_requirement_before_projection(parent_required, expr); + if new_adjusted.is_some() { + Ok(Some(PlanWithSortRequirements { + required_ordering: None, + adjusted_request_ordering: vec![new_adjusted], + ..requirements + })) + } 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, + ), + )) + } + } else { + Ok(Some(PlanWithSortRequirements { + required_ordering: None, + adjusted_request_ordering: vec![ + requirements.required_ordering; + requirements + .adjusted_request_ordering + .len() + ], + ..requirements + })) + } + } + } +} + +/// 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.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 + } +} + +/// 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( + input_schema: &SchemaRef, + window_request: &PhysicalSortRequirements, + parent_required_expr: &PhysicalSortRequirements, +) -> 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 { + Ok(false) + } + } else { + Ok(false) + } +} + +fn reverse_window_sort_requirements( + request_child: Option<&[PhysicalSortRequirements]>, +) -> Option> { + 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::>() + }) +} + +/// 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]>, + top_reversed_requirement: Option<&[PhysicalSortRequirements]>, +) -> bool { + 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; + } + // 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() + }) + { + 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, +) -> Result { + match (required, request_ordering) { + (_, None) => Ok(false), + (None, Some(_)) => Ok(false), + (Some(required), Some(request_ordering)) => { + if required.len() > request_ordering.len() { + return Ok(false); + } + let alignment_flags_rs: Result> = 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(Ok(false)) + } + }) + .collect(); + let alignment_flags = alignment_flags_rs?; + if alignment_flags.is_empty() { + Ok(false) + } else { + Ok(alignment_flags.iter().all(|o| *o)) + } + } + } +} + +fn extract_window_info_from_plan( + plan: &Arc, +) -> Result { + if let Some(exec) = plan.as_any().downcast_ref::() { + Ok(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(), + }) + .ok_or_else(|| { + DataFusionError::Plan( + "Expects to receive either WindowAggExec of BoundedWindowAggExec" + .to_string(), + ) + }) + } +} + +fn try_pushdown_requirements_to_join( + requirements: &PlanWithSortRequirements, + parent_required: Option<&[PhysicalSortRequirements]>, + sort_expr: Vec, + push_side: JoinSide, +) -> Result> { + let child_idx = match push_side { + JoinSide::Left => 0, + JoinSide::Right => 1, + }; + 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), + )) + } + } +} + +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().is::() { + 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().is::() { + 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) + } +} + +/// 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 { + /// 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( + "TombStoneExec, invalid plan".to_string(), + )) + } + + 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::aggregates::PhysicalGroupBy; + use crate::physical_plan::aggregates::{AggregateExec, AggregateMode}; + 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; + 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::logical_plan::JoinType; + 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 { + let nullable_column = Field::new("nullable_col", DataType::Int32, true); + let non_nullable_column = Field::new("non_nullable_col", DataType::Int32, false); + let schema = Arc::new(Schema::new(vec![nullable_column, non_nullable_column])); + + Ok(schema) + } + + fn create_test_schema2() -> Result { + let col_a = Field::new("col_a", DataType::Int32, true); + let col_b = Field::new("col_b", DataType::Int32, true); + let schema = Arc::new(Schema::new(vec![col_a, col_b])); + Ok(schema) + } + + // 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()?; + 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(()) + } + + #[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 + /// + /// `$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())?; + // Get string representation of the plan + let actual = get_plan_string(&optimized_physical_plan); + assert_eq!( + expected_optimized_lines, actual, + "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected_optimized_lines:#?}\nactual:\n\n{actual:#?}\n\n" + ); + + }; + } + + #[tokio::test] + async fn test_remove_unnecessary_sort() -> Result<()> { + let schema = create_test_schema()?; + let source = memory_exec(&schema); + let input = sort_exec(vec![sort_expr("non_nullable_col", &schema)], source); + let physical_plan = sort_exec(vec![sort_expr("nullable_col", &schema)], input); + + let expected_input = vec![ + "SortExec: expr=[nullable_col@0 ASC], global=true", + " SortExec: expr=[non_nullable_col@1 ASC], global=true", + " MemoryExec: partitions=0, partition_sizes=[]", + ]; + let expected_optimized = vec![ + "SortExec: expr=[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_not_remove_top_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); + + // 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(), + )), + 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: expr=[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: expr=[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: Preceding(NULL), end_bound: CurrentRow }]", + " FilterExec: NOT non_nullable_col@1", + " SortExec: expr=[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: expr=[non_nullable_col@1 DESC], global=true", + " MemoryExec: partitions=0, partition_sizes=[]", + ]; + assert_optimized!(expected_input, expected_optimized, 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 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: expr=[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: 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(()) + } + + #[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![ + "SortExec: expr=[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 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: expr=[nullable_col@0 ASC], global=true", + " SortPreservingMergeExec: [nullable_col@0 ASC]", + " SortExec: expr=[nullable_col@0 ASC], global=true", + " MemoryExec: partitions=0, partition_sizes=[]", + ]; + let expected_optimized = vec![ + "SortExec: expr=[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 sort_exprs = vec![sort_expr("non_nullable_col", &schema)]; + let sort = sort_exec(sort_exprs.clone(), source); + let spm = sort_preserving_merge_exec(sort_exprs, sort); + + let sort_exprs = vec![ + sort_expr("nullable_col", &schema), + sort_expr("non_nullable_col", &schema), + ]; + let sort2 = sort_exec(sort_exprs.clone(), spm); + let spm2 = sort_preserving_merge_exec(sort_exprs, sort2); + + let sort_exprs = vec![sort_expr("nullable_col", &schema)]; + let sort3 = sort_exec(sort_exprs, spm2); + let physical_plan = repartition_exec(repartition_exec(sort3)); + + let expected_input = vec![ + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=10", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " SortExec: expr=[nullable_col@0 ASC], global=true", + " SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", + " SortExec: expr=[nullable_col@0 ASC,non_nullable_col@1 ASC], global=true", + " SortPreservingMergeExec: [non_nullable_col@1 ASC]", + " SortExec: expr=[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_sort3() -> Result<()> { + let schema = create_test_schema()?; + let source = memory_exec(&schema); + let sort_exprs = vec![sort_expr("non_nullable_col", &schema)]; + let sort = sort_exec(sort_exprs.clone(), source); + let spm = sort_preserving_merge_exec(sort_exprs, sort); + + let sort_exprs = vec![ + sort_expr("nullable_col", &schema), + sort_expr("non_nullable_col", &schema), + ]; + 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 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![ + "AggregateExec: mode=Final, gby=[], aggr=[]", + " SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", + " SortExec: expr=[nullable_col@0 ASC,non_nullable_col@1 ASC], global=true", + " RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", + " SortPreservingMergeExec: [non_nullable_col@1 ASC]", + " SortExec: expr=[non_nullable_col@1 ASC], global=true", + " MemoryExec: partitions=0, partition_sizes=[]", + ]; + + let expected_optimized = vec![ + "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: expr=[nullable_col@0 ASC], global=true", + " SortPreservingMergeExec: [nullable_col@0 ASC]", + " SortExec: expr=[nullable_col@0 ASC], global=true", + " MemoryExec: partitions=0, partition_sizes=[]", + ]; + let expected_optimized = vec![ + "SortExec: expr=[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: expr=[nullable_col@0 ASC], global=true", + " SortExec: expr=[nullable_col@0 ASC,non_nullable_col@1 ASC], global=true", + " SortExec: expr=[non_nullable_col@1 ASC], global=true", + " MemoryExec: partitions=0, partition_sizes=[]", + ]; + // Keep the middle SortExec + let expected_optimized = [ + "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(()) + } + + #[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()?; + 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: expr=[nullable_col@0 ASC], global=true", + " SortPreservingMergeExec: [non_nullable_col@1 ASC]", + " MemoryExec: partitions=0, partition_sizes=[]", + ]; + let expected_optimized = vec![ + "SortExec: expr=[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![ + "SortExec: expr=[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_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: 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]", + ]; + + // We should keep the bottom `SortExec`. + 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=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: 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]", + ]; + 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: expr=[nullable_col@0 ASC], global=true", + " MemoryExec: partitions=0, partition_sizes=[]", + ]; + let expected_optimized = vec![ + "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(()) + } + + #[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: expr=[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![ + "SortExec: expr=[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: expr=[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: expr=[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. + 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: 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]", + ]; + + let expected_optimized = vec![ + "SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", + " UnionExec", + " SortExec: expr=[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: 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]", + ]; + 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), + 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: 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]", + " ParquetExec: limit=None, partitions={1 group: [[x]]}, output_ordering=[nullable_col@0 ASC], projection=[nullable_col, non_nullable_col]", + " SortExec: expr=[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: expr=[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: expr=[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_sorted4() -> Result<()> { + let schema = create_test_schema()?; + + let source1 = parquet_exec(&schema); + let sort_exprs1 = vec![ + sort_expr("nullable_col", &schema), + sort_expr("non_nullable_col", &schema), + ]; + let sort_exprs2 = vec![sort_expr("nullable_col", &schema)]; + let sort1 = sort_exec(sort_exprs2.clone(), source1.clone()); + let sort2 = sort_exec(sort_exprs2.clone(), source1); + + let source2 = parquet_exec_sorted(&schema, sort_exprs2); + + let union = union_exec(vec![sort1, source2, sort2]); + let physical_plan = sort_preserving_merge_exec(sort_exprs1, union); + + // Ordering requirement of the `SortPreservingMergeExec` is not met. + // Should modify the plan to ensure that all three inputs to the + // `UnionExec` satisfy the ordering, OR add a single sort after + // the `UnionExec` (both of which are equally good for this example). + let expected_input = vec![ + "SortPreservingMergeExec: [nullable_col@0 ASC,non_nullable_col@1 ASC]", + " UnionExec", + " SortExec: expr=[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: expr=[nullable_col@0 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: 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 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: 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]", + ]; + 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), + sort_expr_options( + "non_nullable_col", + &schema, + SortOptions { + descending: true, + nulls_first: false, + }, + ), + ]; + let sort_exprs3 = vec![sort_expr("nullable_col", &schema)]; + let sort1 = sort_exec(sort_exprs1, source1.clone()); + let sort2 = sort_exec(sort_exprs2, source1); + + let union = union_exec(vec![sort1, sort2]); + let physical_plan = sort_preserving_merge_exec(sort_exprs3, union); + + // The `UnionExec` doesn't preserve any of the inputs ordering in the + // example below. However, we should be able to change the unnecessarily + // fine `SortExec`s below with required `SortExec`s that are absolutely necessary. + let expected_input = 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]", + " 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]", + " SortExec: expr=[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_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: expr=[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]", + " UnionExec", + " SortExec: expr=[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: expr=[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_sorted7() -> Result<()> { + let schema = create_test_schema()?; + + let source1 = parquet_exec(&schema); + let sort_exprs1 = vec![ + sort_expr("nullable_col", &schema), + sort_expr("non_nullable_col", &schema), + ]; + let sort_exprs3 = vec![sort_expr("nullable_col", &schema)]; + let sort1 = sort_exec(sort_exprs1.clone(), source1.clone()); + let sort2 = sort_exec(sort_exprs1, source1); + + let union = union_exec(vec![sort1, sort2]); + let physical_plan = sort_preserving_merge_exec(sort_exprs3, union); + + // Union 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: 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 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); + 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()?; + + 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); + let sort1 = sort_exec(reversed_sort_exprs2.clone(), source1); + let sort2 = sort_exec(reversed_sort_exprs2.clone(), source2); + + 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. + // 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: expr=[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: expr=[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` 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", + " SortExec: expr=[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: expr=[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]", + ]; + 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 }]", + " 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(()) + } + + #[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![ + 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![ + "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]", + ] + } + }; + 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![ + 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![ + "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]", + ] + } + }; + 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![ + "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![ + "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()) + } + + /// 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 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)) + } + + fn limit_exec(input: Arc) -> Arc { + global_limit_exec(local_limit_exec(input)) + } + + fn local_limit_exec(input: Arc) -> Arc { + Arc::new(LocalLimitExec::new(input, 100)) + } + + fn global_limit_exec(input: Arc) -> Arc { + Arc::new(GlobalLimitExec::new(input, 0, Some(100))) + } + + fn repartition_exec(input: Arc) -> Arc { + Arc::new( + RepartitionExec::try_new(input, Partitioning::RoundRobinBatch(10)).unwrap(), + ) + } + + fn aggregate_exec(input: Arc) -> Arc { + let schema = input.schema(); + Arc::new( + AggregateExec::try_new( + AggregateMode::Final, + PhysicalGroupBy::default(), + vec![], + input, + schema, + ) + .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 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/core/src/physical_plan/joins/sort_merge_join.rs b/datafusion/core/src/physical_plan/joins/sort_merge_join.rs index 88667b692b541..de9d7b4dc6ea4 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; 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 c8f5eb793910f..93ed9f62baeae 100644 --- a/datafusion/core/src/physical_plan/mod.rs +++ b/datafusion/core/src/physical_plan/mod.rs @@ -141,7 +141,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()] } @@ -591,11 +591,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/sorts/sort.rs b/datafusion/core/src/physical_plan/sorts/sort.rs index 9bacf9fad6ffd..a215bfaed03fd 100644 --- a/datafusion/core/src/physical_plan/sorts/sort.rs +++ b/datafusion/core/src/physical_plan/sorts/sort.rs @@ -795,9 +795,19 @@ impl ExecutionPlan for SortExec { let expr: Vec = self.expr.iter().map(|e| e.to_string()).collect(); match self.fetch { Some(fetch) => { - write!(f, "SortExec: fetch={fetch}, expr=[{}]", expr.join(",")) + write!( + f, + "SortExec: fetch={fetch}, expr=[{}], global={}", + expr.join(","), + !self.preserve_partitioning + ) } - None => write!(f, "SortExec: expr=[{}]", expr.join(",")), + None => write!( + f, + "SortExec: expr=[{}], 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 7ef4d3bf8e868..2d7018f38a335 100644 --- a/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs +++ b/datafusion/core/src/physical_plan/sorts/sort_preserving_merge.rs @@ -46,7 +46,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 /// @@ -125,12 +127,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/windows/bounded_window_agg_exec.rs b/datafusion/core/src/physical_plan/windows/bounded_window_agg_exec.rs index 0ac836cf28ed9..36efd86356bd2 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,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.sort_keys.as_deref().unwrap_or(&[]); + let sort_keys = self + .output_ordering() + .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()); @@ -167,9 +171,31 @@ 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 expr_partition_keys = self.window_expr()[0].partition_by(); + let expr_order_keys = self.window_expr()[0].order_by(); + let requirements = self.sort_keys.as_deref().map(|ordering| { + ordering + .iter() + .map(|o| { + let is_partition_only = + expr_partition_keys.iter().any(|e| e.eq(&o.expr)) + && !expr_order_keys.iter().any(|e| e.expr.eq(&o.expr)); + if is_partition_only { + PhysicalSortRequirements { + expr: o.expr.clone(), + sort_options: None, + } + } else { + PhysicalSortRequirements { + expr: o.expr.clone(), + sort_options: Some(o.options), + } + } + }) + .collect::>() + }); + vec![requirements] } fn required_input_distribution(&self) -> Vec { @@ -177,7 +203,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())] } } 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 a667f0a3c2168..43614e34adc82 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,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.sort_keys.as_deref().unwrap_or(&[]); + let sort_keys = self + .output_ordering() + .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()); @@ -172,9 +175,31 @@ 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 expr_partition_keys = self.window_expr()[0].partition_by(); + let expr_order_keys = self.window_expr()[0].order_by(); + let requirements = self.sort_keys.as_deref().map(|ordering| { + ordering + .iter() + .map(|o| { + let is_partition_only = + expr_partition_keys.iter().any(|e| e.eq(&o.expr)) + && !expr_order_keys.iter().any(|e| e.expr.eq(&o.expr)); + if is_partition_only { + PhysicalSortRequirements { + expr: o.expr.clone(), + sort_options: None, + } + } else { + PhysicalSortRequirements { + expr: o.expr.clone(), + sort_options: Some(o.options), + } + } + }) + .collect::>() + }); + vec![requirements] } fn required_input_distribution(&self) -> Vec { @@ -182,7 +207,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 99f5bd2d670dc..5941946ce8de9 100644 --- a/datafusion/core/tests/sql/explain_analyze.rs +++ b/datafusion/core/tests/sql/explain_analyze.rs @@ -600,7 +600,7 @@ async fn test_physical_plan_display_indent() { let expected = vec![ "GlobalLimitExec: skip=0, fetch=10", " SortPreservingMergeExec: [the_min@2 DESC]", - " SortExec: fetch=10, expr=[the_min@2 DESC]", + " SortExec: fetch=10, expr=[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 6d1b1e91b66ef..c872a1c86af77 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: expr=[c1@0 ASC]", + " SortExec: expr=[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: expr=[c1@0 ASC]", + " SortExec: expr=[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: expr=[CAST(t1.c3 AS Decimal128(10, 2))@4 ASC]", + " SortExec: expr=[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: expr=[c3@2 ASC]", + " SortExec: expr=[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", @@ -1981,7 +1981,7 @@ async fn left_semi_join() -> Result<()> { let expected = if repartition_joins { vec![ "SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST]", - " SortExec: expr=[t1_id@0 ASC NULLS LAST]", + " SortExec: expr=[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 })]", @@ -1997,7 +1997,7 @@ async fn left_semi_join() -> Result<()> { ] } else { vec![ - "SortExec: expr=[t1_id@0 ASC NULLS LAST]", + "SortExec: expr=[t1_id@0 ASC NULLS LAST], global=true", " 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 })]", @@ -2061,7 +2061,7 @@ async fn left_semi_join() -> Result<()> { let expected = if repartition_joins { vec![ "SortPreservingMergeExec: [t1_id@0 ASC NULLS LAST]", - " SortExec: expr=[t1_id@0 ASC NULLS LAST]", + " SortExec: expr=[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 })]", @@ -2076,7 +2076,7 @@ async fn left_semi_join() -> Result<()> { ] } else { vec![ - "SortExec: expr=[t1_id@0 ASC NULLS LAST]", + "SortExec: expr=[t1_id@0 ASC NULLS LAST], global=true", " 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 })]", @@ -2255,7 +2255,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: expr=[t1_id@0 ASC NULLS LAST]", + " SortExec: expr=[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 } }", @@ -2271,7 +2271,7 @@ async fn right_semi_join() -> Result<()> { ] } else { vec![ - "SortExec: expr=[t1_id@0 ASC NULLS LAST]", + "SortExec: expr=[t1_id@0 ASC NULLS LAST], global=true", " 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=CollectLeft, 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 } }", @@ -2303,7 +2303,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: expr=[t1_id@0 ASC NULLS LAST]", + " SortExec: expr=[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 } }", @@ -2318,7 +2318,7 @@ async fn right_semi_join() -> Result<()> { ] } else { vec![ - "SortExec: expr=[t1_id@0 ASC NULLS LAST]", + "SortExec: expr=[t1_id@0 ASC NULLS LAST], global=true", " 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=CollectLeft, 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 } }", diff --git a/datafusion/core/tests/sql/window.rs b/datafusion/core/tests/sql/window.rs index 7ef4af23a0595..8d7039442da75 100644 --- a/datafusion/core/tests/sql/window.rs +++ b/datafusion/core/tests/sql/window.rs @@ -1346,7 +1346,7 @@ async fn test_window_agg_sort() -> Result<()> { "ProjectionExec: expr=[c9@1 as c9, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as sum1, SUM(aggregate_test_100.c9) ORDER BY [aggregate_test_100.c9 ASC NULLS LAST, aggregate_test_100.c8 ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 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: 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: expr=[c9@1 ASC NULLS LAST,c8@0 ASC NULLS LAST]", + " SortExec: expr=[c9@1 ASC NULLS LAST,c8@0 ASC NULLS LAST], global=true", ] }; @@ -1378,7 +1378,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: expr=[c2@0 ASC NULLS LAST,c9@1 ASC NULLS LAST]", + " SortExec: expr=[c2@0 ASC NULLS LAST,c9@1 ASC NULLS LAST], global=true", ] }; @@ -1406,13 +1406,13 @@ async fn over_order_by_sort_keys_sorting_global_order_compacting() -> Result<()> // 3 SortExec are added let expected = { vec![ - "SortExec: expr=[c2@0 ASC NULLS LAST]", + "SortExec: expr=[c2@0 ASC NULLS LAST], global=true", " 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)]", " 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: expr=[c9@1 ASC NULLS LAST,c2@0 ASC NULLS LAST]", + " SortExec: expr=[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: expr=[c2@0 ASC NULLS LAST,c9@1 ASC NULLS LAST]", + " SortExec: expr=[c2@0 ASC NULLS LAST,c9@1 ASC NULLS LAST], global=true", ] }; @@ -1448,11 +1448,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: expr=[c1@0 ASC NULLS LAST,c2@1 ASC NULLS LAST]", + " SortExec: expr=[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: expr=[c1@0 ASC NULLS LAST,c2@1 ASC NULLS LAST]", + " SortExec: expr=[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", @@ -1491,7 +1491,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: expr=[c9@0 DESC]", + " SortExec: expr=[c9@0 DESC], global=true", ] }; @@ -1546,7 +1546,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: expr=[c9@0 DESC]", + " SortExec: expr=[c9@0 DESC], global=true", ] }; @@ -1596,9 +1596,9 @@ async fn test_window_agg_sort_non_reversed_plan() -> Result<()> { "ProjectionExec: expr=[c9@0 as c9, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@2 as rn1, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@1 as rn2]", " 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: expr=[c9@0 ASC NULLS LAST]", + " SortExec: expr=[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: expr=[c9@0 DESC]" + " SortExec: expr=[c9@0 DESC], global=true", ] }; @@ -1649,10 +1649,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]", " 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: expr=[c9@2 ASC NULLS LAST,c1@0 ASC NULLS LAST,c2@1 ASC NULLS LAST]", + " 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]", + " SortExec: expr=[c9@2 DESC,c1@0 DESC], global=true", ] }; @@ -1737,15 +1737,17 @@ 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: expr=[c3@2 ASC NULLS LAST,c2@1 ASC NULLS LAST]", + " SortExec: expr=[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: expr=[c3@2 ASC NULLS LAST,c1@0 ASC]", - " 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)) }]", + " SortExec: expr=[c3@2 ASC NULLS LAST,c1@0 ASC], 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: expr=[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: expr=[c3@2 DESC,c1@0 ASC NULLS LAST]", + " SortExec: expr=[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: expr=[c3@2 DESC,c1@0 ASC NULLS LAST], global=true", ] }; @@ -1785,7 +1787,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: expr=[c1@0 ASC NULLS LAST,c9@1 DESC]", + " SortExec: expr=[c1@0 ASC NULLS LAST,c9@1 DESC], global=true", ] }; @@ -1839,7 +1841,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: expr=[c1@0 ASC NULLS LAST,c9@1 DESC]", + " SortExec: expr=[c1@0 ASC NULLS LAST,c9@1 DESC], global=true", ] }; @@ -1892,7 +1894,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: expr=[CAST(c3@1 AS Int16) + c4@2 DESC,c9@3 DESC,c2@0 ASC NULLS LAST]", + " SortExec: expr=[CAST(c3@1 AS Int16) + c4@2 DESC,c9@3 DESC,c2@0 ASC NULLS LAST], global=true", ] }; @@ -2003,7 +2005,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: expr=[c3@1 DESC,c9@2 DESC,c2@0 ASC NULLS LAST]", + " SortExec: expr=[c3@1 DESC,c9@2 DESC,c2@0 ASC NULLS LAST], global=true", ] }; @@ -2052,7 +2054,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: expr=[c1@0 ASC NULLS LAST]", + " SortExec: expr=[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", @@ -2087,11 +2089,11 @@ async fn test_window_agg_global_sort_parallelize_sort_disabled() -> Result<()> { // Only 1 SortExec was added let expected = { vec![ - "SortExec: expr=[c1@0 ASC NULLS LAST]", + "SortExec: expr=[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: expr=[c1@0 ASC NULLS LAST]", + " SortExec: expr=[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", @@ -2129,13 +2131,13 @@ async fn test_window_agg_global_sort_intermediate_parallel_sort() -> Result<()> // Only 1 SortExec was added let expected = { vec![ - "SortExec: expr=[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]", + " 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]", + " 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", @@ -2173,7 +2175,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: fetch=1, expr=[c13@0 ASC NULLS LAST]", + " SortExec: fetch=1, expr=[c13@0 ASC NULLS LAST], global=true", " ProjectionExec: expr=[c13@0 as c13]", ] }; @@ -2367,7 +2369,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: fetch=5, expr=[inc_col@24 DESC]", + " SortExec: fetch=5, expr=[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)) }]", @@ -2442,7 +2444,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: fetch=5, expr=[ts@24 DESC]", + " SortExec: fetch=5, expr=[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)) }]", @@ -2501,7 +2503,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: fetch=5, expr=[inc_col@10 ASC NULLS LAST]", + " SortExec: fetch=5, expr=[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)) }]", @@ -2555,7 +2557,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: fetch=5, expr=[inc_col@5 ASC NULLS LAST]", + " SortExec: fetch=5, expr=[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/core/tests/sqllogictests/test_files/window.slt b/datafusion/core/tests/sqllogictests/test_files/window.slt index 7c83c1b9b5550..34f43062d491b 100644 --- a/datafusion/core/tests/sqllogictests/test_files/window.slt +++ b/datafusion/core/tests/sqllogictests/test_files/window.slt @@ -262,7 +262,7 @@ Sort: d.b ASC NULLS LAST EmptyRelation physical_plan SortPreservingMergeExec: [b@0 ASC NULLS LAST] - SortExec: expr=[b@0 ASC NULLS LAST] + SortExec: expr=[b@0 ASC NULLS LAST], global=false ProjectionExec: expr=[b@0 as b, MAX(d.a)@1 as max_a] AggregateExec: mode=FinalPartitioned, gby=[b@0 as b], aggr=[MAX(d.a)] CoalesceBatchesExec: target_batch_size=8192 @@ -348,13 +348,13 @@ Sort: d.b ASC NULLS LAST EmptyRelation physical_plan SortPreservingMergeExec: [b@0 ASC NULLS LAST] - SortExec: expr=[b@0 ASC NULLS LAST] + SortExec: expr=[b@0 ASC NULLS LAST], global=false ProjectionExec: expr=[b@0 as b, MAX(d.a)@1 as max_a, MAX(d.seq)@2 as MAX(d.seq)] AggregateExec: mode=FinalPartitioned, gby=[b@0 as b], aggr=[MAX(d.a), MAX(d.seq)] AggregateExec: mode=Partial, gby=[b@2 as b], aggr=[MAX(d.a), MAX(d.seq)] ProjectionExec: expr=[ROW_NUMBER() PARTITION BY [s.b] ORDER BY [s.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as seq, a@0 as a, b@1 as b] 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: Range, start_bound: Preceding(Int64(NULL)), end_bound: CurrentRow }] - SortExec: expr=[b@1 ASC NULLS LAST,a@0 ASC NULLS LAST] + SortExec: expr=[b@1 ASC NULLS LAST,a@0 ASC NULLS LAST], global=false CoalesceBatchesExec: target_batch_size=8192 RepartitionExec: partitioning=Hash([Column { name: "b", index: 1 }], 4), input_partitions=4 UnionExec 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..8e0b3fb98e784 100644 --- a/datafusion/physical-expr/src/sort_expr.rs +++ b/datafusion/physical-expr/src/sort_expr.rs @@ -69,4 +69,55 @@ impl PhysicalSortExpr { options: Some(self.options), }) } + + pub fn satisfy(&self, requirement: &PhysicalSortRequirements) -> bool { + if let Some(sort_options) = requirement.sort_options { + self.options == sort_options && 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..3e14b8cc524f6 100644 --- a/datafusion/physical-expr/src/utils.rs +++ b/datafusion/physical-expr/src/utils.rs @@ -21,12 +21,14 @@ 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::ops::Deref; use std::sync::Arc; /// Compare the two expr lists are equal no matter the order. @@ -168,6 +170,20 @@ pub fn normalize_expr_with_equivalence_properties( .unwrap_or(expr) } +pub fn new_sort_requirements( + sort_keys: Option<&[PhysicalSortExpr]>, +) -> Option> { + sort_keys.map(|ordering| { + ordering + .iter() + .map(|o| PhysicalSortRequirements { + expr: o.expr.clone(), + sort_options: Some(o.options), + }) + .collect::>() + }) +} + pub fn normalize_sort_expr_with_equivalence_properties( sort_expr: PhysicalSortExpr, eq_properties: &[EquivalentClass], @@ -185,11 +201,29 @@ pub fn normalize_sort_expr_with_equivalence_properties( } } -/// Checks whether given ordering requirements are satisfied by provided [PhysicalSortExpr]s. -pub fn ordering_satisfy EquivalenceProperties>( +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, + } + } else { + sort_requirement + } +} + +/// Checks whether the required [PhysicalSortExpr]s are satisfied by the provided [PhysicalSortExpr]s. +pub fn ordering_satisfy EquivalenceProperties>( provided: Option<&[PhysicalSortExpr]>, required: Option<&[PhysicalSortExpr]>, - equal_properties: F, + equal_properties: &F, ) -> bool { match (provided, required) { (_, None) => true, @@ -200,10 +234,10 @@ pub fn ordering_satisfy EquivalenceProperties>( } } -pub 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 @@ -213,7 +247,9 @@ pub 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| { @@ -230,11 +266,198 @@ pub fn ordering_satisfy_concrete EquivalenceProperties>( .iter() .zip(normalized_provided_exprs.iter()) .all(|(order1, order2)| order1.eq(order2)) - } else { + } +} + +/// 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]>, + 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 { + let eq_props = equal_properties(); + let eq_classes = eq_props.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)) } } +/// 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 { + 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.deref(), + ) + }) + .collect::>(); + let normalized_provided = provided + .iter() + .map(|e| { + normalize_sort_requirement_with_equivalence_properties( + e.clone(), + eq_classes.deref(), + ) + }) + .collect::>(); + normalized_required + .iter() + .zip(normalized_provided.iter()) + .all(|(req, pro)| pro.compatible(req)) + } + } + } +} + +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)], +) -> 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, + }) + .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 let Some(sort_options) = prop.sort_options { + PhysicalSortExpr { + expr: prop.expr.clone(), + options: sort_options, + } + } 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 { @@ -416,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(()) 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(), )),