From 44ee85c6b828266d161dc6149b884cfa5fb239e6 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 20 Sep 2023 12:56:49 +0300 Subject: [PATCH 001/122] combine equivalence and ordering equivalence --- .../combine_partial_final_agg.rs | 9 + .../enforce_distribution.rs | 32 +- .../src/physical_optimizer/enforce_sorting.rs | 571 +++++++++++++++- .../replace_with_order_preserving_variants.rs | 1 - .../src/physical_optimizer/sort_pushdown.rs | 38 +- .../core/src/physical_optimizer/utils.rs | 9 +- datafusion/core/src/physical_planner.rs | 7 + datafusion/physical-expr/src/equivalence.rs | 613 +++++++++++++++++- .../src/expressions/get_indexed_field.rs | 25 +- datafusion/physical-expr/src/partitioning.rs | 19 +- .../physical-expr/src/sort_properties.rs | 2 - datafusion/physical-expr/src/utils.rs | 174 ++--- .../physical-expr/src/window/built_in.rs | 7 +- .../physical-plan/src/aggregates/mod.rs | 236 ++++--- .../physical-plan/src/coalesce_batches.rs | 6 +- .../physical-plan/src/coalesce_partitions.rs | 6 +- datafusion/physical-plan/src/filter.rs | 18 +- .../physical-plan/src/joins/cross_join.rs | 29 +- .../physical-plan/src/joins/hash_join.rs | 30 +- .../src/joins/nested_loop_join.rs | 22 +- .../src/joins/sort_merge_join.rs | 24 +- .../src/joins/symmetric_hash_join.rs | 22 +- datafusion/physical-plan/src/joins/utils.rs | 93 ++- datafusion/physical-plan/src/lib.rs | 8 +- datafusion/physical-plan/src/limit.rs | 12 +- datafusion/physical-plan/src/projection.rs | 151 +++-- .../physical-plan/src/repartition/mod.rs | 14 +- datafusion/physical-plan/src/sorts/sort.rs | 16 +- .../src/sorts/sort_preserving_merge.rs | 6 +- datafusion/physical-plan/src/unnest.rs | 6 +- .../src/windows/bounded_window_agg_exec.rs | 8 +- datafusion/physical-plan/src/windows/mod.rs | 21 +- .../src/windows/window_agg_exec.rs | 6 +- 33 files changed, 1768 insertions(+), 473 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs b/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs index 40b2bcc3e140..0fbabef323f7 100644 --- a/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs +++ b/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs @@ -43,12 +43,21 @@ impl CombinePartialFinalAggregate { } } +fn print_plan(plan: &Arc) -> () { + let formatted = crate::physical_plan::displayable(plan.as_ref()) + .indent(true) + .to_string(); + let actual: Vec<&str> = formatted.trim().lines().collect(); + println!("{:#?}", actual); +} + impl PhysicalOptimizerRule for CombinePartialFinalAggregate { fn optimize( &self, plan: Arc, _config: &ConfigOptions, ) -> Result> { + // print_plan(&plan); plan.transform_down(&|plan| { let transformed = plan.as_any() diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index 565f76affa9f..f5e9ca183a8b 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -54,7 +54,8 @@ use datafusion_physical_expr::utils::{ map_columns_before_projection, ordering_satisfy_requirement_concrete, }; use datafusion_physical_expr::{ - expr_list_eq_strict_order, PhysicalExpr, PhysicalSortRequirement, + expr_list_eq_strict_order, OrderingEquivalenceProperties, PhysicalExpr, + PhysicalSortRequirement, }; use datafusion_common::internal_err; @@ -447,7 +448,7 @@ where )) = try_reorder( join_key_pairs.clone(), parent_required, - &join_plan.equivalence_properties(), + &join_plan.ordering_equivalence_properties(), ) { if !new_positions.is_empty() { let new_join_on = new_join_conditions(&left_keys, &right_keys); @@ -688,8 +689,8 @@ pub(crate) fn reorder_join_keys_to_inputs( join_key_pairs, Some(left.output_partitioning()), Some(right.output_partitioning()), - &left.equivalence_properties(), - &right.equivalence_properties(), + &left.ordering_equivalence_properties(), + &right.ordering_equivalence_properties(), ) { if !new_positions.is_empty() { let new_join_on = new_join_conditions(&left_keys, &right_keys); @@ -732,8 +733,8 @@ pub(crate) fn reorder_join_keys_to_inputs( join_key_pairs, Some(left.output_partitioning()), Some(right.output_partitioning()), - &left.equivalence_properties(), - &right.equivalence_properties(), + &left.ordering_equivalence_properties(), + &right.ordering_equivalence_properties(), ) { if !new_positions.is_empty() { let new_join_on = new_join_conditions(&left_keys, &right_keys); @@ -765,8 +766,8 @@ fn reorder_current_join_keys( join_keys: JoinKeyPairs, left_partition: Option, right_partition: Option, - left_equivalence_properties: &EquivalenceProperties, - right_equivalence_properties: &EquivalenceProperties, + left_equivalence_properties: &OrderingEquivalenceProperties, + right_equivalence_properties: &OrderingEquivalenceProperties, ) -> Option<(JoinKeyPairs, Vec)> { match (left_partition, right_partition.clone()) { (Some(Partitioning::Hash(left_exprs, _)), _) => { @@ -791,8 +792,11 @@ fn reorder_current_join_keys( fn try_reorder( join_keys: JoinKeyPairs, expected: &[Arc], - equivalence_properties: &EquivalenceProperties, + equivalence_properties: &OrderingEquivalenceProperties, ) -> Option<(JoinKeyPairs, Vec)> { + // println!("join_keys:{:?}", join_keys); + // println!("expected:{:?}", expected); + // println!("equivalence_properties:{:?}", equivalence_properties); let mut normalized_expected = vec![]; let mut normalized_left_keys = vec![]; let mut normalized_right_keys = vec![]; @@ -803,7 +807,7 @@ fn try_reorder( || expr_list_eq_strict_order(expected, &join_keys.right_keys) { return Some((join_keys, vec![])); - } else if !equivalence_properties.classes().is_empty() { + } else if !equivalence_properties.eq_classes().is_empty() { normalized_expected = expected .iter() .map(|e| equivalence_properties.normalize_expr(e.clone())) @@ -824,6 +828,9 @@ fn try_reorder( .collect::>(); assert_eq!(join_keys.right_keys.len(), normalized_right_keys.len()); + println!("normalized_expected: {:?}", normalized_expected); + println!("normalized_left_keys: {:?}", normalized_left_keys); + println!("normalized_right_keys: {:?}", normalized_right_keys); if expr_list_eq_strict_order(&normalized_expected, &normalized_left_keys) || expr_list_eq_strict_order(&normalized_expected, &normalized_right_keys) { @@ -1331,7 +1338,9 @@ fn ensure_distribution( if !child .output_partitioning() - .satisfy(requirement.clone(), || child.equivalence_properties()) + .satisfy(requirement.clone(), || { + child.ordering_equivalence_properties() + }) { // Satisfy the distribution requirement if it is unmet. match requirement { @@ -1359,7 +1368,6 @@ fn ensure_distribution( if !ordering_satisfy_requirement_concrete( existing_ordering, required_input_ordering, - || child.equivalence_properties(), || child.ordering_equivalence_properties(), ) || !order_preserving_variants_desirable { diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index b6f2adac1be4..4764e38a0581 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -426,6 +426,14 @@ fn parallelize_sorts( })) } +fn print_plan(plan: &Arc) -> () { + let formatted = crate::physical_plan::displayable(plan.as_ref()) + .indent(true) + .to_string(); + let actual: Vec<&str> = formatted.trim().lines().collect(); + println!("{:#?}", actual); +} + /// This function enforces sorting requirements and makes optimizations without /// violating these requirements whenever possible. fn ensure_sorting( @@ -438,6 +446,7 @@ fn ensure_sorting( let plan = requirements.plan; let mut children = plan.children(); let mut sort_onwards = requirements.sort_onwards; + // print_plan(&plan); if let Some(result) = analyze_immediate_sort_removal(&plan, &sort_onwards) { return Ok(Transformed::Yes(result)); } @@ -454,7 +463,6 @@ fn ensure_sorting( if !ordering_satisfy_requirement_concrete( physical_ordering, &required_ordering, - || child.equivalence_properties(), || child.ordering_equivalence_properties(), ) { // Make sure we preserve the ordering requirements: @@ -517,11 +525,16 @@ fn analyze_immediate_sort_removal( ) -> Option { if let Some(sort_exec) = plan.as_any().downcast_ref::() { let sort_input = sort_exec.input().clone(); + // println!("sort input"); + // print_plan(&sort_input); + // println!("sort_input.output_ordering(): {:?}", sort_input.output_ordering()); + // println!("sort_exec.output_ordering(): {:?}", sort_exec.output_ordering()); + // println!("sort_input.ordering_equivalence_properties(): {:?}", sort_input.ordering_equivalence_properties()); + // If this sort is unnecessary, we should remove it: if ordering_satisfy( sort_input.output_ordering(), sort_exec.output_ordering(), - || sort_input.equivalence_properties(), || sort_input.ordering_equivalence_properties(), ) { // Since we know that a `SortExec` has exactly one child, @@ -807,24 +820,17 @@ fn can_skip_sort( }; let orderby_exprs = convert_to_expr(orderby_keys); let physical_ordering_exprs = convert_to_expr(physical_ordering); - let equal_properties = || input.equivalence_properties(); // indices of the order by expressions among input ordering expressions - let ob_indices = get_indices_of_matching_exprs( - &orderby_exprs, - &physical_ordering_exprs, - equal_properties, - ); + let ob_indices = + get_indices_of_matching_exprs(&orderby_exprs, &physical_ordering_exprs); if ob_indices.len() != orderby_exprs.len() { // If all order by expressions are not in the input ordering, // there is no way to remove a sort -- immediately return: return Ok(None); } // indices of the partition by expressions among input ordering expressions - let pb_indices = get_indices_of_matching_exprs( - partitionby_exprs, - &physical_ordering_exprs, - equal_properties, - ); + let pb_indices = + get_indices_of_matching_exprs(partitionby_exprs, &physical_ordering_exprs); let ordered_merged_indices = merge_and_order_indices(&pb_indices, &ob_indices); // Indices of order by columns that doesn't seen in partition by // Equivalently (Order by columns) ∖ (Partition by columns) where `∖` represents set difference. @@ -870,11 +876,8 @@ fn can_skip_sort( // All of the partition by columns defines a consecutive range from zero. let ordered_range = &ordered_pb_indices[0..first_n]; let input_pb_exprs = get_at_indices(&physical_ordering_exprs, ordered_range)?; - let partially_ordered_indices = get_indices_of_matching_exprs( - &input_pb_exprs, - partitionby_exprs, - equal_properties, - ); + let partially_ordered_indices = + get_indices_of_matching_exprs(&input_pb_exprs, partitionby_exprs); PartitionSearchMode::PartiallySorted(partially_ordered_indices) } else { // None of the partition by columns defines a consecutive range from zero. @@ -2781,3 +2784,535 @@ mod tests { Ok(()) } } + +mod tmp_tests { + use crate::assert_batches_eq; + use crate::physical_optimizer::utils::get_plan_string; + use crate::physical_plan::{collect, displayable, ExecutionPlan}; + use crate::prelude::SessionContext; + use arrow::util::pretty::print_batches; + use datafusion_common::Result; + use datafusion_execution::config::SessionConfig; + use std::sync::Arc; + + fn print_plan(plan: &Arc) -> Result<()> { + let formatted = displayable(plan.as_ref()).indent(true).to_string(); + let actual: Vec<&str> = formatted.trim().lines().collect(); + println!("{:#?}", actual); + Ok(()) + } + + #[tokio::test] + async fn test_query() -> Result<()> { + let config = SessionConfig::new().with_target_partitions(1); + let ctx = SessionContext::with_config(config); + + ctx.sql("CREATE TABLE tab0(col0 INTEGER, col1 INTEGER, col2 INTEGER)") + .await?; + + let sql = "SELECT l.col0, LAST_VALUE(r.col1 ORDER BY r.col0) as last_col1 + FROM tab0 as l + JOIN tab0 as r + ON l.col0 = r.col0 + GROUP BY l.col0, l.col1, l.col2 + ORDER BY l.col0;"; + + let msg = format!("Creating logical plan for '{sql}'"); + let dataframe = ctx.sql(sql).await.expect(&msg); + let physical_plan = dataframe.create_physical_plan().await?; + print_plan(&physical_plan)?; + let actual = collect(physical_plan.clone(), ctx.task_ctx()).await?; + print_batches(&actual)?; + + let expected_optimized_lines: Vec<&str> = vec![ + "ProjectionExec: expr=[col0@0 as col0, LAST_VALUE(r.col1) ORDER BY [r.col0 ASC NULLS LAST]@3 as last_col1]", + " AggregateExec: mode=Final, gby=[col0@0 as col0, col1@1 as col1, col2@2 as col2], aggr=[LAST_VALUE(r.col1)], ordering_mode=PartiallyOrdered", + " AggregateExec: mode=Partial, gby=[col0@0 as col0, col1@1 as col1, col2@2 as col2], aggr=[LAST_VALUE(r.col1)], ordering_mode=PartiallyOrdered", + " SortExec: expr=[col0@3 ASC NULLS LAST]", + " CoalesceBatchesExec: target_batch_size=8192", + " HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(col0@0, col0@0)]", + " MemoryExec: partitions=1, partition_sizes=[0]", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + + // Get string representation of the plan + let actual = get_plan_string(&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" + ); + + Ok(()) + } + + #[tokio::test] + async fn test_query2() -> Result<()> { + let config = SessionConfig::new().with_target_partitions(2); + let ctx = SessionContext::with_config(config); + + ctx.sql("CREATE TABLE tab0(col0 INTEGER, col1 INTEGER, col2 INTEGER)") + .await?; + + let sql = "SELECT l.col0, LAST_VALUE(r.col1 ORDER BY r.col0) as last_col1 + FROM tab0 as l + JOIN tab0 as r + ON l.col0 = r.col0 + GROUP BY l.col0, l.col1, l.col2 + ORDER BY l.col0;"; + + let msg = format!("Creating logical plan for '{sql}'"); + let dataframe = ctx.sql(sql).await.expect(&msg); + let physical_plan = dataframe.create_physical_plan().await?; + print_plan(&physical_plan)?; + let actual = collect(physical_plan.clone(), ctx.task_ctx()).await?; + print_batches(&actual)?; + + let expected_optimized_lines: Vec<&str> = vec![ + "SortPreservingMergeExec: [col0@0 ASC NULLS LAST]", + " ProjectionExec: expr=[col0@0 as col0, LAST_VALUE(r.col1) ORDER BY [r.col0 ASC NULLS LAST]@3 as last_col1]", + " AggregateExec: mode=FinalPartitioned, gby=[col0@0 as col0, col1@1 as col1, col2@2 as col2], aggr=[LAST_VALUE(r.col1)], ordering_mode=PartiallyOrdered", + " SortExec: expr=[col0@0 ASC NULLS LAST]", + " CoalesceBatchesExec: target_batch_size=8192", + " RepartitionExec: partitioning=Hash([col0@0, col1@1, col2@2], 2), input_partitions=2", + " AggregateExec: mode=Partial, gby=[col0@0 as col0, col1@1 as col1, col2@2 as col2], aggr=[LAST_VALUE(r.col1)], ordering_mode=PartiallyOrdered", + " SortExec: expr=[col0@3 ASC NULLS LAST]", + " CoalesceBatchesExec: target_batch_size=8192", + " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(col0@0, col0@0)]", + " CoalesceBatchesExec: target_batch_size=8192", + " RepartitionExec: partitioning=Hash([col0@0], 2), input_partitions=2", + " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", + " MemoryExec: partitions=1, partition_sizes=[0]", + " CoalesceBatchesExec: target_batch_size=8192", + " RepartitionExec: partitioning=Hash([col0@0], 2), input_partitions=2", + " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", + " MemoryExec: partitions=1, partition_sizes=[0]", + ]; + + // Get string representation of the plan + let actual = get_plan_string(&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" + ); + + Ok(()) + } + + #[tokio::test] + async fn test_query3() -> Result<()> { + let config = SessionConfig::new().with_target_partitions(1); + let ctx = SessionContext::with_config(config); + + ctx.sql( + "CREATE EXTERNAL TABLE multiple_ordered_table ( + a0 INTEGER, + a INTEGER, + b INTEGER, + c INTEGER, + d INTEGER + ) + STORED AS CSV + WITH HEADER ROW + WITH ORDER (a ASC) + WITH ORDER (b ASC) + WITH ORDER (c ASC) + LOCATION '../core/tests/data/window_2.csv'", + ) + .await?; + + let sql = "SELECT (b+a+c) AS result + FROM multiple_ordered_table + ORDER BY result;"; + + let msg = format!("Creating logical plan for '{sql}'"); + let dataframe = ctx.sql(sql).await.expect(&msg); + let physical_plan = dataframe.create_physical_plan().await?; + print_plan(&physical_plan)?; + let actual = collect(physical_plan.clone(), ctx.task_ctx()).await?; + print_batches(&actual)?; + + let expected_optimized_lines: Vec<&str> = vec![ + "ProjectionExec: expr=[b@1 + a@0 + c@2 as result]", + " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + ]; + + // Get string representation of the plan + let actual = get_plan_string(&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" + ); + + Ok(()) + } + + #[tokio::test] + async fn test_query4() -> Result<()> { + let config = SessionConfig::new().with_target_partitions(1); + let ctx = SessionContext::with_config(config); + + ctx.sql( + "CREATE EXTERNAL TABLE aggregate_test_100 ( + c1 VARCHAR NOT NULL, + c2 TINYINT NOT NULL, + c3 SMALLINT NOT NULL, + c4 SMALLINT, + c5 INT, + c6 BIGINT NOT NULL, + c7 SMALLINT NOT NULL, + c8 INT NOT NULL, + c9 BIGINT UNSIGNED NOT NULL, + c10 VARCHAR NOT NULL, + c11 FLOAT NOT NULL, + c12 DOUBLE NOT NULL, + c13 VARCHAR NOT NULL + ) + STORED AS CSV + WITH HEADER ROW + LOCATION '../../testing/data/csv/aggregate_test_100.csv'", + ) + .await?; + + let sql = "SELECT c3, + SUM(c9) OVER(ORDER BY c3+c4 DESC, c9 DESC, c2 ASC) as sum1, + SUM(c9) OVER(ORDER BY c3+c4 ASC, c9 ASC ) as sum2 + FROM aggregate_test_100 + LIMIT 5"; + + let msg = format!("Creating logical plan for '{sql}'"); + let dataframe = ctx.sql(sql).await.expect(&msg); + let physical_plan = dataframe.create_physical_plan().await?; + print_plan(&physical_plan)?; + let actual = collect(physical_plan.clone(), ctx.task_ctx()).await?; + print_batches(&actual)?; + + let expected_optimized_lines: Vec<&str> = vec![ + "ProjectionExec: expr=[c3@0 as c3, 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@3 as sum1, 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@4 as sum2]", + " 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)) }]", + " ProjectionExec: expr=[c3@1 as c3, c4@2 as c4, c9@3 as c9, 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@4 as 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]", + " 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 }], mode=[Sorted]", + " SortExec: expr=[c3@1 + c4@2 DESC,c9@3 DESC,c2@0 ASC NULLS LAST]", + " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3, c4, c9], has_header=true", + ]; + + // Get string representation of the plan + let actual = get_plan_string(&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" + ); + + Ok(()) + } + + #[tokio::test] + async fn test_query5() -> Result<()> { + let config = SessionConfig::new().with_target_partitions(1); + let ctx = SessionContext::with_config(config); + + ctx.sql( + "CREATE EXTERNAL TABLE annotated_data ( + a0 INTEGER, + a INTEGER, + b INTEGER, + c INTEGER, + d INTEGER + ) + STORED AS CSV + WITH HEADER ROW + WITH ORDER (a ASC NULLS FIRST, b ASC, c ASC) + LOCATION '../core/tests/data/window_2.csv'", + ) + .await?; + + let sql = "SELECT * + FROM annotated_data as l_table + JOIN (SELECT *, ROW_NUMBER() OVER() as rn1 + FROM annotated_data) as r_table + ON l_table.a = r_table.a + ORDER BY r_table.rn1"; + + let msg = format!("Creating logical plan for '{sql}'"); + let dataframe = ctx.sql(sql).await.expect(&msg); + let physical_plan = dataframe.create_physical_plan().await?; + print_plan(&physical_plan)?; + let actual = collect(physical_plan.clone(), ctx.task_ctx()).await?; + // print_batches(&actual)?; + + let expected_optimized_lines: Vec<&str> = vec![ + "CoalesceBatchesExec: target_batch_size=8192", + " HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@1, a@1)]", + " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true", + " ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@5 as rn1]", + " BoundedWindowAggExec: wdw=[ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: \"ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING\", 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)) }], mode=[Sorted]", + " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true", + ]; + + // Get string representation of the plan + let actual = get_plan_string(&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" + ); + + Ok(()) + } + + #[tokio::test] + async fn test_query6() -> Result<()> { + let config = SessionConfig::new().with_target_partitions(1); + let ctx = SessionContext::with_config(config); + + ctx.sql( + "CREATE EXTERNAL TABLE annotated_data_finite2 ( + a0 INTEGER, + a INTEGER, + b INTEGER, + c INTEGER, + d INTEGER + ) + STORED AS CSV + WITH HEADER ROW + WITH ORDER (a ASC, b ASC, c ASC) + LOCATION '../core/tests/data/window_2.csv'", + ) + .await?; + + let sql = "SELECT a, b, c, + SUM(c) OVER(PARTITION BY a, d ORDER BY b, c ASC ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING) as sum1, + SUM(c) OVER(PARTITION BY a, d ORDER BY b, c ASC ROWS BETWEEN 1 FOLLOWING AND 5 FOLLOWING) as sum2, + SUM(c) OVER(PARTITION BY d ORDER BY a, b, c ASC ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING) as sum3, + SUM(c) OVER(PARTITION BY d ORDER BY a, b, c ASC ROWS BETWEEN 5 PRECEDING AND 1 PRECEDING) as sum4, + SUM(c) OVER(PARTITION BY a, b ORDER BY c ASC ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING) as sum5, + SUM(c) OVER(PARTITION BY a, b ORDER BY c ASC ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING) as sum6, + SUM(c) OVER(PARTITION BY b, a ORDER BY c ASC ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING) as sum7, + SUM(c) OVER(PARTITION BY b, a ORDER BY c ASC ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING) as sum8, + SUM(c) OVER(PARTITION BY a, b, d ORDER BY c ASC ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING) as sum9, + SUM(c) OVER(PARTITION BY a, b, d ORDER BY c ASC ROWS BETWEEN 5 PRECEDING AND CURRENT ROW) as sum10, + SUM(c) OVER(PARTITION BY b, a, d ORDER BY c ASC ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING) as sum11, + SUM(c) OVER(PARTITION BY b, a, d ORDER BY c ASC ROWS BETWEEN CURRENT ROW AND 1 FOLLOWING) as sum12 + FROM annotated_data_finite2 + ORDER BY c + LIMIT 5"; + + let msg = format!("Creating logical plan for '{sql}'"); + let dataframe = ctx.sql(sql).await.expect(&msg); + let physical_plan = dataframe.create_physical_plan().await?; + print_plan(&physical_plan)?; + let actual = collect(physical_plan.clone(), ctx.task_ctx()).await?; + // print_batches(&actual)?; + + let expected_optimized_lines: Vec<&str> = vec![ + "GlobalLimitExec: skip=0, fetch=5", + " SortExec: fetch=5, expr=[c@2 ASC NULLS LAST]", + " ProjectionExec: expr=[a@1 as a, b@2 as b, c@3 as c, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@9 as sum1, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 1 FOLLOWING AND 5 FOLLOWING@10 as sum2, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@15 as sum3, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 1 PRECEDING@16 as sum4, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@5 as sum5, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING@6 as sum6, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@11 as sum7, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING@12 as sum8, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@7 as sum9, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND CURRENT ROW@8 as sum10, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@13 as sum11, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN CURRENT ROW AND 1 FOLLOWING@14 as sum12]", + " BoundedWindowAggExec: wdw=[SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: \"SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(2)), end_bound: Following(UInt64(1)) }, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 1 PRECEDING: Ok(Field { name: \"SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 1 PRECEDING\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: Preceding(UInt64(1)) }], mode=[Sorted]", + " SortExec: expr=[d@4 ASC NULLS LAST,a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST]", + " BoundedWindowAggExec: wdw=[SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: \"SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(2)), end_bound: Following(UInt64(1)) }, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN CURRENT ROW AND 1 FOLLOWING: Ok(Field { name: \"SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN CURRENT ROW AND 1 FOLLOWING\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: CurrentRow, end_bound: Following(UInt64(1)) }], mode=[Sorted]", + " SortExec: expr=[b@2 ASC NULLS LAST,a@1 ASC NULLS LAST,d@4 ASC NULLS LAST,c@3 ASC NULLS LAST]", + " BoundedWindowAggExec: wdw=[SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: \"SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(2)), end_bound: Following(UInt64(1)) }, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING: Ok(Field { name: \"SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: Following(UInt64(5)) }], mode=[Sorted]", + " SortExec: expr=[b@2 ASC NULLS LAST,a@1 ASC NULLS LAST,c@3 ASC NULLS LAST]", + " BoundedWindowAggExec: wdw=[SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: \"SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(2)), end_bound: Following(UInt64(1)) }, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 1 FOLLOWING AND 5 FOLLOWING: Ok(Field { name: \"SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 1 FOLLOWING AND 5 FOLLOWING\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Following(UInt64(1)), end_bound: Following(UInt64(5)) }], mode=[Sorted]", + " SortExec: expr=[a@1 ASC NULLS LAST,d@4 ASC NULLS LAST,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST]", + " BoundedWindowAggExec: wdw=[SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: \"SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(2)), end_bound: Following(UInt64(1)) }, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND CURRENT ROW: Ok(Field { name: \"SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND CURRENT ROW\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: CurrentRow }], mode=[Sorted]", + " SortExec: expr=[a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,d@4 ASC NULLS LAST,c@3 ASC NULLS LAST]", + " BoundedWindowAggExec: wdw=[SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: \"SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(2)), end_bound: Following(UInt64(1)) }, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING: Ok(Field { name: \"SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: Following(UInt64(5)) }], mode=[Sorted]", + " ProjectionExec: expr=[CAST(c@2 AS Int64) as CAST(annotated_data_finite2.c AS Int64)annotated_data_finite2.c, a@0 as a, b@1 as b, c@2 as c, d@3 as d]", + " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true", + ]; + + // Get string representation of the plan + let actual = get_plan_string(&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" + ); + + Ok(()) + } + + #[tokio::test] + async fn test_query7() -> Result<()> { + let config = SessionConfig::new().with_target_partitions(1); + let ctx = SessionContext::with_config(config); + + ctx.sql( + "CREATE TABLE t1(t1_id INT, t1_name TEXT, t1_int INT) AS VALUES + (11, 'a', 1), + (22, 'b', 2), + (33, 'c', 3), + (44, 'd', 4);", + ) + .await?; + + ctx.sql( + "CREATE TABLE t2(t2_id INT, t2_name TEXT, t2_int INT) AS VALUES + (11, 'z', 3), + (22, 'y', 1), + (44, 'x', 3), + (55, 'w', 3);", + ) + .await?; + + let sql = + "SELECT t1_id, (SELECT count(*) FROM t2 WHERE t2.t2_int = t1.t1_int) from t1"; + + let msg = format!("Creating logical plan for '{sql}'"); + let dataframe = ctx.sql(sql).await.expect(&msg); + let physical_plan = dataframe.create_physical_plan().await?; + print_plan(&physical_plan)?; + let actual = collect(physical_plan.clone(), ctx.task_ctx()).await?; + // print_batches(&actual)?; + + let expected_optimized_lines: Vec<&str> = vec![ + "ProjectionExec: expr=[t1_id@0 as t1_id, CASE WHEN __always_true@4 IS NULL THEN 0 ELSE COUNT(*)@2 END as COUNT(*)]", + " ProjectionExec: expr=[t1_id@3 as t1_id, t1_int@4 as t1_int, COUNT(*)@0 as COUNT(*), t2_int@1 as t2_int, __always_true@2 as __always_true]", + " CoalesceBatchesExec: target_batch_size=8192", + " HashJoinExec: mode=CollectLeft, join_type=Right, on=[(t2_int@1, t1_int@1)]", + " ProjectionExec: expr=[COUNT(*)@2 as COUNT(*), t2_int@0 as t2_int, __always_true@1 as __always_true]", + " AggregateExec: mode=Final, gby=[t2_int@0 as t2_int, __always_true@1 as __always_true], aggr=[COUNT(*)]", + " AggregateExec: mode=Partial, gby=[t2_int@0 as t2_int, true as __always_true], aggr=[COUNT(*)]", + " MemoryExec: partitions=1, partition_sizes=[1]", + " MemoryExec: partitions=1, partition_sizes=[1]", + ]; + + // Get string representation of the plan + let actual = get_plan_string(&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" + ); + + Ok(()) + } + + #[tokio::test] + async fn test_query8() -> Result<()> { + let config = SessionConfig::new().with_target_partitions(1); + let ctx = SessionContext::with_config(config); + + ctx.sql( + "CREATE UNBOUNDED EXTERNAL TABLE annotated_data_infinite2 ( + a0 INTEGER, + a INTEGER, + b INTEGER, + c INTEGER, + d INTEGER + ) + STORED AS CSV + WITH HEADER ROW + WITH ORDER (a ASC, b ASC, c ASC) + LOCATION '../core/tests/data/window_2.csv'", + ) + .await?; + + let sql = "SELECT a, d, + SUM(c ORDER BY a DESC) as summation1 + FROM annotated_data_infinite2 + GROUP BY d, a"; + + let msg = format!("Creating logical plan for '{sql}'"); + let dataframe = ctx.sql(sql).await.expect(&msg); + let physical_plan = dataframe.create_physical_plan().await?; + print_plan(&physical_plan)?; + let actual = collect(physical_plan.clone(), ctx.task_ctx()).await?; + // print_batches(&actual)?; + + let expected_optimized_lines: Vec<&str> = vec![ + "ProjectionExec: expr=[a@1 as a, d@0 as d, SUM(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]@2 as summation1]", + " AggregateExec: mode=Single, gby=[d@2 as d, a@0 as a], aggr=[SUM(annotated_data_infinite2.c)], ordering_mode=PartiallyOrdered", + " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", + ]; + + // Get string representation of the plan + let actual = get_plan_string(&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" + ); + + Ok(()) + } + + #[tokio::test] + async fn test_query9() -> Result<()> { + let config = SessionConfig::new().with_target_partitions(1); + let ctx = SessionContext::with_config(config); + + ctx.sql( + "CREATE EXTERNAL TABLE aggregate_test_100 ( + c1 VARCHAR NOT NULL, + c2 TINYINT NOT NULL, + c3 SMALLINT NOT NULL, + c4 SMALLINT, + c5 INT, + c6 BIGINT NOT NULL, + c7 SMALLINT NOT NULL, + c8 INT NOT NULL, + c9 INT UNSIGNED NOT NULL, + c10 BIGINT UNSIGNED NOT NULL, + c11 FLOAT NOT NULL, + c12 DOUBLE NOT NULL, + c13 VARCHAR NOT NULL + ) + STORED AS CSV + WITH HEADER ROW + LOCATION '../../testing/data/csv/aggregate_test_100.csv'", + ) + .await?; + + let sql = "WITH indices AS ( + SELECT 1 AS idx UNION ALL + SELECT 2 AS idx UNION ALL + SELECT 3 AS idx UNION ALL + SELECT 4 AS idx UNION ALL + SELECT 5 AS idx + ) + SELECT data.arr[indices.idx] as element, array_length(data.arr) as array_len, dummy + FROM ( + SELECT array_agg(distinct c2) as arr, count(1) as dummy FROM aggregate_test_100 + ) data + CROSS JOIN indices + ORDER BY 1"; + + // let sql = "SELECT array_agg(distinct c2) as arr, count(1) as dummy FROM aggregate_test_100"; + + let msg = format!("Creating logical plan for '{sql}'"); + let dataframe = ctx.sql(sql).await.expect(&msg); + let physical_plan = dataframe.create_physical_plan().await?; + print_plan(&physical_plan)?; + let actual = collect(physical_plan.clone(), ctx.task_ctx()).await?; + // print_batches(&actual)?; + + let expected_optimized_lines: Vec<&str> = vec![ + "SortPreservingMergeExec: [element@0 ASC NULLS LAST]", + " SortExec: expr=[element@0 ASC NULLS LAST]", + " ProjectionExec: expr=[(arr@0).[idx@2] as element, array_length(arr@0) as array_len, dummy@1 as dummy]", + " CrossJoinExec", + " ProjectionExec: expr=[ARRAY_AGG(DISTINCT aggregate_test_100.c2)@0 as arr, COUNT(Int64(1))@1 as dummy]", + " AggregateExec: mode=Single, gby=[], aggr=[ARRAY_AGG(DISTINCT aggregate_test_100.c2), COUNT(Int64(1))]", + " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2], has_header=true", + " UnionExec", + " ProjectionExec: expr=[1 as idx]", + " EmptyExec: produce_one_row=true", + " ProjectionExec: expr=[2 as idx]", + " EmptyExec: produce_one_row=true", + " ProjectionExec: expr=[3 as idx]", + " EmptyExec: produce_one_row=true", + " ProjectionExec: expr=[4 as idx]", + " EmptyExec: produce_one_row=true", + " ProjectionExec: expr=[5 as idx]", + " EmptyExec: produce_one_row=true", + ]; + + // Get string representation of the plan + let actual = get_plan_string(&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" + ); + + Ok(()) + } +} diff --git a/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs b/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs index b406a541054d..58ce3954fd79 100644 --- a/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs +++ b/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs @@ -262,7 +262,6 @@ pub(crate) fn replace_with_order_preserving_variants( if ordering_satisfy( updated_sort_input.output_ordering(), plan.output_ordering(), - || updated_sort_input.equivalence_properties(), || updated_sort_input.ordering_equivalence_properties(), ) { return Ok(Transformed::Yes(OrderPreservationContext { diff --git a/datafusion/core/src/physical_optimizer/sort_pushdown.rs b/datafusion/core/src/physical_optimizer/sort_pushdown.rs index 629011cb0faa..54fdac646bbd 100644 --- a/datafusion/core/src/physical_optimizer/sort_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/sort_pushdown.rs @@ -130,12 +130,9 @@ pub(crate) fn pushdown_sorts( let err = || DataFusionError::Plan(ERR_MSG.to_string()); if let Some(sort_exec) = plan.as_any().downcast_ref::() { let mut new_plan = plan.clone(); - if !ordering_satisfy_requirement( - plan.output_ordering(), - parent_required, - || plan.equivalence_properties(), - || plan.ordering_equivalence_properties(), - ) { + if !ordering_satisfy_requirement(plan.output_ordering(), parent_required, || { + plan.ordering_equivalence_properties() + }) { // If the current plan is a SortExec, modify it to satisfy parent requirements: let parent_required_expr = PhysicalSortRequirement::to_sort_exprs( parent_required.ok_or_else(err)?.iter().cloned(), @@ -163,12 +160,9 @@ pub(crate) fn pushdown_sorts( } } else { // Executors other than SortExec - if ordering_satisfy_requirement( - plan.output_ordering(), - parent_required, - || plan.equivalence_properties(), - || plan.ordering_equivalence_properties(), - ) { + if ordering_satisfy_requirement(plan.output_ordering(), parent_required, || { + plan.ordering_equivalence_properties() + }) { // Satisfies parent requirements, immediately return. return Ok(Transformed::Yes(SortPushDown { required_ordering: None, @@ -269,7 +263,6 @@ fn pushdown_requirement_to_children( .map(|req| PhysicalSortRequirement::to_sort_exprs(req.to_vec())) .as_deref(), plan.output_ordering(), - || plan.equivalence_properties(), || plan.ordering_equivalence_properties(), ) ) @@ -304,20 +297,14 @@ fn determine_children_requirement( request_child: Option<&[PhysicalSortRequirement]>, child_plan: Arc, ) -> RequirementsCompatibility { - if requirements_compatible( - request_child, - parent_required, - || child_plan.ordering_equivalence_properties(), - || child_plan.equivalence_properties(), - ) { + if requirements_compatible(request_child, parent_required, || { + child_plan.ordering_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.ordering_equivalence_properties(), - || child_plan.equivalence_properties(), - ) { + } else if requirements_compatible(parent_required, request_child, || { + child_plan.ordering_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) @@ -349,7 +336,6 @@ fn try_pushdown_requirements_to_join( Ok(ordering_satisfy_requirement( new_output_ordering.as_deref(), parent_required, - || smj.equivalence_properties(), || smj.ordering_equivalence_properties(), ) .then(|| { diff --git a/datafusion/core/src/physical_optimizer/utils.rs b/datafusion/core/src/physical_optimizer/utils.rs index b4dd75e5864b..435cf8fd2dc1 100644 --- a/datafusion/core/src/physical_optimizer/utils.rs +++ b/datafusion/core/src/physical_optimizer/utils.rs @@ -109,12 +109,9 @@ pub fn add_sort_above( fetch: Option, ) -> Result<()> { // If the ordering requirement is already satisfied, do not add a sort. - if !ordering_satisfy( - node.output_ordering(), - Some(&sort_expr), - || node.equivalence_properties(), - || node.ordering_equivalence_properties(), - ) { + if !ordering_satisfy(node.output_ordering(), Some(&sort_expr), || { + node.ordering_equivalence_properties() + }) { let new_sort = SortExec::new(sort_expr, node.clone()).with_fetch(fetch); *node = Arc::new(if node.output_partitioning().partition_count() > 1 { diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 2328ffce235d..6eea504fc307 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -2433,6 +2433,7 @@ mod tests { .build()?; let execution_plan = plan(&logical_plan).await?; + print_plan(&execution_plan); let final_hash_agg = execution_plan .as_any() .downcast_ref::() @@ -2789,3 +2790,9 @@ digraph { assert_contains!(generated_graph, expected_tooltip); } } + +fn print_plan(plan: &Arc) -> () { + let formatted = crate::physical_plan::displayable(plan.as_ref()).indent(true).to_string(); + let actual: Vec<&str> = formatted.trim().lines().collect(); + println!("{:#?}", actual); +} diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 369c139aa30b..b21abbf58a73 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -16,17 +16,19 @@ // under the License. use crate::expressions::{CastExpr, Column}; -use crate::utils::{collect_columns, merge_vectors}; +use crate::utils::{collect_columns, get_indices_of_matching_exprs, merge_vectors}; use crate::{ LexOrdering, LexOrderingRef, LexOrderingReq, PhysicalExpr, PhysicalSortExpr, PhysicalSortRequirement, }; use arrow::datatypes::SchemaRef; -use arrow_schema::Fields; +use arrow_schema::{Fields, SortOptions}; +use crate::sort_properties::{ExprOrdering, SortProperties}; use datafusion_common::tree_node::{Transformed, TreeNode}; -use datafusion_common::Result; +use datafusion_common::utils::longest_consecutive_prefix; +use datafusion_common::{JoinType, Result}; use itertools::izip; use std::collections::{HashMap, HashSet}; use std::hash::Hash; @@ -211,6 +213,7 @@ impl EquivalenceProperties { /// and treat `a ASC` and `b DESC` as the same ordering requirement. #[derive(Debug, Clone)] pub struct OrderingEquivalenceProperties { + eq_classes: Vec>>, oeq_class: Option, /// Keeps track of expressions that have constant value. constants: Vec>, @@ -221,6 +224,7 @@ impl OrderingEquivalenceProperties { /// Create an empty `OrderingEquivalenceProperties` pub fn new(schema: SchemaRef) -> Self { Self { + eq_classes: vec![], oeq_class: None, constants: vec![], schema, @@ -244,9 +248,16 @@ impl OrderingEquivalenceProperties { self.oeq_class.as_ref() } + pub fn eq_classes(&self) -> &[Vec>] { + &self.eq_classes + } + /// Adds new equal conditions into the EquivalenceProperties. New equal /// conditions usually come from equality predicates in a join/filter. - pub fn add_equal_conditions(&mut self, new_conditions: (&LexOrdering, &LexOrdering)) { + pub fn add_ordering_equal_conditions( + &mut self, + new_conditions: (&LexOrdering, &LexOrdering), + ) { if let Some(class) = &mut self.oeq_class { class.insert(new_conditions.0.clone()); class.insert(new_conditions.1.clone()); @@ -257,6 +268,25 @@ impl OrderingEquivalenceProperties { } } + pub fn add_equal_conditions( + &mut self, + new_conditions: (&Arc, &Arc), + ) { + let (first, second) = new_conditions; + let mut added_to_existing_equalities = false; + self.eq_classes.iter_mut().for_each(|eq_class| { + if physical_exprs_contains(eq_class, first) + && !physical_exprs_contains(eq_class, second) + { + eq_class.push(second.clone()); + added_to_existing_equalities = true; + } + }); + if !added_to_existing_equalities && !first.eq(second) { + self.eq_classes.push(vec![first.clone(), second.clone()]); + } + } + /// Add physical expression that have constant value to the `self.constants` pub fn with_constants(mut self, constants: Vec>) -> Self { constants.into_iter().for_each(|constant| { @@ -271,6 +301,31 @@ impl OrderingEquivalenceProperties { self.schema.clone() } + fn normalize_with_eq_classes( + &self, + sort_reqs: &[PhysicalSortRequirement], + ) -> Vec { + // println!("sort_reqs: {:?}", sort_reqs); + // println!("self.eq_classes: {:?}", self.eq_classes); + let normalized_sort_reqs = sort_reqs + .iter() + .map(|sort_req| { + for eq_class in &self.eq_classes { + if physical_exprs_contains(eq_class, &sort_req.expr) { + return PhysicalSortRequirement { + expr: eq_class[0].clone(), + options: sort_req.options, + }; + } + } + sort_req.clone() + }) + .collect::>(); + let normalized_sort_reqs = + prune_sort_reqs_with_constants(&normalized_sort_reqs, &self.constants); + normalized_sort_reqs + } + /// This function normalizes `sort_reqs` by /// - removing expressions that have constant value from requirement /// - replacing sections that are in the `self.oeq_class.others` with `self.oeq_class.head` @@ -279,17 +334,18 @@ impl OrderingEquivalenceProperties { &self, sort_reqs: &[PhysicalSortRequirement], ) -> Vec { - let normalized_sort_reqs = - prune_sort_reqs_with_constants(sort_reqs, &self.constants); + let normalized_sort_reqs = self.normalize_with_eq_classes(sort_reqs); let mut normalized_sort_reqs = collapse_lex_req(normalized_sort_reqs); if let Some(oeq_class) = &self.oeq_class { for item in oeq_class.others() { let item = PhysicalSortRequirement::from_sort_exprs(item); + let item = self.normalize_with_eq_classes(&item); let item = prune_sort_reqs_with_constants(&item, &self.constants); let ranges = get_compatible_ranges(&normalized_sort_reqs, &item); let mut offset: i64 = 0; for Range { start, end } in ranges { let head = PhysicalSortRequirement::from_sort_exprs(oeq_class.head()); + let head = self.normalize_with_eq_classes(&head); let mut head = prune_sort_reqs_with_constants(&head, &self.constants); let updated_start = (start as i64 + offset) as usize; let updated_end = (end as i64 + offset) as usize; @@ -330,6 +386,370 @@ impl OrderingEquivalenceProperties { } false } + + /// Normalizes physical expression according to `EquivalentClass`es inside `self.classes`. + /// expression is replaced with `EquivalentClass::head` expression if it is among `EquivalentClass::others`. + pub fn normalize_expr(&self, expr: Arc) -> Arc { + expr.clone() + .transform(&|expr| { + for class in self.eq_classes() { + if physical_exprs_contains(class, &expr) { + return Ok(Transformed::Yes(class[0].clone())); + } + } + Ok(Transformed::No(expr)) + }) + .unwrap_or(expr) + } + + pub fn normalize_sort_expr(&self, sort_expr: PhysicalSortExpr) -> PhysicalSortExpr { + let PhysicalSortExpr { expr, options } = sort_expr; + let new_expr = self.normalize_expr(expr); + PhysicalSortExpr { + expr: new_expr, + options, + } + } + + pub fn normalize_exprs( + &self, + exprs: &[Arc], + ) -> Vec> { + let res = exprs + .iter() + .map(|expr| self.normalize_expr(expr.clone())) + .collect::>(); + // TODO: Add deduplication check here after normalization + res + } + + pub fn normalize_sort_exprs( + &self, + sort_exprs: &[PhysicalSortExpr], + ) -> Vec { + let res = sort_exprs + .iter() + .map(|sort_expr| self.normalize_sort_expr(sort_expr.clone())) + .collect::>(); + // TODO: Add deduplication check here after normalization + res + } + + fn get_aliased_expr( + alias_map: &HashMap>, + source_to_target_mapping: &[(Arc, Arc)], + expr: &Arc, + ) -> Option> { + for (source, target) in source_to_target_mapping { + if expr.eq(source) { + return Some(target.clone()); + } + } + // for (column, columns) in alias_map { + // let column_expr = Arc::new(column.clone()) as Arc; + // // println!("column_expr:{:?}, expr:{:?}",column_expr, expr); + // if column_expr.eq(expr) { + // // println!("return some"); + // return Some(Arc::new(columns[0].clone())); + // } + // } + None + } + + fn get_eq_class_group( + old_eq_class: &[Vec>], + expr: &Arc, + ) -> Option>> { + for eq_class in old_eq_class { + if physical_exprs_contains(eq_class, expr) { + return Some(eq_class.to_vec()); + } + } + None + } + + fn get_corresponding_expr( + old_eq_class: &[Vec>], + new_eq_class: &[Option>>], + alias_map: &HashMap>, + source_to_target_mapping: &[(Arc, Arc)], + expr: &Arc, + ) -> Option> { + assert_eq!(old_eq_class.len(), new_eq_class.len()); + // for (old, new) in izip!(old_eq_class.iter(), new_eq_class.iter()) { + // if let (true, Some(new_eq)) = (physical_exprs_contains(old, expr), new) { + // return Some(new_eq[0].clone()); + // } + // } + // for (column, columns) in alias_map { + // let column_expr = Arc::new(column.clone()) as Arc; + // if column_expr.eq(expr) { + // return Some(Arc::new(columns[0].clone())); + // } + // } + let children = expr.children(); + if children.is_empty() { + for (source, target) in source_to_target_mapping.iter() { + if source.eq(expr) + || old_eq_class + .iter() + .any(|eq_class| eq_class.iter().any(|item| item.eq(source))) + { + return Some(target.clone()); + } else if let Some(group) = Self::get_eq_class_group(old_eq_class, source) + { + if physical_exprs_contains(&group, expr) { + return Some(target.clone()); + } + } + } + None + } else if let Some(children) = children + .into_iter() + .map(|child| { + Self::get_corresponding_expr( + old_eq_class, + new_eq_class, + alias_map, + source_to_target_mapping, + &child, + ) + }) + .collect::>>() + { + Some(expr.clone().with_new_children(children).unwrap()) + } else { + None + } + + // for (source, target) in source_to_target_mapping.iter(){ + // + // if source.eq(expr) || old_eq_class.iter().any(|eq_class| eq_class.iter().any(|item| item.eq(source))){ + // return Some(target.clone()) + // } else if let Some(group) = Self::get_eq_class_group(old_eq_class, source){ + // if physical_exprs_contains(&group, expr){ + // return Some(target.clone()) + // } + // } + // } + // None + } + + fn get_projected_ordering( + old_eq_class: &[Vec>], + new_eq_class: &[Option>>], + alias_map: &HashMap>, + source_to_target_mapping: &[(Arc, Arc)], + ordering: &[PhysicalSortExpr], + ) -> Option> { + // println!("old_eq_class: {:?}", old_eq_class); + // println!("new_eq_class: {:?}", new_eq_class); + // println!("ordering: {:?}", ordering); + let mut res = vec![]; + for order in ordering { + // println!("order.expr:{:?}", order.expr); + if let Some(new_expr) = Self::get_corresponding_expr( + old_eq_class, + new_eq_class, + alias_map, + source_to_target_mapping, + &order.expr, + ) { + // println!("new_expr:{:?}", new_expr); + res.push(PhysicalSortExpr { + expr: new_expr, + options: order.options, + }) + } else { + break; + } + } + if res.is_empty() { + None + } else { + Some(res) + } + } + + fn get_equivalent_groups( + source_to_target_mapping: &[(Arc, Arc)], + ) -> Vec>> { + let mut res = vec![]; + for (source, target) in source_to_target_mapping { + if res.is_empty() { + res.push((source, vec![target.clone()])); + } + if let Some(idx) = res.iter_mut().position(|(key, _values)| key.eq(source)) { + let (_, values) = &mut res[idx]; + if !physical_exprs_contains(values, target) { + values.push(target.clone()); + } + } + } + res.into_iter() + .filter_map( + |(_, values)| { + if values.len() > 1 { + Some(values) + } else { + None + } + }, + ) + .collect() + // vec![] + } + + pub fn project( + &self, + alias_map: &HashMap>, + source_to_target_mapping: &[(Arc, Arc)], + output_schema: SchemaRef, + ) -> OrderingEquivalenceProperties { + // println!("alias_map: {:?}", alias_map); + // println!("source_to_target_mapping: {:?}", source_to_target_mapping); + let mut res = OrderingEquivalenceProperties::new(output_schema); + + let mut new_eq_classes = vec![]; + let mut new_eq_classes2 = vec![]; + for eq_class in &self.eq_classes { + let new_eq_class = eq_class + .iter() + .filter_map(|expr| { + Self::get_aliased_expr(alias_map, source_to_target_mapping, expr) + }) + .collect::>(); + // println!("new_eq_class:{:?}", new_eq_class); + if new_eq_class.len() > 1 { + new_eq_classes.push(new_eq_class.clone()); + } + if new_eq_class.is_empty() { + new_eq_classes2.push(None); + } else { + new_eq_classes2.push(Some(new_eq_class)); + } + } + let new_classes = Self::get_equivalent_groups(source_to_target_mapping); + // println!("new_classes alias group:{:?}", new_classes); + // TODO: Add check for redundant group + // combine groups with common entries + new_eq_classes.extend(new_classes); + res.eq_classes = new_eq_classes; + + if let Some(oeq_class) = &self.oeq_class { + // println!("old oeq class: {:?}", oeq_class); + let new_ordering = oeq_class + .iter() + .filter_map(|order| { + Self::get_projected_ordering( + &self.eq_classes, + &new_eq_classes2, + alias_map, + source_to_target_mapping, + order, + ) + }) + .collect::>(); + // println!("new_ordering: {:?}", new_ordering); + if !new_ordering.is_empty() { + let head = new_ordering[0].clone(); + let others = new_ordering[1..].to_vec(); + res.oeq_class = Some(OrderingEquivalentClass::new(head, others)); + } + } + for (source, target) in source_to_target_mapping { + let initial_expr = ExprOrdering::new(source.clone()); + let transformed = initial_expr + .transform_up(&|expr| update_ordering(expr, &self)) + .unwrap(); + if let Some(SortProperties::Ordered(sort_options)) = transformed.state { + let sort_expr = PhysicalSortExpr { + expr: target.clone(), + options: sort_options, + }; + if let Some(oeq_class) = &mut res.oeq_class { + // println!("oeq_class before: {:?}", oeq_class); + oeq_class.add_new_ordering(&[sort_expr]); + // println!("oeq_class after: {:?}", oeq_class); + // oeq_class.others.insert(vec![sort_expr]); + } else { + let head = vec![sort_expr]; + res.oeq_class = Some(OrderingEquivalentClass::new(head, vec![])); + } + } + } + + res + } + + pub fn with_reorder( + mut self, + sort_expr: Vec, + ) -> OrderingEquivalenceProperties { + // TODO: In some cases, existing ordering equivalences may still be valid add this analysis + // Equivalences and constants are still valid after reorder + self.oeq_class = Some(OrderingEquivalentClass::new(sort_expr, vec![])); + self + } + + pub fn set_satisfy( + &self, + exprs: &[Arc], + ) -> Option> { + let exprs_normalized = self.normalize_exprs(exprs); + // println!("exprs: {:?}", exprs); + // println!("exprs_normalized: {:?}", exprs_normalized); + // println!("self.eq_classes: {:?}", self.eq_classes); + // println!("self.oeq_class: {:?}", self.oeq_class); + let mut best = vec![]; + if let Some(oeq_class) = &self.oeq_class { + for ordering in oeq_class.iter() { + let ordering = self.normalize_sort_exprs(ordering); + let ordering_exprs = ordering + .iter() + .map(|sort_expr| sort_expr.expr.clone()) + .collect::>(); + // let ordering_exprs = self.normalize_exprs(&ordering); + // println!("exprs_normalized: {:?}, normalized_ordering_exprs:{:?}", exprs_normalized, ordering_exprs); + let mut ordered_indices = + get_indices_of_matching_exprs(&exprs_normalized, &ordering_exprs); + // println!("ordered_indices: {:?}", ordered_indices); + ordered_indices.sort(); + // Find out how many expressions of the existing ordering define ordering + // for expressions in the GROUP BY clause. For example, if the input is + // ordered by a, b, c, d and we group by b, a, d; the result below would be. + // 2, meaning 2 elements (a, b) among the GROUP BY columns define ordering. + let first_n = longest_consecutive_prefix(ordered_indices); + if first_n > best.len() { + let ordered_exprs = ordering_exprs[0..first_n].to_vec(); + // Find indices for the GROUP BY expressions such that when we iterate with + // these indices, we would match existing ordering. For the example above, + // this would produce 1, 0; meaning 1st and 0th entries (a, b) among the + // GROUP BY expressions b, a, d match input ordering. + let indices = + get_indices_of_matching_exprs(&ordered_exprs, &exprs_normalized); + // println!("indices:{:?}, ordered_exprs: {:?}, exprs_normalized:{:?}", indices, ordered_exprs, exprs_normalized); + best = indices + .iter() + .enumerate() + .map(|(order_idx, &match_idx)| { + (match_idx, ordering[order_idx].options) + }) + .collect(); + } + } + } + if best.is_empty() { + None + } else { + Some(best) + } + } + + pub fn with_empty_ordering_equivalence(mut self) -> OrderingEquivalenceProperties { + self.oeq_class = None; + self + } } /// EquivalentClass is a set of [`Column`]s or [`PhysicalSortExpr`]s that are known @@ -495,21 +915,56 @@ impl OrderingEquivalentClass { pub fn prefix_ordering_equivalent_class_with_existing_ordering( &self, existing_ordering: &[PhysicalSortExpr], - eq_properties: &EquivalenceProperties, ) -> OrderingEquivalentClass { - let existing_ordering = eq_properties.normalize_sort_exprs(existing_ordering); - let normalized_head = eq_properties.normalize_sort_exprs(self.head()); + // let existing_ordering = eq_properties.normalize_sort_exprs(existing_ordering); + // let normalized_head = eq_properties.normalize_sort_exprs(self.head()); + let normalized_head = self.head(); let updated_head = merge_vectors(&existing_ordering, &normalized_head); let updated_others = self .others() .iter() .map(|ordering| { - let normalized_ordering = eq_properties.normalize_sort_exprs(ordering); + // let normalized_ordering = eq_properties.normalize_sort_exprs(ordering); + let normalized_ordering = ordering; merge_vectors(&existing_ordering, &normalized_ordering) }) .collect(); OrderingEquivalentClass::new(updated_head, updated_others) } + + fn get_finer( + lhs: &[PhysicalSortExpr], + rhs: &[PhysicalSortExpr], + ) -> Option> { + if izip!(lhs.iter(), rhs.iter()).all(|(lhs, rhs)| lhs.eq(rhs)) { + if lhs.len() > rhs.len() { + return Some(lhs.to_vec()); + } else { + return Some(rhs.to_vec()); + } + } + None + } + + fn add_new_ordering(&mut self, ordering: &[PhysicalSortExpr]) { + let mut is_redundant = false; + let mut new_res = vec![]; + for existing_ordering in self.iter() { + if let Some(finer) = Self::get_finer(existing_ordering, ordering) { + // existing_ordering = finer; + new_res.push(finer); + is_redundant = true; + } else { + new_res.push(existing_ordering.to_vec()); + } + } + if !is_redundant { + new_res.push(ordering.to_vec()); + } + let head = new_res[0].clone(); + let others = new_res[1..].to_vec(); + *self = OrderingEquivalentClass::new(head, others); + } } /// This is a builder object facilitating incremental construction @@ -572,7 +1027,7 @@ impl OrderingEquivalenceBuilder { } // If there is an existing ordering, add new ordering as an equivalence: if !normalized_out_ordering.is_empty() { - self.ordering_eq_properties.add_equal_conditions(( + self.ordering_eq_properties.add_ordering_equal_conditions(( &normalized_out_ordering, &new_equivalent_ordering, )); @@ -765,19 +1220,29 @@ pub fn ordering_equivalence_properties_helper( eq_orderings: &[LexOrdering], ) -> OrderingEquivalenceProperties { let mut oep = OrderingEquivalenceProperties::new(schema); - let first_ordering = if let Some(first) = eq_orderings.first() { - first - } else { + if eq_orderings.is_empty() { // Return an empty OrderingEquivalenceProperties: return oep; - }; - // First entry among eq_orderings is the head, skip it: - for ordering in eq_orderings.iter().skip(1) { - if !ordering.is_empty() { - oep.add_equal_conditions((first_ordering, ordering)) - } + } else { + let head = eq_orderings[0].clone(); + let others = eq_orderings[1..].to_vec(); + oep.extend(Some(OrderingEquivalentClass::new(head, others))); + return oep; } - oep + // oep.extend(Some(OrderingEquivalentClass::new())) + // let first_ordering = if let Some(first) = eq_orderings.first() { + // first + // } else { + // // Return an empty OrderingEquivalenceProperties: + // return oep; + // }; + // // First entry among eq_orderings is the head, skip it: + // for ordering in eq_orderings.iter().skip(1) { + // if !ordering.is_empty() { + // oep.add_ordering_equal_conditions((first_ordering, ordering)) + // } + // } + // oep } /// This function constructs a duplicate-free vector by filtering out duplicate @@ -916,6 +1381,7 @@ pub fn physical_exprs_contains( physical_exprs: &[Arc], expr: &Arc, ) -> bool { + // println!("physical_exprs:{:?}, expr:{:?}", physical_exprs, expr); physical_exprs .iter() .any(|physical_expr| physical_expr.eq(expr)) @@ -933,6 +1399,18 @@ fn prune_sort_reqs_with_constants( .collect() } +/// Adds the `offset` value to `Column` indices inside `expr`. This function is +/// generally used during the update of the right table schema in join operations. +pub(crate) fn add_offset_to_exprs( + exprs: Vec>, + offset: usize, +) -> Result>> { + exprs + .into_iter() + .map(|item| add_offset_to_expr(item, offset)) + .collect::>>() +} + /// Adds the `offset` value to `Column` indices inside `expr`. This function is /// generally used during the update of the right table schema in join operations. pub(crate) fn add_offset_to_expr( @@ -971,6 +1449,99 @@ pub fn add_offset_to_lex_ordering( .collect() } +/// Calculates the [`SortProperties`] of a given [`ExprOrdering`] node. +/// The node is either a leaf node, or an intermediate node: +/// - If it is a leaf node, the children states are `None`. We directly find +/// the order of the node by looking at the given sort expression and equivalence +/// properties if it is a `Column` leaf, or we mark it as unordered. In the case +/// of a `Literal` leaf, we mark it as singleton so that it can cooperate with +/// some ordered columns at the upper steps. +/// - If it is an intermediate node, the children states matter. Each `PhysicalExpr` +/// and operator has its own rules about how to propagate the children orderings. +/// However, before the children order propagation, it is checked that whether +/// the intermediate node can be directly matched with the sort expression. If there +/// is a match, the sort expression emerges at that node immediately, discarding +/// the order coming from the children. +pub fn update_ordering( + mut node: ExprOrdering, + ordering_equal_properties: &OrderingEquivalenceProperties, +) -> Result> { + if let Some(children_sort_options) = &node.children_states { + // We have an intermediate (non-leaf) node, account for its children: + node.state = Some(node.expr.get_ordering(children_sort_options)); + Ok(Transformed::Yes(node)) + } else if let Some(column) = node.expr.as_any().downcast_ref::() { + // We have a Column, which is one of the two possible leaf node types: + // TODO: Make this a method of ordering equivalence + if let Some(oeq_class) = ordering_equal_properties.oeq_class() { + for ordering in oeq_class.iter() { + let global_ordering = &ordering[0]; + if node.expr.eq(&global_ordering.expr) { + node.state = Some(SortProperties::Ordered(global_ordering.options)); + return Ok(Transformed::Yes(node)); + } + } + } + node.state = None; + Ok(Transformed::No(node)) + } else { + // We have a Literal, which is the other possible leaf node type: + node.state = Some(node.expr.get_ordering(&[])); + Ok(Transformed::Yes(node)) + } + // Ok(Transformed::Yes(node)) +} + +/// Combine equivalence properties of the given join inputs. +pub fn combine_join_equivalence_properties2( + join_type: &JoinType, + left_eq_classes: &[Vec>], + right_eq_classes: &[Vec>], + left_columns_len: usize, + on: &[(Column, Column)], + out_properties: &mut OrderingEquivalenceProperties, +) -> Result<()> { + let mut res = vec![]; + match join_type { + JoinType::Inner | JoinType::Left | JoinType::Full | JoinType::Right => { + res.extend(left_eq_classes.to_vec()); + let updated_eq_classes = right_eq_classes + .iter() + .map(|eq_class| add_offset_to_exprs(eq_class.to_vec(), left_columns_len)) + .collect::>>()?; + + res.extend(updated_eq_classes); + } + JoinType::LeftSemi | JoinType::LeftAnti => { + res.extend(left_eq_classes.to_vec()); + } + JoinType::RightSemi | JoinType::RightAnti => { + res.extend(right_eq_classes.to_vec()); + } + } + out_properties.eq_classes = res; + if *join_type == JoinType::Inner { + on.iter().for_each(|(lhs, rhs)| { + let new_lhs = Arc::new(lhs.clone()) as _; + let new_rhs = + Arc::new(Column::new(rhs.name(), rhs.index() + left_columns_len)) as _; + // (new_lhs, new_rhs) + // println!("new_lhs: {:?}, new_rhs: {:?}", new_lhs, new_rhs); + out_properties.add_equal_conditions((&new_lhs, &new_rhs)); + }); + } + Ok(()) + + // if join_type == JoinType::Inner { + // on.iter().for_each(|(column1, column2)| { + // let new_column2 = + // Column::new(column2.name(), left_columns_len + column2.index()); + // new_properties.add_equal_conditions((column1, &new_column2)) + // }) + // } + // new_properties +} + #[cfg(test)] mod tests { use super::*; diff --git a/datafusion/physical-expr/src/expressions/get_indexed_field.rs b/datafusion/physical-expr/src/expressions/get_indexed_field.rs index 3a7ce568b3fe..b7a19f4638db 100644 --- a/datafusion/physical-expr/src/expressions/get_indexed_field.rs +++ b/datafusion/physical-expr/src/expressions/get_indexed_field.rs @@ -61,9 +61,31 @@ impl std::fmt::Display for GetFieldAccessExpr { impl PartialEq for GetFieldAccessExpr { fn eq(&self, other: &dyn Any) -> bool { + // println!("self:{:?}", self); + // println!("other:{:?}", other); down_cast_any_ref(other) .downcast_ref::() - .map(|x| self.eq(x)) + .map(|x| match (self, x) { + ( + GetFieldAccessExpr::NamedStructField { name: lhs }, + GetFieldAccessExpr::NamedStructField { name: rhs }, + ) => lhs.eq(rhs), + ( + GetFieldAccessExpr::ListIndex { key: lhs }, + GetFieldAccessExpr::ListIndex { key: rhs }, + ) => lhs.eq(rhs), + ( + GetFieldAccessExpr::ListRange { + start: start_lhs, + stop: stop_lhs, + }, + GetFieldAccessExpr::ListRange { + start: start_rhs, + stop: stop_rhs, + }, + ) => start_lhs.eq(start_rhs) && stop_lhs.eq(stop_rhs), + (_, _) => false, + }) .unwrap_or(false) } } @@ -233,6 +255,7 @@ impl PhysicalExpr for GetIndexedFieldExpr { impl PartialEq for GetIndexedFieldExpr { fn eq(&self, other: &dyn Any) -> bool { + // println!("self:{:?}, other:{:?}", self, other); down_cast_any_ref(other) .downcast_ref::() .map(|x| self.arg.eq(&x.arg) && self.field.eq(&x.field)) diff --git a/datafusion/physical-expr/src/partitioning.rs b/datafusion/physical-expr/src/partitioning.rs index 773eac40dc8a..07dcf7c7bd5c 100644 --- a/datafusion/physical-expr/src/partitioning.rs +++ b/datafusion/physical-expr/src/partitioning.rs @@ -20,7 +20,10 @@ use std::fmt; use std::sync::Arc; -use crate::{expr_list_eq_strict_order, EquivalenceProperties, PhysicalExpr}; +use crate::{ + expr_list_eq_strict_order, EquivalenceProperties, OrderingEquivalenceProperties, + PhysicalExpr, +}; /// Partitioning schemes supported by operators. #[derive(Debug, Clone)] @@ -63,7 +66,7 @@ impl Partitioning { /// Returns true when the guarantees made by this [[Partitioning]] are sufficient to /// satisfy the partitioning scheme mandated by the `required` [[Distribution]] - pub fn satisfy EquivalenceProperties>( + pub fn satisfy OrderingEquivalenceProperties>( &self, required: Distribution, equal_properties: F, @@ -83,7 +86,7 @@ impl Partitioning { // and normalize both exprs based on the eq_properties if !fast_match { let eq_properties = equal_properties(); - let eq_classes = eq_properties.classes(); + let eq_classes = eq_properties.eq_classes(); if !eq_classes.is_empty() { let normalized_required_exprs = required_exprs .iter() @@ -200,19 +203,19 @@ mod tests { for distribution in distribution_types { let result = ( single_partition.satisfy(distribution.clone(), || { - EquivalenceProperties::new(schema.clone()) + OrderingEquivalenceProperties::new(schema.clone()) }), unspecified_partition.satisfy(distribution.clone(), || { - EquivalenceProperties::new(schema.clone()) + OrderingEquivalenceProperties::new(schema.clone()) }), round_robin_partition.satisfy(distribution.clone(), || { - EquivalenceProperties::new(schema.clone()) + OrderingEquivalenceProperties::new(schema.clone()) }), hash_partition1.satisfy(distribution.clone(), || { - EquivalenceProperties::new(schema.clone()) + OrderingEquivalenceProperties::new(schema.clone()) }), hash_partition2.satisfy(distribution.clone(), || { - EquivalenceProperties::new(schema.clone()) + OrderingEquivalenceProperties::new(schema.clone()) }), ); diff --git a/datafusion/physical-expr/src/sort_properties.rs b/datafusion/physical-expr/src/sort_properties.rs index 001b86e60a86..e9b77f8b65af 100644 --- a/datafusion/physical-expr/src/sort_properties.rs +++ b/datafusion/physical-expr/src/sort_properties.rs @@ -239,7 +239,6 @@ impl TreeNode for ExprOrdering { pub fn update_ordering( mut node: ExprOrdering, sort_expr: &PhysicalSortExpr, - equal_properties: &EquivalenceProperties, ordering_equal_properties: &OrderingEquivalenceProperties, ) -> Result> { // If we can directly match a sort expr with the current node, we can set @@ -260,7 +259,6 @@ pub fn update_ordering( node.state = get_indices_of_matching_sort_exprs_with_order_eq( &[sort_expr.clone()], &[column.clone()], - equal_properties, ordering_equal_properties, ) .map(|(sort_options, _)| { diff --git a/datafusion/physical-expr/src/utils.rs b/datafusion/physical-expr/src/utils.rs index b2a6bb5ca6d2..a2c8da03b2c5 100644 --- a/datafusion/physical-expr/src/utils.rs +++ b/datafusion/physical-expr/src/utils.rs @@ -140,15 +140,11 @@ pub fn normalize_out_expr_with_columns_map( /// Standardized version `vec![d ASC]` is used in subsequent operations. fn normalize_sort_exprs( sort_exprs: &[PhysicalSortExpr], - eq_properties: &EquivalenceProperties, ordering_eq_properties: &OrderingEquivalenceProperties, ) -> Vec { let sort_requirements = PhysicalSortRequirement::from_sort_exprs(sort_exprs.iter()); - let normalized_exprs = normalize_sort_requirements( - &sort_requirements, - eq_properties, - ordering_eq_properties, - ); + let normalized_exprs = + normalize_sort_requirements(&sort_requirements, ordering_eq_properties); PhysicalSortRequirement::to_sort_exprs(normalized_exprs) } @@ -162,52 +158,40 @@ fn normalize_sort_exprs( /// Standardized version `vec![d Some(ASC)]` is used in subsequent operations. fn normalize_sort_requirements( sort_reqs: &[PhysicalSortRequirement], - eq_properties: &EquivalenceProperties, ordering_eq_properties: &OrderingEquivalenceProperties, ) -> Vec { - let normalized_sort_reqs = eq_properties.normalize_sort_requirements(sort_reqs); - ordering_eq_properties.normalize_sort_requirements(&normalized_sort_reqs) + // let normalized_sort_reqs = eq_properties.normalize_sort_requirements(sort_reqs); + // ordering_eq_properties.normalize_sort_requirements(&normalized_sort_reqs) + ordering_eq_properties.normalize_sort_requirements(&sort_reqs) } /// Checks whether given ordering requirements are satisfied by provided [PhysicalSortExpr]s. -pub fn ordering_satisfy< - F: FnOnce() -> EquivalenceProperties, - F2: FnOnce() -> OrderingEquivalenceProperties, ->( +pub fn ordering_satisfy OrderingEquivalenceProperties>( provided: Option<&[PhysicalSortExpr]>, required: Option<&[PhysicalSortExpr]>, - equal_properties: F, ordering_equal_properties: F2, ) -> bool { match (provided, required) { (_, None) => true, (None, Some(_)) => false, - (Some(provided), Some(required)) => ordering_satisfy_concrete( - provided, - required, - equal_properties, - ordering_equal_properties, - ), + (Some(provided), Some(required)) => { + ordering_satisfy_concrete(provided, required, ordering_equal_properties) + } } } /// Checks whether the required [`PhysicalSortExpr`]s are satisfied by the /// provided [`PhysicalSortExpr`]s. -pub fn ordering_satisfy_concrete< - F: FnOnce() -> EquivalenceProperties, - F2: FnOnce() -> OrderingEquivalenceProperties, ->( +pub fn ordering_satisfy_concrete OrderingEquivalenceProperties>( provided: &[PhysicalSortExpr], required: &[PhysicalSortExpr], - equal_properties: F, ordering_equal_properties: F2, ) -> bool { let oeq_properties = ordering_equal_properties(); - let eq_properties = equal_properties(); - let required_normalized = - normalize_sort_exprs(required, &eq_properties, &oeq_properties); - let provided_normalized = - normalize_sort_exprs(provided, &eq_properties, &oeq_properties); + let required_normalized = normalize_sort_exprs(required, &oeq_properties); + let provided_normalized = normalize_sort_exprs(provided, &oeq_properties); + // println!("required_normalized:{:?}", required_normalized); + // println!("provided_normalized:{:?}", provided_normalized); if required_normalized.len() > provided_normalized.len() { return false; } @@ -219,13 +203,9 @@ pub fn ordering_satisfy_concrete< /// Checks whether the given [`PhysicalSortRequirement`]s are satisfied by the /// provided [`PhysicalSortExpr`]s. -pub fn ordering_satisfy_requirement< - F: FnOnce() -> EquivalenceProperties, - F2: FnOnce() -> OrderingEquivalenceProperties, ->( +pub fn ordering_satisfy_requirement OrderingEquivalenceProperties>( provided: Option<&[PhysicalSortExpr]>, required: Option<&[PhysicalSortRequirement]>, - equal_properties: F, ordering_equal_properties: F2, ) -> bool { match (provided, required) { @@ -234,7 +214,6 @@ pub fn ordering_satisfy_requirement< (Some(provided), Some(required)) => ordering_satisfy_requirement_concrete( provided, required, - equal_properties, ordering_equal_properties, ), } @@ -243,20 +222,16 @@ pub fn ordering_satisfy_requirement< /// Checks whether the given [`PhysicalSortRequirement`]s are satisfied by the /// provided [`PhysicalSortExpr`]s. pub fn ordering_satisfy_requirement_concrete< - F: FnOnce() -> EquivalenceProperties, F2: FnOnce() -> OrderingEquivalenceProperties, >( provided: &[PhysicalSortExpr], required: &[PhysicalSortRequirement], - equal_properties: F, ordering_equal_properties: F2, ) -> bool { let oeq_properties = ordering_equal_properties(); - let eq_properties = equal_properties(); - let required_normalized = - normalize_sort_requirements(required, &eq_properties, &oeq_properties); - let provided_normalized = - normalize_sort_exprs(provided, &eq_properties, &oeq_properties); + // let eq_properties = equal_properties(); + let required_normalized = normalize_sort_requirements(required, &oeq_properties); + let provided_normalized = normalize_sort_exprs(provided, &oeq_properties); if required_normalized.len() > provided_normalized.len() { return false; } @@ -268,14 +243,10 @@ pub fn ordering_satisfy_requirement_concrete< /// Checks whether the given [`PhysicalSortRequirement`]s are equal or more /// specific than the provided [`PhysicalSortRequirement`]s. -pub fn requirements_compatible< - F: FnOnce() -> OrderingEquivalenceProperties, - F2: FnOnce() -> EquivalenceProperties, ->( +pub fn requirements_compatible OrderingEquivalenceProperties>( provided: Option<&[PhysicalSortRequirement]>, required: Option<&[PhysicalSortRequirement]>, ordering_equal_properties: F, - equal_properties: F2, ) -> bool { match (provided, required) { (_, None) => true, @@ -284,29 +255,21 @@ pub fn requirements_compatible< provided, required, ordering_equal_properties, - equal_properties, ), } } /// Checks whether the given [`PhysicalSortRequirement`]s are equal or more /// specific than the provided [`PhysicalSortRequirement`]s. -fn requirements_compatible_concrete< - F: FnOnce() -> OrderingEquivalenceProperties, - F2: FnOnce() -> EquivalenceProperties, ->( +fn requirements_compatible_concrete OrderingEquivalenceProperties>( provided: &[PhysicalSortRequirement], required: &[PhysicalSortRequirement], ordering_equal_properties: F, - equal_properties: F2, ) -> bool { let oeq_properties = ordering_equal_properties(); - let eq_properties = equal_properties(); - let required_normalized = - normalize_sort_requirements(required, &eq_properties, &oeq_properties); - let provided_normalized = - normalize_sort_requirements(provided, &eq_properties, &oeq_properties); + let required_normalized = normalize_sort_requirements(required, &oeq_properties); + let provided_normalized = normalize_sort_requirements(provided, &oeq_properties); if required_normalized.len() > provided_normalized.len() { return false; } @@ -360,15 +323,15 @@ pub fn convert_to_expr>( /// This function finds the indices of `targets` within `items`, taking into /// account equivalences according to `equal_properties`. -pub fn get_indices_of_matching_exprs EquivalenceProperties>( +pub fn get_indices_of_matching_exprs( targets: &[Arc], items: &[Arc], - equal_properties: F, ) -> Vec { - let eq_properties = equal_properties(); - let normalized_items = eq_properties.normalize_exprs(items); - let normalized_targets = eq_properties.normalize_exprs(targets); - get_indices_of_exprs_strict(normalized_targets, &normalized_items) + // let eq_properties = equal_properties(); + // let normalized_items = eq_properties.normalize_exprs(items); + // let normalized_targets = eq_properties.normalize_exprs(targets); + // get_indices_of_exprs_strict(normalized_targets, &normalized_items) + get_indices_of_exprs_strict(targets, &items) } /// This function finds the indices of `targets` within `items` using strict @@ -570,21 +533,16 @@ pub fn reverse_order_bys(order_bys: &[PhysicalSortExpr]) -> Vec EquivalenceProperties, - F2: Fn() -> OrderingEquivalenceProperties, ->( +pub fn get_finer_ordering<'a, F2: Fn() -> OrderingEquivalenceProperties>( req1: &'a [PhysicalSortExpr], req2: &'a [PhysicalSortExpr], - eq_properties: F, ordering_eq_properties: F2, ) -> Option<&'a [PhysicalSortExpr]> { - if ordering_satisfy_concrete(req1, req2, &eq_properties, &ordering_eq_properties) { + if ordering_satisfy_concrete(req1, req2, &ordering_eq_properties) { // Finer requirement is `provided`, since it satisfies the other: return Some(req1); } - if ordering_satisfy_concrete(req2, req1, &eq_properties, &ordering_eq_properties) { + if ordering_satisfy_concrete(req2, req1, &ordering_eq_properties) { // Finer requirement is `req`, since it satisfies the other: return Some(req2); } @@ -663,7 +621,6 @@ fn get_lexicographical_match_indices( pub fn get_indices_of_matching_sort_exprs_with_order_eq( provided_sorts: &[PhysicalSortExpr], required_columns: &[Column], - eq_properties: &EquivalenceProperties, order_eq_properties: &OrderingEquivalenceProperties, ) -> Option<(Vec, Vec)> { // Create a vector of `PhysicalSortRequirement`s from the required columns: @@ -677,12 +634,10 @@ pub fn get_indices_of_matching_sort_exprs_with_order_eq( let normalized_required = normalize_sort_requirements( &sort_requirement_on_requirements, - eq_properties, &OrderingEquivalenceProperties::new(order_eq_properties.schema()), ); let normalized_provided = normalize_sort_requirements( &PhysicalSortRequirement::from_sort_exprs(provided_sorts.iter()), - eq_properties, &OrderingEquivalenceProperties::new(order_eq_properties.schema()), ); @@ -756,7 +711,6 @@ pub fn get_indices_of_matching_sort_exprs_with_order_eq( pub fn find_orderings_of_exprs( expr: &[(Arc, String)], input_output_ordering: Option<&[PhysicalSortExpr]>, - input_equal_properties: EquivalenceProperties, input_ordering_equal_properties: OrderingEquivalenceProperties, ) -> Result>> { let mut orderings: Vec> = vec![]; @@ -766,12 +720,7 @@ pub fn find_orderings_of_exprs( for (index, (expression, name)) in expr.iter().enumerate() { let initial_expr = ExprOrdering::new(expression.clone()); let transformed = initial_expr.transform_up(&|expr| { - update_ordering( - expr, - leading_ordering, - &input_equal_properties, - &input_ordering_equal_properties, - ) + update_ordering(expr, leading_ordering, &input_ordering_equal_properties) })?; if let Some(SortProperties::Ordered(sort_options)) = transformed.state { orderings.push(Some(PhysicalSortExpr { @@ -899,7 +848,7 @@ mod tests { eq_properties.add_equal_conditions((col_a, col_c)); let mut ordering_eq_properties = OrderingEquivalenceProperties::new(test_schema.clone()); - ordering_eq_properties.add_equal_conditions(( + ordering_eq_properties.add_ordering_equal_conditions(( &vec![PhysicalSortExpr { expr: Arc::new(col_a.clone()), options: option1, @@ -915,7 +864,7 @@ mod tests { }, ], )); - ordering_eq_properties.add_equal_conditions(( + ordering_eq_properties.add_ordering_equal_conditions(( &vec![PhysicalSortExpr { expr: Arc::new(col_a.clone()), options: option1, @@ -1030,14 +979,8 @@ mod tests { Arc::new(Column::new("c", 2)), Arc::new(Column::new("a", 0)), ]; - assert_eq!( - get_indices_of_matching_exprs(&list1, &list2, equal_properties), - vec![2, 0, 1] - ); - assert_eq!( - get_indices_of_matching_exprs(&list2, &list1, equal_properties), - vec![1, 2, 0] - ); + assert_eq!(get_indices_of_matching_exprs(&list1, &list2), vec![2, 0, 1]); + assert_eq!(get_indices_of_matching_exprs(&list2, &list1), vec![1, 2, 0]); } #[test] @@ -1116,18 +1059,12 @@ mod tests { ]; let finer = Some(&finer[..]); let empty_schema = &Arc::new(Schema::empty()); - assert!(ordering_satisfy( - finer, - crude, - || { EquivalenceProperties::new(empty_schema.clone()) }, - || { OrderingEquivalenceProperties::new(empty_schema.clone()) }, - )); - assert!(!ordering_satisfy( - crude, - finer, - || { EquivalenceProperties::new(empty_schema.clone()) }, - || { OrderingEquivalenceProperties::new(empty_schema.clone()) }, - )); + assert!(ordering_satisfy(finer, crude, || { + OrderingEquivalenceProperties::new(empty_schema.clone()) + },)); + assert!(!ordering_satisfy(crude, finer, || { + OrderingEquivalenceProperties::new(empty_schema.clone()) + },)); Ok(()) } @@ -1250,12 +1187,7 @@ mod tests { let required = Some(&required[..]); assert_eq!( - ordering_satisfy( - provided, - required, - || eq_properties.clone(), - || ordering_eq_properties.clone(), - ), + ordering_satisfy(provided, required, || ordering_eq_properties.clone(),), expected, "{err_msg}" ); @@ -1328,11 +1260,7 @@ mod tests { let expected_normalized = convert_to_requirement(&expected_normalized); assert_eq!( - normalize_sort_requirements( - &req, - &eq_properties, - &ordering_eq_properties, - ), + normalize_sort_requirements(&req, &ordering_eq_properties,), expected_normalized ); } @@ -1471,7 +1399,7 @@ mod tests { // Column a and e are ordering equivalent (e.g global ordering of the table can be described both as a ASC and e ASC.) let mut ordering_eq_properties = OrderingEquivalenceProperties::new(test_schema); - ordering_eq_properties.add_equal_conditions(( + ordering_eq_properties.add_ordering_equal_conditions(( &vec![PhysicalSortExpr { expr: Arc::new(col_a.clone()), options: option1, @@ -1513,7 +1441,6 @@ mod tests { sort_req_d.clone(), sort_req_e.clone(), ], - || eq_properties.clone(), || ordering_eq_properties.clone(), )); @@ -1528,7 +1455,6 @@ mod tests { sort_req_d.clone(), sort_req_e.clone(), ], - || eq_properties.clone(), || ordering_eq_properties.clone(), )); @@ -1537,7 +1463,6 @@ mod tests { &[sort_req_a.clone(), sort_req_b.clone(), sort_req_d.clone()], // After normalization would be a ASC, d ASC, b ASC &[sort_req_c, sort_req_d, sort_req_a, sort_req_b, sort_req_e,], - || eq_properties.clone(), || ordering_eq_properties.clone(), )); @@ -1657,7 +1582,6 @@ mod tests { get_indices_of_matching_sort_exprs_with_order_eq( &provided_sorts, &required_columns, - &equal_properties, &ordering_equal_properties, ), Some((vec![sort_options_not, sort_options], vec![0, 1])) @@ -1677,7 +1601,7 @@ mod tests { let equal_properties = EquivalenceProperties::new(Arc::new(schema.clone())); let mut ordering_equal_properties = OrderingEquivalenceProperties::new(Arc::new(schema)); - ordering_equal_properties.add_equal_conditions(( + ordering_equal_properties.add_ordering_equal_conditions(( &vec![PhysicalSortExpr { expr: Arc::new(Column::new("c", 2)), options: sort_options, @@ -1697,7 +1621,6 @@ mod tests { get_indices_of_matching_sort_exprs_with_order_eq( &provided_sorts, &required_columns, - &equal_properties, &ordering_equal_properties, ), Some((vec![sort_options_not, sort_options], vec![0, 1])) @@ -1731,7 +1654,6 @@ mod tests { get_indices_of_matching_sort_exprs_with_order_eq( &provided_sorts, &required_columns, - &equal_properties, &ordering_equal_properties, ), None @@ -1764,7 +1686,7 @@ mod tests { }]]; let oeq_class = OrderingEquivalentClass::new(head, others); - expected_oeq.add_equal_conditions(( + expected_oeq.add_ordering_equal_conditions(( &vec![PhysicalSortExpr { expr: Arc::new(Column::new("b", 1)), options: sort_options, @@ -1802,7 +1724,6 @@ mod tests { expr: Arc::new(Column::new("b", 1)), options: SortOptions::default(), }]), - EquivalenceProperties::new(Arc::new(schema.clone())), OrderingEquivalenceProperties::new(Arc::new(schema.clone())), )?; @@ -1828,7 +1749,6 @@ mod tests { (Arc::new(Column::new("b", 1)), "b_new".to_string()), ], Some(&[]), - EquivalenceProperties::new(Arc::new(schema.clone())), OrderingEquivalenceProperties::new(Arc::new(schema)), )?; diff --git a/datafusion/physical-expr/src/window/built_in.rs b/datafusion/physical-expr/src/window/built_in.rs index a00d32e201fb..5172412643c7 100644 --- a/datafusion/physical-expr/src/window/built_in.rs +++ b/datafusion/physical-expr/src/window/built_in.rs @@ -75,11 +75,7 @@ impl BuiltInWindowExpr { /// If `self.expr` doesn't have an ordering, ordering equivalence properties /// are not updated. Otherwise, ordering equivalence properties are updated /// by the ordering of `self.expr`. - pub fn add_equal_orderings EquivalenceProperties>( - &self, - builder: &mut OrderingEquivalenceBuilder, - equal_properties: F, - ) { + pub fn add_equal_orderings(&self, builder: &mut OrderingEquivalenceBuilder) { let schema = builder.schema(); if let Some(fn_res_ordering) = self.expr.get_result_ordering(schema) { if self.partition_by.is_empty() { @@ -98,7 +94,6 @@ impl BuiltInWindowExpr { let pb_indices = get_indices_of_matching_exprs( &self.partition_by, &existing_ordering_exprs, - equal_properties, ); // Existing ordering should match exactly with PARTITION BY expressions. // There should be no missing/extra entries in the existing ordering. diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 7c7a593c48fc..2642253e4c62 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -22,8 +22,8 @@ use crate::aggregates::{ }; use crate::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use crate::{ - DisplayFormatType, Distribution, EquivalenceProperties, ExecutionPlan, Partitioning, - SendableRecordBatchStream, Statistics, + displayable, DisplayFormatType, Distribution, EquivalenceProperties, ExecutionPlan, + Partitioning, SendableRecordBatchStream, Statistics, }; use arrow::array::ArrayRef; @@ -42,6 +42,8 @@ use datafusion_physical_expr::{ PhysicalExpr, PhysicalSortExpr, PhysicalSortRequirement, }; +use arrow_schema::SortOptions; +use datafusion_common::tree_node::{Transformed, TreeNode}; use std::any::Any; use std::collections::HashMap; use std::sync::Arc; @@ -289,6 +291,7 @@ pub struct AggregateExec { /// The columns map used to normalize out expressions like Partitioning and PhysicalSortExpr /// The key is the column from the input schema and the values are the columns from the output schema columns_map: HashMap>, + source_to_target_mapping: Vec<(Arc, Arc)>, /// Execution Metrics metrics: ExecutionPlanMetricsSet, /// Stores mode and output ordering information for the `AggregateExec`. @@ -303,17 +306,17 @@ pub struct AggregateExec { fn get_working_mode( input: &Arc, group_by: &PhysicalGroupBy, -) -> Option<(GroupByOrderMode, Vec)> { +) -> Option<(GroupByOrderMode, Vec<(usize, SortOptions)>)> { if !group_by.is_single() { // We do not currently support streaming execution if we have more // than one group (e.g. we have grouping sets). return None; }; - let output_ordering = input.output_ordering().unwrap_or(&[]); + // let output_ordering = input.output_ordering().unwrap_or(&[]); // Since direction of the ordering is not important for GROUP BY columns, // we convert PhysicalSortExpr to PhysicalExpr in the existing ordering. - let ordering_exprs = convert_to_expr(output_ordering); + // let ordering_exprs = convert_to_expr(output_ordering); let groupby_exprs = group_by .expr .iter() @@ -321,34 +324,39 @@ fn get_working_mode( .collect::>(); // Find where each expression of the GROUP BY clause occurs in the existing // ordering (if it occurs): - let mut ordered_indices = - get_indices_of_matching_exprs(&groupby_exprs, &ordering_exprs, || { - input.equivalence_properties() - }); - ordered_indices.sort(); - // Find out how many expressions of the existing ordering define ordering - // for expressions in the GROUP BY clause. For example, if the input is - // ordered by a, b, c, d and we group by b, a, d; the result below would be. - // 2, meaning 2 elements (a, b) among the GROUP BY columns define ordering. - let first_n = longest_consecutive_prefix(ordered_indices); - if first_n == 0 { - // No GROUP by columns are ordered, we can not do streaming execution. - return None; - } - let ordered_exprs = ordering_exprs[0..first_n].to_vec(); - // Find indices for the GROUP BY expressions such that when we iterate with - // these indices, we would match existing ordering. For the example above, - // this would produce 1, 0; meaning 1st and 0th entries (a, b) among the - // GROUP BY expressions b, a, d match input ordering. - let ordered_group_by_indices = - get_indices_of_matching_exprs(&ordered_exprs, &groupby_exprs, || { - input.equivalence_properties() - }); - Some(if first_n == group_by.expr.len() { - (GroupByOrderMode::FullyOrdered, ordered_group_by_indices) + + // let mut ordered_indices = + // get_indices_of_matching_exprs(&groupby_exprs, &ordering_exprs); + // ordered_indices.sort(); + // // Find out how many expressions of the existing ordering define ordering + // // for expressions in the GROUP BY clause. For example, if the input is + // // ordered by a, b, c, d and we group by b, a, d; the result below would be. + // // 2, meaning 2 elements (a, b) among the GROUP BY columns define ordering. + // let first_n = longest_consecutive_prefix(ordered_indices); + // if first_n == 0 { + // // No GROUP by columns are ordered, we can not do streaming execution. + // return None; + // } + // let ordered_exprs = ordering_exprs[0..first_n].to_vec(); + // // Find indices for the GROUP BY expressions such that when we iterate with + // // these indices, we would match existing ordering. For the example above, + // // this would produce 1, 0; meaning 1st and 0th entries (a, b) among the + // // GROUP BY expressions b, a, d match input ordering. + // let ordered_group_by_indices = + // get_indices_of_matching_exprs(&ordered_exprs, &groupby_exprs); + + if let Some(ordered_group_by_info) = input + .ordering_equivalence_properties() + .set_satisfy(&groupby_exprs) + { + Some(if ordered_group_by_info.len() == group_by.expr.len() { + (GroupByOrderMode::FullyOrdered, ordered_group_by_info) + } else { + (GroupByOrderMode::PartiallyOrdered, ordered_group_by_info) + }) } else { - (GroupByOrderMode::PartiallyOrdered, ordered_group_by_indices) - }) + None + } } /// This function gathers the ordering information for the GROUP BY columns. @@ -356,18 +364,18 @@ fn calc_aggregation_ordering( input: &Arc, group_by: &PhysicalGroupBy, ) -> Option { - get_working_mode(input, group_by).map(|(mode, order_indices)| { - let existing_ordering = input.output_ordering().unwrap_or(&[]); + get_working_mode(input, group_by).map(|(mode, order_info)| { + // let existing_ordering = input.output_ordering().unwrap_or(&[]); let out_group_expr = output_group_expr_helper(group_by); // Calculate output ordering information for the operator: - let out_ordering = order_indices + let out_ordering = order_info .iter() - .zip(existing_ordering) - .map(|(idx, input_col)| PhysicalSortExpr { + .map(|(idx, sort_options)| PhysicalSortExpr { expr: out_group_expr[*idx].clone(), - options: input_col.options, + options: *sort_options, }) .collect::>(); + let order_indices = order_info.iter().map(|(idx, _)| *idx).collect(); AggregationOrdering { mode, order_indices, @@ -414,13 +422,9 @@ fn get_init_req( /// This function gets the finest ordering requirement among all the aggregation /// functions. If requirements are conflicting, (i.e. we can not compute the /// aggregations in a single [`AggregateExec`]), the function returns an error. -fn get_finest_requirement< - F: Fn() -> EquivalenceProperties, - F2: Fn() -> OrderingEquivalenceProperties, ->( +fn get_finest_requirement OrderingEquivalenceProperties>( aggr_expr: &mut [Arc], order_by_expr: &mut [Option], - eq_properties: F, ordering_eq_properties: F2, ) -> Result> { let mut finest_req = get_init_req(aggr_expr, order_by_expr); @@ -431,12 +435,9 @@ fn get_finest_requirement< continue; }; if let Some(finest_req) = &mut finest_req { - if let Some(finer) = get_finer_ordering( - finest_req, - fn_req, - &eq_properties, - &ordering_eq_properties, - ) { + if let Some(finer) = + get_finer_ordering(finest_req, fn_req, &ordering_eq_properties) + { *finest_req = finer.to_vec(); continue; } @@ -447,7 +448,6 @@ fn get_finest_requirement< if let Some(finer) = get_finer_ordering( finest_req, &fn_req_reverse, - &eq_properties, &ordering_eq_properties, ) { // We need to update `aggr_expr` with its reverse, since only its @@ -471,6 +471,12 @@ fn get_finest_requirement< Ok(finest_req) } +fn print_plan(plan: &Arc) -> () { + let formatted = displayable(plan.as_ref()).indent(true).to_string(); + let actual: Vec<&str> = formatted.trim().lines().collect(); + println!("{:#?}", actual); +} + /// Calculate the required input ordering for the [`AggregateExec`] by considering /// ordering requirements of order-sensitive aggregation functions. fn calc_required_input_ordering( @@ -514,6 +520,8 @@ fn calc_required_input_ordering( // FullyOrdered or PartiallyOrdered modes: let requirement_prefix = if let Some(existing_ordering) = input.output_ordering() { + // print_plan(&input); + // println!("existing ordering:{:?}", existing_ordering); &existing_ordering[0..order_indices.len()] } else { &[] @@ -549,7 +557,6 @@ fn calc_required_input_ordering( if ordering_satisfy_requirement_concrete( existing_ordering, &required_input_ordering, - || input.equivalence_properties(), || input.ordering_equivalence_properties(), ) { break; @@ -653,12 +660,10 @@ impl AggregateExec { // We only support `Single` mode, where we are sure that output produced is final, and it // is produced in a single step. - let requirement = get_finest_requirement( - &mut aggr_expr, - &mut order_by_expr, - || input.equivalence_properties(), - || input.ordering_equivalence_properties(), - )?; + let requirement = + get_finest_requirement(&mut aggr_expr, &mut order_by_expr, || { + input.ordering_equivalence_properties() + })?; let aggregator_requirement = requirement .as_ref() .map(|exprs| PhysicalSortRequirement::from_sort_exprs(exprs.iter())); @@ -677,15 +682,38 @@ impl AggregateExec { // construct a map from the input columns to the output columns of the Aggregation let mut columns_map: HashMap> = HashMap::new(); - for (expression, name) in group_by.expr.iter() { + let mut source_to_target_mapping = vec![]; + // println!(" group_by.expr: {:?}", group_by.expr); + // println!("input schema: {:?}", input.schema()); + // println!("input_schema: {:?}", _input_schema); + let schema_of_input = input.schema(); + for (expr_idx, (expression, name)) in group_by.expr.iter().enumerate() { if let Some(column) = expression.as_any().downcast_ref::() { let new_col_idx = schema.index_of(name)?; let entry = columns_map.entry(column.clone()).or_insert_with(Vec::new); entry.push(Column::new(name, new_col_idx)); }; - } + let target_expr = + Arc::new(Column::new(name, expr_idx)) as Arc; + let source_expr = expression.clone().transform_down(&|e| match e + .as_any() + .downcast_ref::( + ) { + Some(col) => { + let idx = col.index(); + let matching_input_field = schema_of_input.field(idx); + let matching_input_column = + Column::new(matching_input_field.name(), idx); + Ok(Transformed::Yes(Arc::new(matching_input_column))) + } + None => Ok(Transformed::No(e)), + })?; + source_to_target_mapping.push((source_expr, target_expr)); + } + // println!("source_to_target_mapping: {:?}", source_to_target_mapping); let mut aggregation_ordering = calc_aggregation_ordering(&input, &group_by); + // println!("start aggregation_ordering: {:?}", aggregation_ordering); let required_input_ordering = calc_required_input_ordering( &input, &mut aggr_expr, @@ -695,6 +723,7 @@ impl AggregateExec { &mut aggregation_ordering, &mode, )?; + // println!("end aggregation_ordering: {:?}", aggregation_ordering); Ok(AggregateExec { mode, @@ -706,6 +735,7 @@ impl AggregateExec { schema, input_schema, columns_map, + source_to_target_mapping, metrics: ExecutionPlanMetricsSet::new(), aggregation_ordering, required_input_ordering, @@ -956,14 +986,22 @@ impl ExecutionPlan for AggregateExec { vec![self.required_input_ordering.clone()] } - fn equivalence_properties(&self) -> EquivalenceProperties { - let mut new_properties = EquivalenceProperties::new(self.schema()); - project_equivalence_properties( - self.input.equivalence_properties(), + // fn equivalence_properties(&self) -> EquivalenceProperties { + // let mut new_properties = EquivalenceProperties::new(self.schema()); + // project_equivalence_properties( + // self.input.equivalence_properties(), + // &self.columns_map, + // &mut new_properties, + // ); + // new_properties + // } + + fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { + self.input.ordering_equivalence_properties().project( &self.columns_map, - &mut new_properties, - ); - new_properties + &self.source_to_target_mapping, + self.schema(), + ) } fn children(&self) -> Vec> { @@ -992,6 +1030,24 @@ impl ExecutionPlan for AggregateExec { partition: usize, context: Arc, ) -> Result { + // println!("self.columns_map: {:?}", self.columns_map); + + // if self.mode.is_first_stage() { + // println!("aggregate self.input().ordering_equivalence_properties():\n {:?}", self.input().ordering_equivalence_properties()); + // println!("self.source_to_target_mapping: {:?}", self.source_to_target_mapping); + // println!("aggregate self.ordering_equivalence_properties():\n {:?}", self.ordering_equivalence_properties()); + // } + + // println!("self.aggregation_ordering:{:?}", self.aggregation_ordering); + // println!("aggregate self.required_input_ordering: {:?}", self.required_input_ordering); + + // let groupby_exprs = self.group_by + // .expr + // .iter() + // .map(|(item, _)| item.clone()) + // .collect::>(); + // let res = self.input.ordering_equivalence_properties().set_satisfy(&groupby_exprs); + // println!("res:{:?}", res); self.execute_typed(partition, context) .map(|stream| stream.into()) } @@ -1359,23 +1415,39 @@ mod tests { // (We need to add SortExec to be able to run it). // Some(GroupByOrderMode) represents, we can run algorithm with existing ordering; and algorithm should work in // GroupByOrderMode. + let options = SortOptions::default(); let test_cases = vec![ - (vec!["a"], Some((FullyOrdered, vec![0]))), + (vec!["a"], Some((FullyOrdered, vec![(0, options)]))), (vec!["b"], None), (vec!["c"], None), - (vec!["b", "a"], Some((FullyOrdered, vec![1, 0]))), + ( + vec!["b", "a"], + Some((FullyOrdered, vec![(1, options), (0, options)])), + ), (vec!["c", "b"], None), - (vec!["c", "a"], Some((PartiallyOrdered, vec![1]))), - (vec!["c", "b", "a"], Some((FullyOrdered, vec![2, 1, 0]))), - (vec!["d", "a"], Some((PartiallyOrdered, vec![1]))), + (vec!["c", "a"], Some((PartiallyOrdered, vec![(1, options)]))), + ( + vec!["c", "b", "a"], + Some((FullyOrdered, vec![(2, options), (1, options), (0, options)])), + ), + (vec!["d", "a"], Some((PartiallyOrdered, vec![(1, options)]))), (vec!["d", "b"], None), (vec!["d", "c"], None), - (vec!["d", "b", "a"], Some((PartiallyOrdered, vec![2, 1]))), + ( + vec!["d", "b", "a"], + Some((PartiallyOrdered, vec![(2, options), (1, options)])), + ), (vec!["d", "c", "b"], None), - (vec!["d", "c", "a"], Some((PartiallyOrdered, vec![2]))), + ( + vec!["d", "c", "a"], + Some((PartiallyOrdered, vec![(2, options)])), + ), ( vec!["d", "c", "b", "a"], - Some((PartiallyOrdered, vec![3, 2, 1])), + Some(( + PartiallyOrdered, + vec![(3, options), (2, options), (1, options)], + )), ), ]; for (case_idx, test_case) in test_cases.iter().enumerate() { @@ -2248,9 +2320,12 @@ mod tests { let col_b = Column::new("b", 1); let col_c = Column::new("c", 2); let col_d = Column::new("d", 3); - eq_properties.add_equal_conditions((&col_a, &col_b)); + let col_a_expr = (Arc::new(col_a.clone()) as Arc); + let col_b_expr = (Arc::new(col_b.clone()) as Arc); + // eq_properties.add_equal_conditions((&col_a, &col_b)); let mut ordering_eq_properties = OrderingEquivalenceProperties::new(test_schema); - ordering_eq_properties.add_equal_conditions(( + ordering_eq_properties.add_equal_conditions((&col_a_expr, &col_b_expr)); + ordering_eq_properties.add_ordering_equal_conditions(( &vec![PhysicalSortExpr { expr: Arc::new(col_a.clone()) as _, options: options1, @@ -2299,12 +2374,9 @@ mod tests { vec![], )) as _; let mut aggr_exprs = vec![aggr_expr; order_by_exprs.len()]; - let res = get_finest_requirement( - &mut aggr_exprs, - &mut order_by_exprs, - || eq_properties.clone(), - || ordering_eq_properties.clone(), - )?; + let res = get_finest_requirement(&mut aggr_exprs, &mut order_by_exprs, || { + ordering_eq_properties.clone() + })?; assert_eq!(res, order_by_exprs[4]); Ok(()) } diff --git a/datafusion/physical-plan/src/coalesce_batches.rs b/datafusion/physical-plan/src/coalesce_batches.rs index f46a228064fe..e4457100f081 100644 --- a/datafusion/physical-plan/src/coalesce_batches.rs +++ b/datafusion/physical-plan/src/coalesce_batches.rs @@ -136,9 +136,9 @@ impl ExecutionPlan for CoalesceBatchesExec { vec![false] } - fn equivalence_properties(&self) -> EquivalenceProperties { - self.input.equivalence_properties() - } + // fn equivalence_properties(&self) -> EquivalenceProperties { + // self.input.equivalence_properties() + // } fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { self.input.ordering_equivalence_properties() diff --git a/datafusion/physical-plan/src/coalesce_partitions.rs b/datafusion/physical-plan/src/coalesce_partitions.rs index 8eddf57ae551..badee11203d0 100644 --- a/datafusion/physical-plan/src/coalesce_partitions.rs +++ b/datafusion/physical-plan/src/coalesce_partitions.rs @@ -100,9 +100,9 @@ impl ExecutionPlan for CoalescePartitionsExec { None } - fn equivalence_properties(&self) -> EquivalenceProperties { - self.input.equivalence_properties() - } + // fn equivalence_properties(&self) -> EquivalenceProperties { + // self.input.equivalence_properties() + // } fn with_new_children( self: Arc, diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index 4a8b18914411..d41419e5c558 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -143,15 +143,15 @@ impl ExecutionPlan for FilterExec { vec![true] } - fn equivalence_properties(&self) -> EquivalenceProperties { - // Combine the equal predicates with the input equivalence properties - let mut input_properties = self.input.equivalence_properties(); - let (equal_pairs, _ne_pairs) = collect_columns_from_predicate(&self.predicate); - for new_condition in equal_pairs { - input_properties.add_equal_conditions(new_condition) - } - input_properties - } + // fn equivalence_properties(&self) -> EquivalenceProperties { + // // Combine the equal predicates with the input equivalence properties + // let mut input_properties = self.input.equivalence_properties(); + // let (equal_pairs, _ne_pairs) = collect_columns_from_predicate(&self.predicate); + // for new_condition in equal_pairs { + // input_properties.add_equal_conditions(new_condition) + // } + // input_properties + // } fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { let stats = self.statistics(); diff --git a/datafusion/physical-plan/src/joins/cross_join.rs b/datafusion/physical-plan/src/joins/cross_join.rs index 4ba29524b3e2..f4e8e3f0a476 100644 --- a/datafusion/physical-plan/src/joins/cross_join.rs +++ b/datafusion/physical-plan/src/joins/cross_join.rs @@ -25,6 +25,7 @@ use std::{any::Any, sync::Arc, task::Poll}; use arrow::datatypes::{Fields, Schema, SchemaRef}; use arrow::record_batch::RecordBatch; +use crate::joins::utils::combine_join_ordering_equivalence_properties; use crate::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use crate::DisplayAs; use crate::{ @@ -34,10 +35,11 @@ use crate::{ SendableRecordBatchStream, Statistics, }; use async_trait::async_trait; -use datafusion_common::{plan_err, DataFusionError}; +use datafusion_common::{plan_err, DataFusionError, JoinType}; use datafusion_common::{Result, ScalarValue}; use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; use datafusion_execution::TaskContext; +use datafusion_physical_expr::OrderingEquivalenceProperties; use super::utils::{ adjust_right_output_partitioning, cross_join_equivalence_properties, @@ -215,14 +217,27 @@ impl ExecutionPlan for CrossJoinExec { None } - fn equivalence_properties(&self) -> EquivalenceProperties { - let left_columns_len = self.left.schema().fields.len(); - cross_join_equivalence_properties( - self.left.equivalence_properties(), - self.right.equivalence_properties(), - left_columns_len, + // fn equivalence_properties(&self) -> EquivalenceProperties { + // let left_columns_len = self.left.schema().fields.len(); + // cross_join_equivalence_properties( + // self.left.equivalence_properties(), + // self.right.equivalence_properties(), + // left_columns_len, + // self.schema(), + // ) + // } + + fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { + combine_join_ordering_equivalence_properties( + &JoinType::Full, + self.left(), + self.right(), self.schema(), + &[false, false], + None, + &[], ) + .unwrap() } fn execute( diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index 8e204634f3d9..f7a6126f76dc 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -365,17 +365,17 @@ impl ExecutionPlan for HashJoinExec { Self::maintains_input_order(self.join_type) } - fn equivalence_properties(&self) -> EquivalenceProperties { - let left_columns_len = self.left.schema().fields.len(); - combine_join_equivalence_properties( - self.join_type, - self.left.equivalence_properties(), - self.right.equivalence_properties(), - left_columns_len, - self.on(), - self.schema(), - ) - } + // fn equivalence_properties(&self) -> EquivalenceProperties { + // let left_columns_len = self.left.schema().fields.len(); + // combine_join_equivalence_properties( + // self.join_type, + // self.left.equivalence_properties(), + // self.right.equivalence_properties(), + // left_columns_len, + // self.on(), + // self.schema(), + // ) + // } fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { combine_join_ordering_equivalence_properties( @@ -385,7 +385,7 @@ impl ExecutionPlan for HashJoinExec { self.schema(), &self.maintains_input_order(), Some(Self::probe_side()), - self.equivalence_properties(), + self.on(), ) .unwrap() } @@ -418,6 +418,12 @@ impl ExecutionPlan for HashJoinExec { let on_right = self.on.iter().map(|on| on.1.clone()).collect::>(); let left_partitions = self.left.output_partitioning().partition_count(); let right_partitions = self.right.output_partitioning().partition_count(); + + // for child in self.children(){ + // println!("hash join child ordering_equivalence_properties()\n{:?}", child.ordering_equivalence_properties()); + // } + // println!("hash join self.ordering_equivalence_properties()\n{:?}", self.ordering_equivalence_properties()); + if self.mode == PartitionMode::Partitioned && left_partitions != right_partitions { return internal_err!( diff --git a/datafusion/physical-plan/src/joins/nested_loop_join.rs b/datafusion/physical-plan/src/joins/nested_loop_join.rs index c49c16dba313..5f848a531071 100644 --- a/datafusion/physical-plan/src/joins/nested_loop_join.rs +++ b/datafusion/physical-plan/src/joins/nested_loop_join.rs @@ -191,17 +191,17 @@ impl ExecutionPlan for NestedLoopJoinExec { distribution_from_join_type(&self.join_type) } - fn equivalence_properties(&self) -> EquivalenceProperties { - let left_columns_len = self.left.schema().fields.len(); - combine_join_equivalence_properties( - self.join_type, - self.left.equivalence_properties(), - self.right.equivalence_properties(), - left_columns_len, - &[], // empty join keys - self.schema(), - ) - } + // fn equivalence_properties(&self) -> EquivalenceProperties { + // let left_columns_len = self.left.schema().fields.len(); + // combine_join_equivalence_properties( + // self.join_type, + // self.left.equivalence_properties(), + // self.right.equivalence_properties(), + // left_columns_len, + // &[], // empty join keys + // self.schema(), + // ) + // } fn children(&self) -> Vec> { vec![self.left.clone(), self.right.clone()] diff --git a/datafusion/physical-plan/src/joins/sort_merge_join.rs b/datafusion/physical-plan/src/joins/sort_merge_join.rs index 4de723ab73ea..4dacb39f1af6 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join.rs @@ -283,17 +283,17 @@ impl ExecutionPlan for SortMergeJoinExec { Self::maintains_input_order(self.join_type) } - fn equivalence_properties(&self) -> EquivalenceProperties { - let left_columns_len = self.left.schema().fields.len(); - combine_join_equivalence_properties( - self.join_type, - self.left.equivalence_properties(), - self.right.equivalence_properties(), - left_columns_len, - self.on(), - self.schema(), - ) - } + // fn equivalence_properties(&self) -> EquivalenceProperties { + // let left_columns_len = self.left.schema().fields.len(); + // combine_join_equivalence_properties( + // self.join_type, + // self.left.equivalence_properties(), + // self.right.equivalence_properties(), + // left_columns_len, + // self.on(), + // self.schema(), + // ) + // } fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { combine_join_ordering_equivalence_properties( @@ -303,7 +303,7 @@ impl ExecutionPlan for SortMergeJoinExec { self.schema(), &self.maintains_input_order(), Some(Self::probe_side(&self.join_type)), - self.equivalence_properties(), + self.on(), ) .unwrap() } diff --git a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs index a2fd127112e3..70ba374fd0de 100644 --- a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs +++ b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs @@ -442,17 +442,17 @@ impl ExecutionPlan for SymmetricHashJoinExec { None } - fn equivalence_properties(&self) -> EquivalenceProperties { - let left_columns_len = self.left.schema().fields.len(); - combine_join_equivalence_properties( - self.join_type, - self.left.equivalence_properties(), - self.right.equivalence_properties(), - left_columns_len, - self.on(), - self.schema(), - ) - } + // fn equivalence_properties(&self) -> EquivalenceProperties { + // let left_columns_len = self.left.schema().fields.len(); + // combine_join_equivalence_properties( + // self.join_type, + // self.left.equivalence_properties(), + // self.right.equivalence_properties(), + // left_columns_len, + // self.on(), + // self.schema(), + // ) + // } fn children(&self) -> Vec> { vec![self.left.clone(), self.right.clone()] diff --git a/datafusion/physical-plan/src/joins/utils.rs b/datafusion/physical-plan/src/joins/utils.rs index 67f60e57d7d1..655b927f82de 100644 --- a/datafusion/physical-plan/src/joins/utils.rs +++ b/datafusion/physical-plan/src/joins/utils.rs @@ -50,6 +50,7 @@ use datafusion_physical_expr::{ PhysicalSortExpr, }; +use datafusion_physical_expr::equivalence::combine_join_equivalence_properties2; use datafusion_physical_expr::utils::merge_vectors; use futures::future::{BoxFuture, Shared}; use futures::{ready, FutureExt}; @@ -281,6 +282,57 @@ pub fn combine_join_equivalence_properties( new_properties } +// /// Combine equivalence properties of the given join inputs. +// pub fn combine_join_equivalence_properties2( +// join_type: JoinType, +// left_eq_classes: &[Vec>], +// right_eq_classes: &[Vec>], +// left_columns_len: usize, +// on: &[(Column, Column)], +// ) -> Vec>> { +// let mut res = vec![]; +// match join_type { +// JoinType::Inner | JoinType::Left | JoinType::Full | JoinType::Right => { +// res.extend(left_eq_classes.to_vec()); +// +// let new_right_properties = right_eq_classes +// .iter() +// .map(|prop| { +// let new_head = Column::new( +// prop.head().name(), +// left_columns_len + prop.head().index(), +// ); +// let new_others = prop +// .others() +// .iter() +// .map(|col| { +// Column::new(col.name(), left_columns_len + col.index()) +// }) +// .collect::>(); +// EquivalentClass::new(new_head, new_others) +// }) +// .collect::>(); +// +// new_properties.extend(new_right_properties); +// } +// JoinType::LeftSemi | JoinType::LeftAnti => { +// new_properties.extend(left_properties.classes().to_vec()) +// } +// JoinType::RightSemi | JoinType::RightAnti => { +// new_properties.extend(right_properties.classes().to_vec()) +// } +// } +// +// if join_type == JoinType::Inner { +// on.iter().for_each(|(column1, column2)| { +// let new_column2 = +// Column::new(column2.name(), left_columns_len + column2.index()); +// new_properties.add_equal_conditions((column1, &new_column2)) +// }) +// } +// new_properties +// } + /// Calculate equivalence properties for the given cross join operation. pub fn cross_join_equivalence_properties( left_properties: EquivalenceProperties, @@ -326,7 +378,6 @@ fn get_updated_right_ordering_equivalent_class( join_type: &JoinType, right_oeq_class: &OrderingEquivalentClass, left_columns_len: usize, - join_eq_properties: &EquivalenceProperties, ) -> Result { match join_type { // In these modes, indices of the right schema should be offset by @@ -334,15 +385,18 @@ fn get_updated_right_ordering_equivalent_class( JoinType::Inner | JoinType::Left | JoinType::Full | JoinType::Right => { let right_oeq_class = right_oeq_class.add_offset(left_columns_len)?; return Ok( - right_oeq_class.normalize_with_equivalence_properties(join_eq_properties) + // right_oeq_class.normalize_with_equivalence_properties(join_eq_properties) + right_oeq_class, ); } _ => {} }; - Ok(right_oeq_class.normalize_with_equivalence_properties(join_eq_properties)) + // Ok(right_oeq_class.normalize_with_equivalence_properties(join_eq_properties)) + Ok(right_oeq_class.clone()) } /// Calculate ordering equivalence properties for the given join operation. +/// TODO: Move this to method as join pub fn combine_join_ordering_equivalence_properties( join_type: &JoinType, left: &Arc, @@ -350,12 +404,36 @@ pub fn combine_join_ordering_equivalence_properties( schema: SchemaRef, maintains_input_order: &[bool], probe_side: Option, - join_eq_properties: EquivalenceProperties, + on: &[(Column, Column)], ) -> Result { + // println!("combine join start"); + let n_schema = schema.fields.len(); let mut new_properties = OrderingEquivalenceProperties::new(schema); let left_columns_len = left.schema().fields.len(); let left_oeq_properties = left.ordering_equivalence_properties(); let right_oeq_properties = right.ordering_equivalence_properties(); + // if n_schema > left.schema().fields.len() && n_schema > right.schema().fields.len() { + // on.iter().for_each(|(lhs, rhs)| { + // let new_lhs = Arc::new(lhs.clone()) as _; + // let new_rhs = + // Arc::new(Column::new(rhs.name(), rhs.index() + left_columns_len)) as _; + // // (new_lhs, new_rhs) + // // println!("new_lhs: {:?}, new_rhs: {:?}", new_lhs, new_rhs); + // new_properties.add_equal_conditions((&new_lhs, &new_rhs)); + // }); + // // println!("on: {:?}", updated_on); + // // new_properties.add_equal_conditions() + // } + + combine_join_equivalence_properties2( + join_type, + &left_oeq_properties.eq_classes(), + &right_oeq_properties.eq_classes(), + left_columns_len, + on, + &mut new_properties, + )?; + // println!("new_eq_classes join before: {:?}", new_properties.eq_classes()); // All joins have 2 children assert_eq!(maintains_input_order.len(), 2); let left_maintains = maintains_input_order[0]; @@ -387,7 +465,6 @@ pub fn combine_join_ordering_equivalence_properties( join_type, oeq_class, left_columns_len, - &join_eq_properties, )?; // Right side ordering equivalence properties should be prepended with @@ -401,7 +478,6 @@ pub fn combine_join_ordering_equivalence_properties( let updated_right_oeq_class = updated_right_oeq .prefix_ordering_equivalent_class_with_existing_ordering( left_output_ordering, - &join_eq_properties, ); new_properties.extend(Some(updated_right_oeq_class)); } @@ -414,7 +490,6 @@ pub fn combine_join_ordering_equivalence_properties( join_type, right_oeq_class, left_columns_len, - &join_eq_properties, ) }) .transpose()?; @@ -447,13 +522,14 @@ pub fn combine_join_ordering_equivalence_properties( let updated_left_oeq_class = left_oeq_class .prefix_ordering_equivalent_class_with_existing_ordering( &right_output_ordering, - &join_eq_properties, ); new_properties.extend(Some(updated_left_oeq_class)); } } (false, false) => {} } + // println!("combine join end"); + // println!("new_eq_classes join after: {:?}", new_properties.eq_classes()); Ok(new_properties) } @@ -1879,7 +1955,6 @@ mod tests { &join_type, &right_oeq_class, left_columns_len, - &join_eq_properties, )?; let expected = OrderingEquivalentClass::new( diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs index c500ecde5d2c..4906431e96e3 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -143,10 +143,10 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { .collect() } - /// Get the EquivalenceProperties within the plan - fn equivalence_properties(&self) -> EquivalenceProperties { - EquivalenceProperties::new(self.schema()) - } + // /// Get the EquivalenceProperties within the plan + // fn equivalence_properties(&self) -> EquivalenceProperties { + // EquivalenceProperties::new(self.schema()) + // } /// Get the OrderingEquivalenceProperties within the plan fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { diff --git a/datafusion/physical-plan/src/limit.rs b/datafusion/physical-plan/src/limit.rs index c6d51b7d9c5d..b09a9263b89c 100644 --- a/datafusion/physical-plan/src/limit.rs +++ b/datafusion/physical-plan/src/limit.rs @@ -135,9 +135,9 @@ impl ExecutionPlan for GlobalLimitExec { self.input.output_ordering() } - fn equivalence_properties(&self) -> EquivalenceProperties { - self.input.equivalence_properties() - } + // fn equivalence_properties(&self) -> EquivalenceProperties { + // self.input.equivalence_properties() + // } fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { self.input.ordering_equivalence_properties() @@ -312,9 +312,9 @@ impl ExecutionPlan for LocalLimitExec { vec![true] } - fn equivalence_properties(&self) -> EquivalenceProperties { - self.input.equivalence_properties() - } + // fn equivalence_properties(&self) -> EquivalenceProperties { + // self.input.equivalence_properties() + // } fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { self.input.ordering_equivalence_properties() diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index 4fc48e971ca9..9c2646f919c2 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -44,6 +44,7 @@ use datafusion_physical_expr::{ project_ordering_equivalence_properties, OrderingEquivalenceProperties, }; +use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_physical_expr::utils::find_orderings_of_exprs; use futures::stream::{Stream, StreamExt}; use log::trace; @@ -62,6 +63,7 @@ pub struct ProjectionExec { /// The columns map used to normalize out expressions like Partitioning and PhysicalSortExpr /// The key is the column from the input schema and the values are the columns from the output schema columns_map: HashMap>, + source_to_target_mapping: Vec<(Arc, Arc)>, /// Execution metrics metrics: ExecutionPlanMetricsSet, /// Expressions' normalized orderings (as given by the output ordering API @@ -101,6 +103,28 @@ impl ProjectionExec { // construct a map from the input columns to the output columns of the Projection let mut columns_map: HashMap> = HashMap::new(); + let mut source_to_target_mapping = vec![]; + for (expr_idx, (expression, name)) in expr.iter().enumerate() { + let target_expr = + Arc::new(Column::new(name, expr_idx)) as Arc; + + let source_expr = expression.clone().transform_down(&|e| match e + .as_any() + .downcast_ref::( + ) { + Some(col) => { + let idx = col.index(); + let matching_input_field = input_schema.field(idx); + let matching_input_column = + Column::new(matching_input_field.name(), idx); + Ok(Transformed::Yes(Arc::new(matching_input_column))) + } + None => Ok(Transformed::No(e)), + })?; + + source_to_target_mapping.push((source_expr, target_expr)); + } + // println!("source_to_target_mapping: {:?}", source_to_target_mapping); for (expr_idx, (expression, name)) in expr.iter().enumerate() { if let Some(column) = expression.as_any().downcast_ref::() { // For some executors, logical and physical plan schema fields @@ -143,12 +167,27 @@ impl ProjectionExec { let orderings = find_orderings_of_exprs( &expr, input.output_ordering(), - input.equivalence_properties(), input.ordering_equivalence_properties(), )?; - let output_ordering = - validate_output_ordering(output_ordering, &orderings, &expr); + // println!("source_to_target_mapping:{:?}", source_to_target_mapping); + // println!("input.ordering_equivalence_properties():{:?}", input.ordering_equivalence_properties()); + // println!("ordering_equivalence_properties():{:?}", input + // .ordering_equivalence_properties() + // .project(&columns_map, &source_to_target_mapping, schema.clone())); + + // let output_ordering = + // validate_output_ordering(output_ordering, &orderings, &expr); + + let output_ordering = if let Some(oeq_class) = input + .ordering_equivalence_properties() + .project(&columns_map, &source_to_target_mapping, schema.clone()) + .oeq_class() + { + Some(oeq_class.head().to_vec()) + } else { + None + }; Ok(Self { expr, @@ -156,6 +195,7 @@ impl ProjectionExec { input, output_ordering, columns_map, + source_to_target_mapping, metrics: ExecutionPlanMetricsSet::new(), orderings, }) @@ -241,6 +281,13 @@ impl ExecutionPlan for ProjectionExec { } fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { + // let oeq = self.ordering_equivalence_properties(); + // if let Some(oeq_class) = oeq.oeq_class(){ + // Some(oeq_class.head()) + // } else{ + // None + // } + // oeq.oeq_class().map(|oeq_class| &(oeq_class.head().to_vec())) self.output_ordering.as_deref() } @@ -249,49 +296,57 @@ impl ExecutionPlan for ProjectionExec { vec![true] } - fn equivalence_properties(&self) -> EquivalenceProperties { - let mut new_properties = EquivalenceProperties::new(self.schema()); - project_equivalence_properties( - self.input.equivalence_properties(), - &self.columns_map, - &mut new_properties, - ); - new_properties - } + // fn equivalence_properties(&self) -> EquivalenceProperties { + // let mut new_properties = EquivalenceProperties::new(self.schema()); + // project_equivalence_properties( + // self.input.equivalence_properties(), + // &self.columns_map, + // &mut new_properties, + // ); + // new_properties + // } + + // fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { + // let mut new_properties = OrderingEquivalenceProperties::new(self.schema()); + // if self.output_ordering.is_none() { + // // If there is no output ordering, return an "empty" equivalence set: + // return new_properties; + // } + // + // let input_oeq = self.input().ordering_equivalence_properties(); + // + // project_ordering_equivalence_properties( + // input_oeq, + // &self.columns_map, + // &mut new_properties, + // ); + // + // if let Some(leading_ordering) = self + // .output_ordering + // .as_ref() + // .map(|output_ordering| &output_ordering[0]) + // { + // for order in self.orderings.iter().flatten() { + // if !order.eq(leading_ordering) + // && !new_properties.satisfies_leading_ordering(order) + // { + // new_properties.add_ordering_equal_conditions(( + // &vec![leading_ordering.clone()], + // &vec![order.clone()], + // )); + // } + // } + // } + // + // new_properties + // } fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { - let mut new_properties = OrderingEquivalenceProperties::new(self.schema()); - if self.output_ordering.is_none() { - // If there is no output ordering, return an "empty" equivalence set: - return new_properties; - } - - let input_oeq = self.input().ordering_equivalence_properties(); - - project_ordering_equivalence_properties( - input_oeq, + self.input.ordering_equivalence_properties().project( &self.columns_map, - &mut new_properties, - ); - - if let Some(leading_ordering) = self - .output_ordering - .as_ref() - .map(|output_ordering| &output_ordering[0]) - { - for order in self.orderings.iter().flatten() { - if !order.eq(leading_ordering) - && !new_properties.satisfies_leading_ordering(order) - { - new_properties.add_equal_conditions(( - &vec![leading_ordering.clone()], - &vec![order.clone()], - )); - } - } - } - - new_properties + &self.source_to_target_mapping, + self.schema(), + ) } fn with_new_children( @@ -320,6 +375,16 @@ impl ExecutionPlan for ProjectionExec { context: Arc, ) -> Result { trace!("Start ProjectionExec::execute for partition {} of context session_id {} and task_id {:?}", partition, context.session_id(), context.task_id()); + // println!( + // "proj self.input.ordering_equivalence_properties(): {:?}", + // self.input.ordering_equivalence_properties() + // ); + // println!( + // "proj self.ordering_equivalence_properties(): {:?}", + // self.ordering_equivalence_properties() + // ); + // println!("self.source_to_target_mapping: {:?}", self.source_to_target_mapping); + // println!("proj self.output ordering: {:?}", self.output_ordering()); Ok(Box::pin(ProjectionStream { schema: self.schema.clone(), expr: self.expr.iter().map(|x| x.0.clone()).collect(), diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 14b54dc0614d..94f9d216d5d7 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -457,12 +457,18 @@ impl ExecutionPlan for RepartitionExec { } } - fn equivalence_properties(&self) -> EquivalenceProperties { - self.input.equivalence_properties() - } + // fn equivalence_properties(&self) -> EquivalenceProperties { + // self.input.equivalence_properties() + // } fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { - self.input.ordering_equivalence_properties() + if !self.maintains_input_order()[0] { + self.input + .ordering_equivalence_properties() + .with_empty_ordering_equivalence() + } else { + self.input.ordering_equivalence_properties() + } } fn execute( diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index 7d260d42d9cd..f01dc777de17 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -42,7 +42,7 @@ use datafusion_execution::memory_pool::{ }; use datafusion_execution::runtime_env::RuntimeEnv; use datafusion_execution::TaskContext; -use datafusion_physical_expr::EquivalenceProperties; +use datafusion_physical_expr::{EquivalenceProperties, OrderingEquivalenceProperties}; use futures::{StreamExt, TryStreamExt}; use log::{debug, error, trace}; use std::any::Any; @@ -825,8 +825,18 @@ impl ExecutionPlan for SortExec { Some(&self.expr) } - fn equivalence_properties(&self) -> EquivalenceProperties { - self.input.equivalence_properties() + // fn equivalence_properties(&self) -> EquivalenceProperties { + // self.input.equivalence_properties() + // } + + fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { + let mut input_oeq = self.input.ordering_equivalence_properties(); + // println!("sort input: {:?}", input_oeq); + // let mut res = OrderingEquivalenceProperties::new(self.schema()); + + let res = input_oeq.with_reorder(self.expr.to_vec()); + // println!("sort output: {:?}", res); + res } fn with_new_children( diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index 5b485e0b68e4..fb23ed54323f 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -172,9 +172,9 @@ impl ExecutionPlan for SortPreservingMergeExec { vec![true] } - fn equivalence_properties(&self) -> EquivalenceProperties { - self.input.equivalence_properties() - } + // fn equivalence_properties(&self) -> EquivalenceProperties { + // self.input.equivalence_properties() + // } fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { self.input.ordering_equivalence_properties() diff --git a/datafusion/physical-plan/src/unnest.rs b/datafusion/physical-plan/src/unnest.rs index 410ea97887e0..0e493cfeae95 100644 --- a/datafusion/physical-plan/src/unnest.rs +++ b/datafusion/physical-plan/src/unnest.rs @@ -136,9 +136,9 @@ impl ExecutionPlan for UnnestExec { None } - fn equivalence_properties(&self) -> EquivalenceProperties { - self.input.equivalence_properties() - } + // fn equivalence_properties(&self) -> EquivalenceProperties { + // self.input.equivalence_properties() + // } fn execute( &self, diff --git a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs index c6211c8061ff..a0e91d2227a0 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -283,9 +283,9 @@ impl ExecutionPlan for BoundedWindowAggExec { } } - fn equivalence_properties(&self) -> EquivalenceProperties { - self.input().equivalence_properties() - } + // fn equivalence_properties(&self) -> EquivalenceProperties { + // self.input().equivalence_properties() + // } /// Get the OrderingEquivalenceProperties within the plan fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { @@ -315,6 +315,8 @@ impl ExecutionPlan for BoundedWindowAggExec { context: Arc, ) -> Result { let input = self.input.execute(partition, context)?; + // println!("window self.input.ordering_equivalence_properties(): {:?}", self.input.ordering_equivalence_properties()); + // println!("window self.ordering_equivalence_properties(): {:?}", self.ordering_equivalence_properties()); let search_mode = self.get_search_algo()?; let stream = Box::pin(BoundedWindowAggStream::new( self.schema.clone(), diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index 2a2f8d6d211b..7e3a24dc46d5 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -317,17 +317,11 @@ pub(crate) fn get_ordered_partition_by_indices( ) -> Vec { let input_ordering = input.output_ordering().unwrap_or(&[]); let input_ordering_exprs = convert_to_expr(input_ordering); - let equal_properties = || input.equivalence_properties(); - let input_places = get_indices_of_matching_exprs( - &input_ordering_exprs, - partition_by_exprs, - equal_properties, - ); - let mut partition_places = get_indices_of_matching_exprs( - partition_by_exprs, - &input_ordering_exprs, - equal_properties, - ); + // let equal_properties = || input.equivalence_properties(); + let input_places = + get_indices_of_matching_exprs(&input_ordering_exprs, partition_by_exprs); + let mut partition_places = + get_indices_of_matching_exprs(partition_by_exprs, &input_ordering_exprs); partition_places.sort(); let first_n = longest_consecutive_prefix(partition_places); input_places[0..first_n].to_vec() @@ -341,7 +335,7 @@ pub(crate) fn window_ordering_equivalence( // We need to update the schema, so we can not directly use // `input.ordering_equivalence_properties()`. let mut builder = OrderingEquivalenceBuilder::new(schema.clone()) - .with_equivalences(input.equivalence_properties()) + // .with_equivalences(input.equivalence_properties()) .with_existing_ordering(input.output_ordering().map(|elem| elem.to_vec())) .extend(input.ordering_equivalence_properties()); @@ -349,8 +343,7 @@ pub(crate) fn window_ordering_equivalence( if let Some(builtin_window_expr) = expr.as_any().downcast_ref::() { - builtin_window_expr - .add_equal_orderings(&mut builder, || input.equivalence_properties()); + builtin_window_expr.add_equal_orderings(&mut builder); } } builder.build() diff --git a/datafusion/physical-plan/src/windows/window_agg_exec.rs b/datafusion/physical-plan/src/windows/window_agg_exec.rs index da43f127f07b..7217720c82f1 100644 --- a/datafusion/physical-plan/src/windows/window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/window_agg_exec.rs @@ -212,9 +212,9 @@ impl ExecutionPlan for WindowAggExec { } } - fn equivalence_properties(&self) -> EquivalenceProperties { - self.input().equivalence_properties() - } + // fn equivalence_properties(&self) -> EquivalenceProperties { + // self.input().equivalence_properties() + // } /// Get the OrderingEquivalenceProperties within the plan fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { From 56b92e196a53077fb28d595601c98c2d9bdf79df Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 22 Sep 2023 17:59:39 +0300 Subject: [PATCH 002/122] Remove EquivalenceProperties struct --- .../enforce_distribution.rs | 1 - .../src/physical_optimizer/enforce_sorting.rs | 1 - datafusion/core/src/physical_planner.rs | 4 +- datafusion/physical-expr/src/equivalence.rs | 490 +++++++++--------- datafusion/physical-expr/src/lib.rs | 4 +- datafusion/physical-expr/src/partitioning.rs | 2 +- .../physical-expr/src/sort_properties.rs | 2 +- datafusion/physical-expr/src/utils.rs | 2 +- .../physical-expr/src/window/built_in.rs | 2 +- .../physical-plan/src/aggregates/mod.rs | 5 +- .../physical-plan/src/coalesce_batches.rs | 2 +- .../physical-plan/src/coalesce_partitions.rs | 2 +- datafusion/physical-plan/src/filter.rs | 2 +- .../physical-plan/src/joins/cross_join.rs | 4 +- .../physical-plan/src/joins/hash_join.rs | 4 +- .../src/joins/nested_loop_join.rs | 4 +- .../src/joins/sort_merge_join.rs | 4 +- .../src/joins/symmetric_hash_join.rs | 4 +- datafusion/physical-plan/src/joins/utils.rs | 158 +++--- datafusion/physical-plan/src/lib.rs | 2 +- datafusion/physical-plan/src/limit.rs | 2 +- datafusion/physical-plan/src/projection.rs | 5 +- .../physical-plan/src/repartition/mod.rs | 2 +- datafusion/physical-plan/src/sorts/sort.rs | 2 +- .../src/sorts/sort_preserving_merge.rs | 2 +- datafusion/physical-plan/src/unnest.rs | 2 +- .../src/windows/bounded_window_agg_exec.rs | 2 +- .../src/windows/window_agg_exec.rs | 2 +- 28 files changed, 357 insertions(+), 361 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index f5e9ca183a8b..3e07b34a09f9 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -48,7 +48,6 @@ use crate::physical_plan::{with_new_children_if_necessary, Distribution, Executi use datafusion_common::tree_node::{Transformed, TreeNode, VisitRecursion}; use datafusion_expr::logical_plan::JoinType; -use datafusion_physical_expr::equivalence::EquivalenceProperties; use datafusion_physical_expr::expressions::{Column, NoOp}; use datafusion_physical_expr::utils::{ map_columns_before_projection, ordering_satisfy_requirement_concrete, diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index 4764e38a0581..8eb5c4b04e95 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -2786,7 +2786,6 @@ mod tests { } mod tmp_tests { - use crate::assert_batches_eq; use crate::physical_optimizer::utils::get_plan_string; use crate::physical_plan::{collect, displayable, ExecutionPlan}; use crate::prelude::SessionContext; diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 6eea504fc307..ab4c33350579 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -2792,7 +2792,9 @@ digraph { } fn print_plan(plan: &Arc) -> () { - let formatted = crate::physical_plan::displayable(plan.as_ref()).indent(true).to_string(); + let formatted = crate::physical_plan::displayable(plan.as_ref()) + .indent(true) + .to_string(); let actual: Vec<&str> = formatted.trim().lines().collect(); println!("{:#?}", actual); } diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index b21abbf58a73..a81edd9a4128 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -35,166 +35,166 @@ use std::hash::Hash; use std::ops::Range; use std::sync::Arc; -/// Represents a collection of [`EquivalentClass`] (equivalences -/// between columns in relations) -/// -/// This is used to represent: -/// -/// 1. Equality conditions (like `A=B`), when `T` = [`Column`] -#[derive(Debug, Clone)] -pub struct EquivalenceProperties { - classes: Vec>, - schema: SchemaRef, -} - -impl EquivalenceProperties { - pub fn new(schema: SchemaRef) -> Self { - EquivalenceProperties { - classes: vec![], - schema, - } - } - - /// return the set of equivalences - pub fn classes(&self) -> &[EquivalentClass] { - &self.classes - } - - pub fn schema(&self) -> SchemaRef { - self.schema.clone() - } - - /// Add the [`EquivalentClass`] from `iter` to this list - pub fn extend>>(&mut self, iter: I) { - for ec in iter { - self.classes.push(ec) - } - } - - /// Adds new equal conditions into the EquivalenceProperties. New equal - /// conditions usually come from equality predicates in a join/filter. - pub fn add_equal_conditions(&mut self, new_conditions: (&Column, &Column)) { - let mut idx1: Option = None; - let mut idx2: Option = None; - for (idx, class) in self.classes.iter_mut().enumerate() { - let contains_first = class.contains(new_conditions.0); - let contains_second = class.contains(new_conditions.1); - match (contains_first, contains_second) { - (true, false) => { - class.insert(new_conditions.1.clone()); - idx1 = Some(idx); - } - (false, true) => { - class.insert(new_conditions.0.clone()); - idx2 = Some(idx); - } - (true, true) => { - idx1 = Some(idx); - idx2 = Some(idx); - break; - } - (false, false) => {} - } - } - - match (idx1, idx2) { - (Some(idx_1), Some(idx_2)) if idx_1 != idx_2 => { - // need to merge the two existing EquivalentClasses - let second_eq_class = self.classes.get(idx_2).unwrap().clone(); - let first_eq_class = self.classes.get_mut(idx_1).unwrap(); - for prop in second_eq_class.iter() { - if !first_eq_class.contains(prop) { - first_eq_class.insert(prop.clone()); - } - } - self.classes.remove(idx_2); - } - (None, None) => { - // adding new pairs - self.classes.push(EquivalentClass::::new( - new_conditions.0.clone(), - vec![new_conditions.1.clone()], - )); - } - _ => {} - } - } - - /// Normalizes physical expression according to `EquivalentClass`es inside `self.classes`. - /// expression is replaced with `EquivalentClass::head` expression if it is among `EquivalentClass::others`. - pub fn normalize_expr(&self, expr: Arc) -> Arc { - expr.clone() - .transform(&|expr| { - let normalized_form = - expr.as_any().downcast_ref::().and_then(|column| { - for class in &self.classes { - if class.contains(column) { - return Some(Arc::new(class.head().clone()) as _); - } - } - None - }); - Ok(if let Some(normalized_form) = normalized_form { - Transformed::Yes(normalized_form) - } else { - Transformed::No(expr) - }) - }) - .unwrap_or(expr) - } - - /// This function applies the \[`normalize_expr`] - /// function for all expression in `exprs` and returns a vector of - /// normalized physical expressions. - pub fn normalize_exprs( - &self, - exprs: &[Arc], - ) -> Vec> { - exprs - .iter() - .map(|expr| self.normalize_expr(expr.clone())) - .collect::>() - } - - /// This function normalizes `sort_requirement` according to `EquivalenceClasses` in the `self`. - /// If the given sort requirement doesn't belong to equivalence set inside - /// `self`, it returns `sort_requirement` as is. - pub fn normalize_sort_requirement( - &self, - mut sort_requirement: PhysicalSortRequirement, - ) -> PhysicalSortRequirement { - sort_requirement.expr = self.normalize_expr(sort_requirement.expr); - sort_requirement - } - - /// This function applies the \[`normalize_sort_requirement`] - /// function for all sort requirements in `sort_reqs` and returns a vector of - /// normalized sort expressions. - pub fn normalize_sort_requirements( - &self, - sort_reqs: &[PhysicalSortRequirement], - ) -> Vec { - let normalized_sort_reqs = sort_reqs - .iter() - .map(|sort_req| self.normalize_sort_requirement(sort_req.clone())) - .collect::>(); - collapse_vec(normalized_sort_reqs) - } - - /// Similar to the \[`normalize_sort_requirements`] this function normalizes - /// sort expressions in `sort_exprs` and returns a vector of - /// normalized sort expressions. - pub fn normalize_sort_exprs( - &self, - sort_exprs: &[PhysicalSortExpr], - ) -> Vec { - let sort_requirements = - PhysicalSortRequirement::from_sort_exprs(sort_exprs.iter()); - let normalized_sort_requirement = - self.normalize_sort_requirements(&sort_requirements); - PhysicalSortRequirement::to_sort_exprs(normalized_sort_requirement) - } -} +// /// Represents a collection of [`EquivalentClass`] (equivalences +// /// between columns in relations) +// /// +// /// This is used to represent: +// /// +// /// 1. Equality conditions (like `A=B`), when `T` = [`Column`] +// #[derive(Debug, Clone)] +// pub struct EquivalenceProperties { +// classes: Vec>, +// schema: SchemaRef, +// } +// +// impl EquivalenceProperties { +// pub fn new(schema: SchemaRef) -> Self { +// EquivalenceProperties { +// classes: vec![], +// schema, +// } +// } +// +// /// return the set of equivalences +// pub fn classes(&self) -> &[EquivalentClass] { +// &self.classes +// } +// +// pub fn schema(&self) -> SchemaRef { +// self.schema.clone() +// } +// +// /// Add the [`EquivalentClass`] from `iter` to this list +// pub fn extend>>(&mut self, iter: I) { +// for ec in iter { +// self.classes.push(ec) +// } +// } +// +// /// Adds new equal conditions into the EquivalenceProperties. New equal +// /// conditions usually come from equality predicates in a join/filter. +// pub fn add_equal_conditions(&mut self, new_conditions: (&Column, &Column)) { +// let mut idx1: Option = None; +// let mut idx2: Option = None; +// for (idx, class) in self.classes.iter_mut().enumerate() { +// let contains_first = class.contains(new_conditions.0); +// let contains_second = class.contains(new_conditions.1); +// match (contains_first, contains_second) { +// (true, false) => { +// class.insert(new_conditions.1.clone()); +// idx1 = Some(idx); +// } +// (false, true) => { +// class.insert(new_conditions.0.clone()); +// idx2 = Some(idx); +// } +// (true, true) => { +// idx1 = Some(idx); +// idx2 = Some(idx); +// break; +// } +// (false, false) => {} +// } +// } +// +// match (idx1, idx2) { +// (Some(idx_1), Some(idx_2)) if idx_1 != idx_2 => { +// // need to merge the two existing EquivalentClasses +// let second_eq_class = self.classes.get(idx_2).unwrap().clone(); +// let first_eq_class = self.classes.get_mut(idx_1).unwrap(); +// for prop in second_eq_class.iter() { +// if !first_eq_class.contains(prop) { +// first_eq_class.insert(prop.clone()); +// } +// } +// self.classes.remove(idx_2); +// } +// (None, None) => { +// // adding new pairs +// self.classes.push(EquivalentClass::::new( +// new_conditions.0.clone(), +// vec![new_conditions.1.clone()], +// )); +// } +// _ => {} +// } +// } +// +// /// Normalizes physical expression according to `EquivalentClass`es inside `self.classes`. +// /// expression is replaced with `EquivalentClass::head` expression if it is among `EquivalentClass::others`. +// pub fn normalize_expr(&self, expr: Arc) -> Arc { +// expr.clone() +// .transform(&|expr| { +// let normalized_form = +// expr.as_any().downcast_ref::().and_then(|column| { +// for class in &self.classes { +// if class.contains(column) { +// return Some(Arc::new(class.head().clone()) as _); +// } +// } +// None +// }); +// Ok(if let Some(normalized_form) = normalized_form { +// Transformed::Yes(normalized_form) +// } else { +// Transformed::No(expr) +// }) +// }) +// .unwrap_or(expr) +// } +// +// /// This function applies the \[`normalize_expr`] +// /// function for all expression in `exprs` and returns a vector of +// /// normalized physical expressions. +// pub fn normalize_exprs( +// &self, +// exprs: &[Arc], +// ) -> Vec> { +// exprs +// .iter() +// .map(|expr| self.normalize_expr(expr.clone())) +// .collect::>() +// } +// +// /// This function normalizes `sort_requirement` according to `EquivalenceClasses` in the `self`. +// /// If the given sort requirement doesn't belong to equivalence set inside +// /// `self`, it returns `sort_requirement` as is. +// pub fn normalize_sort_requirement( +// &self, +// mut sort_requirement: PhysicalSortRequirement, +// ) -> PhysicalSortRequirement { +// sort_requirement.expr = self.normalize_expr(sort_requirement.expr); +// sort_requirement +// } +// +// /// This function applies the \[`normalize_sort_requirement`] +// /// function for all sort requirements in `sort_reqs` and returns a vector of +// /// normalized sort expressions. +// pub fn normalize_sort_requirements( +// &self, +// sort_reqs: &[PhysicalSortRequirement], +// ) -> Vec { +// let normalized_sort_reqs = sort_reqs +// .iter() +// .map(|sort_req| self.normalize_sort_requirement(sort_req.clone())) +// .collect::>(); +// collapse_vec(normalized_sort_reqs) +// } +// +// /// Similar to the \[`normalize_sort_requirements`] this function normalizes +// /// sort expressions in `sort_exprs` and returns a vector of +// /// normalized sort expressions. +// pub fn normalize_sort_exprs( +// &self, +// sort_exprs: &[PhysicalSortExpr], +// ) -> Vec { +// let sort_requirements = +// PhysicalSortRequirement::from_sort_exprs(sort_exprs.iter()); +// let normalized_sort_requirement = +// self.normalize_sort_requirements(&sort_requirements); +// PhysicalSortRequirement::to_sort_exprs(normalized_sort_requirement) +// } +// } /// `OrderingEquivalenceProperties` keeps track of columns that describe the /// global ordering of the schema. These columns are not necessarily same; e.g. @@ -893,23 +893,23 @@ impl OrderingEquivalentClass { Ok(OrderingEquivalentClass::new(head, others)) } - /// This function normalizes `OrderingEquivalenceProperties` according to `eq_properties`. - /// More explicitly, it makes sure that expressions in `oeq_class` are head entries - /// in `eq_properties`, replacing any non-head entries with head entries if necessary. - pub fn normalize_with_equivalence_properties( - &self, - eq_properties: &EquivalenceProperties, - ) -> OrderingEquivalentClass { - let head = eq_properties.normalize_sort_exprs(self.head()); - - let others = self - .others() - .iter() - .map(|other| eq_properties.normalize_sort_exprs(other)) - .collect(); - - EquivalentClass::new(head, others) - } + // /// This function normalizes `OrderingEquivalenceProperties` according to `eq_properties`. + // /// More explicitly, it makes sure that expressions in `oeq_class` are head entries + // /// in `eq_properties`, replacing any non-head entries with head entries if necessary. + // pub fn normalize_with_equivalence_properties( + // &self, + // eq_properties: &EquivalenceProperties, + // ) -> OrderingEquivalentClass { + // let head = eq_properties.normalize_sort_exprs(self.head()); + // + // let others = self + // .others() + // .iter() + // .map(|other| eq_properties.normalize_sort_exprs(other)) + // .collect(); + // + // EquivalentClass::new(head, others) + // } /// Prefix with existing ordering. pub fn prefix_ordering_equivalent_class_with_existing_ordering( @@ -970,7 +970,7 @@ impl OrderingEquivalentClass { /// This is a builder object facilitating incremental construction /// for ordering equivalences. pub struct OrderingEquivalenceBuilder { - eq_properties: EquivalenceProperties, + // eq_properties: EquivalenceProperties, ordering_eq_properties: OrderingEquivalenceProperties, existing_ordering: Vec, schema: SchemaRef, @@ -978,10 +978,10 @@ pub struct OrderingEquivalenceBuilder { impl OrderingEquivalenceBuilder { pub fn new(schema: SchemaRef) -> Self { - let eq_properties = EquivalenceProperties::new(schema.clone()); + // let eq_properties = EquivalenceProperties::new(schema.clone()); let ordering_eq_properties = OrderingEquivalenceProperties::new(schema.clone()); Self { - eq_properties, + // eq_properties, ordering_eq_properties, existing_ordering: vec![], schema, @@ -1007,10 +1007,10 @@ impl OrderingEquivalenceBuilder { self } - pub fn with_equivalences(mut self, new_eq_properties: EquivalenceProperties) -> Self { - self.eq_properties = new_eq_properties; - self - } + // pub fn with_equivalences(mut self, new_eq_properties: EquivalenceProperties) -> Self { + // self.eq_properties = new_eq_properties; + // self + // } pub fn add_equal_conditions( &mut self, @@ -1019,9 +1019,9 @@ impl OrderingEquivalenceBuilder { let mut normalized_out_ordering = vec![]; for item in &self.existing_ordering { // To account for ordering equivalences, first normalize the expression: - let normalized = self.eq_properties.normalize_expr(item.expr.clone()); + // let normalized = self.eq_properties.normalize_expr(item.expr.clone()); normalized_out_ordering.push(PhysicalSortExpr { - expr: normalized, + expr: item.expr.clone(), options: item.options, }); } @@ -1065,65 +1065,65 @@ fn get_alias_column( .find_map(|(column, columns)| column.eq(col).then(|| columns[0].clone())) } -/// This function applies the given projection to the given equivalence -/// properties to compute the resulting (projected) equivalence properties; e.g. -/// 1) Adding an alias, which can introduce additional equivalence properties, -/// as in Projection(a, a as a1, a as a2). -/// 2) Truncate the [`EquivalentClass`]es that are not in the output schema. -pub fn project_equivalence_properties( - input_eq: EquivalenceProperties, - alias_map: &HashMap>, - output_eq: &mut EquivalenceProperties, -) { - // Get schema and fields of projection output - let schema = output_eq.schema(); - let fields = schema.fields(); - - let mut eq_classes = input_eq.classes().to_vec(); - for (column, columns) in alias_map { - let mut find_match = false; - for class in eq_classes.iter_mut() { - // If `self.head` is invalidated in the new schema, update head - // with this change `self.head` is not randomly assigned by one of the entries from `self.others` - if is_column_invalid_in_new_schema(&class.head, fields) { - if let Some(alias_col) = get_alias_column(&class.head, alias_map) { - class.head = alias_col; - } - } - if class.contains(column) { - for col in columns { - class.insert(col.clone()); - } - find_match = true; - break; - } - } - if !find_match { - eq_classes.push(EquivalentClass::new(column.clone(), columns.clone())); - } - } - - // Prune columns that are no longer in the schema from equivalences. - for class in eq_classes.iter_mut() { - let columns_to_remove = class - .iter() - .filter(|column| is_column_invalid_in_new_schema(column, fields)) - .cloned() - .collect::>(); - for column in columns_to_remove { - class.remove(&column); - } - } - - eq_classes.retain(|props| { - props.len() > 1 - && - // A column should not give an equivalence with itself. - !(props.len() == 2 && props.head.eq(props.others().iter().next().unwrap())) - }); - - output_eq.extend(eq_classes); -} +// /// This function applies the given projection to the given equivalence +// /// properties to compute the resulting (projected) equivalence properties; e.g. +// /// 1) Adding an alias, which can introduce additional equivalence properties, +// /// as in Projection(a, a as a1, a as a2). +// /// 2) Truncate the [`EquivalentClass`]es that are not in the output schema. +// pub fn project_equivalence_properties( +// input_eq: EquivalenceProperties, +// alias_map: &HashMap>, +// output_eq: &mut EquivalenceProperties, +// ) { +// // Get schema and fields of projection output +// let schema = output_eq.schema(); +// let fields = schema.fields(); +// +// let mut eq_classes = input_eq.classes().to_vec(); +// for (column, columns) in alias_map { +// let mut find_match = false; +// for class in eq_classes.iter_mut() { +// // If `self.head` is invalidated in the new schema, update head +// // with this change `self.head` is not randomly assigned by one of the entries from `self.others` +// if is_column_invalid_in_new_schema(&class.head, fields) { +// if let Some(alias_col) = get_alias_column(&class.head, alias_map) { +// class.head = alias_col; +// } +// } +// if class.contains(column) { +// for col in columns { +// class.insert(col.clone()); +// } +// find_match = true; +// break; +// } +// } +// if !find_match { +// eq_classes.push(EquivalentClass::new(column.clone(), columns.clone())); +// } +// } +// +// // Prune columns that are no longer in the schema from equivalences. +// for class in eq_classes.iter_mut() { +// let columns_to_remove = class +// .iter() +// .filter(|column| is_column_invalid_in_new_schema(column, fields)) +// .cloned() +// .collect::>(); +// for column in columns_to_remove { +// class.remove(&column); +// } +// } +// +// eq_classes.retain(|props| { +// props.len() > 1 +// && +// // A column should not give an equivalence with itself. +// !(props.len() == 2 && props.head.eq(props.others().iter().next().unwrap())) +// }); +// +// output_eq.extend(eq_classes); +// } /// This function applies the given projection to the given ordering /// equivalence properties to compute the resulting (projected) ordering diff --git a/datafusion/physical-expr/src/lib.rs b/datafusion/physical-expr/src/lib.rs index e83dee2e6c80..42a91bfb684c 100644 --- a/datafusion/physical-expr/src/lib.rs +++ b/datafusion/physical-expr/src/lib.rs @@ -56,8 +56,8 @@ pub use aggregate::AggregateExpr; pub use analysis::{analyze, AnalysisContext, ExprBoundaries}; pub use equivalence::{ add_offset_to_lex_ordering, ordering_equivalence_properties_helper, - project_equivalence_properties, project_ordering_equivalence_properties, - EquivalenceProperties, EquivalentClass, OrderingEquivalenceProperties, + project_ordering_equivalence_properties, + EquivalentClass, OrderingEquivalenceProperties, OrderingEquivalentClass, }; diff --git a/datafusion/physical-expr/src/partitioning.rs b/datafusion/physical-expr/src/partitioning.rs index 07dcf7c7bd5c..8b7f9748e4ac 100644 --- a/datafusion/physical-expr/src/partitioning.rs +++ b/datafusion/physical-expr/src/partitioning.rs @@ -21,7 +21,7 @@ use std::fmt; use std::sync::Arc; use crate::{ - expr_list_eq_strict_order, EquivalenceProperties, OrderingEquivalenceProperties, + expr_list_eq_strict_order, OrderingEquivalenceProperties, PhysicalExpr, }; diff --git a/datafusion/physical-expr/src/sort_properties.rs b/datafusion/physical-expr/src/sort_properties.rs index e9b77f8b65af..802d8d57d28c 100644 --- a/datafusion/physical-expr/src/sort_properties.rs +++ b/datafusion/physical-expr/src/sort_properties.rs @@ -20,7 +20,7 @@ use std::{ops::Neg, sync::Arc}; use crate::expressions::Column; use crate::utils::get_indices_of_matching_sort_exprs_with_order_eq; use crate::{ - EquivalenceProperties, OrderingEquivalenceProperties, PhysicalExpr, PhysicalSortExpr, + OrderingEquivalenceProperties, PhysicalExpr, PhysicalSortExpr, }; use arrow_schema::SortOptions; diff --git a/datafusion/physical-expr/src/utils.rs b/datafusion/physical-expr/src/utils.rs index a2c8da03b2c5..174f3e87041d 100644 --- a/datafusion/physical-expr/src/utils.rs +++ b/datafusion/physical-expr/src/utils.rs @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -use crate::equivalence::{EquivalenceProperties, OrderingEquivalenceProperties}; +use crate::equivalence::{ OrderingEquivalenceProperties}; use crate::expressions::{BinaryExpr, Column, UnKnownColumn}; use crate::sort_properties::{ExprOrdering, SortProperties}; use crate::update_ordering; diff --git a/datafusion/physical-expr/src/window/built_in.rs b/datafusion/physical-expr/src/window/built_in.rs index 5172412643c7..9be13d1c20a9 100644 --- a/datafusion/physical-expr/src/window/built_in.rs +++ b/datafusion/physical-expr/src/window/built_in.rs @@ -28,7 +28,7 @@ use crate::expressions::PhysicalSortExpr; use crate::utils::{convert_to_expr, get_indices_of_matching_exprs}; use crate::window::window_expr::{get_orderby_values, WindowFn}; use crate::window::{PartitionBatches, PartitionWindowAggStates, WindowState}; -use crate::{reverse_order_bys, EquivalenceProperties, PhysicalExpr}; +use crate::{reverse_order_bys, PhysicalExpr}; use arrow::array::{new_empty_array, ArrayRef}; use arrow::compute::SortOptions; use arrow::datatypes::Field; diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 2642253e4c62..64d21cc28ca7 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -22,22 +22,19 @@ use crate::aggregates::{ }; use crate::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use crate::{ - displayable, DisplayFormatType, Distribution, EquivalenceProperties, ExecutionPlan, + displayable, DisplayFormatType, Distribution, ExecutionPlan, Partitioning, SendableRecordBatchStream, Statistics, }; use arrow::array::ArrayRef; use arrow::datatypes::{Field, Schema, SchemaRef}; use arrow::record_batch::RecordBatch; -use datafusion_common::utils::longest_consecutive_prefix; use datafusion_common::{not_impl_err, plan_err, DataFusionError, Result}; use datafusion_execution::TaskContext; use datafusion_expr::Accumulator; use datafusion_physical_expr::{ - equivalence::project_equivalence_properties, expressions::Column, normalize_out_expr_with_columns_map, physical_exprs_contains, reverse_order_bys, - utils::{convert_to_expr, get_indices_of_matching_exprs}, AggregateExpr, LexOrdering, LexOrderingReq, OrderingEquivalenceProperties, PhysicalExpr, PhysicalSortExpr, PhysicalSortRequirement, }; diff --git a/datafusion/physical-plan/src/coalesce_batches.rs b/datafusion/physical-plan/src/coalesce_batches.rs index e4457100f081..0fc51a90b0ae 100644 --- a/datafusion/physical-plan/src/coalesce_batches.rs +++ b/datafusion/physical-plan/src/coalesce_batches.rs @@ -24,7 +24,7 @@ use std::sync::Arc; use std::task::{Context, Poll}; use crate::{ - DisplayFormatType, EquivalenceProperties, ExecutionPlan, Partitioning, + DisplayFormatType, ExecutionPlan, Partitioning, RecordBatchStream, SendableRecordBatchStream, }; diff --git a/datafusion/physical-plan/src/coalesce_partitions.rs b/datafusion/physical-plan/src/coalesce_partitions.rs index badee11203d0..cd7e9a8e7c3d 100644 --- a/datafusion/physical-plan/src/coalesce_partitions.rs +++ b/datafusion/physical-plan/src/coalesce_partitions.rs @@ -26,7 +26,7 @@ use super::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; use super::stream::{ObservedStream, RecordBatchReceiverStream}; use super::{DisplayAs, SendableRecordBatchStream, Statistics}; -use crate::{DisplayFormatType, EquivalenceProperties, ExecutionPlan, Partitioning}; +use crate::{DisplayFormatType, ExecutionPlan, Partitioning}; use arrow::datatypes::SchemaRef; use datafusion_common::{internal_err, DataFusionError, Result}; diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index d41419e5c558..fdaff3110b35 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -30,7 +30,7 @@ use super::{ use crate::{ metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}, - Column, DisplayFormatType, EquivalenceProperties, ExecutionPlan, Partitioning, + Column, DisplayFormatType, ExecutionPlan, Partitioning, }; use arrow::compute::filter_record_batch; diff --git a/datafusion/physical-plan/src/joins/cross_join.rs b/datafusion/physical-plan/src/joins/cross_join.rs index f4e8e3f0a476..4ce4515ce909 100644 --- a/datafusion/physical-plan/src/joins/cross_join.rs +++ b/datafusion/physical-plan/src/joins/cross_join.rs @@ -30,7 +30,7 @@ use crate::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use crate::DisplayAs; use crate::{ coalesce_batches::concat_batches, coalesce_partitions::CoalescePartitionsExec, - ColumnStatistics, DisplayFormatType, Distribution, EquivalenceProperties, + ColumnStatistics, DisplayFormatType, Distribution, ExecutionPlan, Partitioning, PhysicalSortExpr, RecordBatchStream, SendableRecordBatchStream, Statistics, }; @@ -42,7 +42,7 @@ use datafusion_execution::TaskContext; use datafusion_physical_expr::OrderingEquivalenceProperties; use super::utils::{ - adjust_right_output_partitioning, cross_join_equivalence_properties, + adjust_right_output_partitioning, BuildProbeJoinMetrics, OnceAsync, OnceFut, }; diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index f7a6126f76dc..32f6637b03f6 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -39,12 +39,12 @@ use crate::{ joins::hash_join_utils::{JoinHashMap, JoinHashMapType}, joins::utils::{ adjust_right_output_partitioning, build_join_schema, check_join_is_valid, - combine_join_equivalence_properties, estimate_join_statistics, + estimate_join_statistics, partitioned_join_output_partitioning, BuildProbeJoinMetrics, ColumnIndex, JoinFilter, JoinOn, }, metrics::{ExecutionPlanMetricsSet, MetricsSet}, - DisplayFormatType, Distribution, EquivalenceProperties, ExecutionPlan, Partitioning, + DisplayFormatType, Distribution, ExecutionPlan, Partitioning, PhysicalExpr, RecordBatchStream, SendableRecordBatchStream, Statistics, }; diff --git a/datafusion/physical-plan/src/joins/nested_loop_join.rs b/datafusion/physical-plan/src/joins/nested_loop_join.rs index 5f848a531071..ffd85d6bd4c2 100644 --- a/datafusion/physical-plan/src/joins/nested_loop_join.rs +++ b/datafusion/physical-plan/src/joins/nested_loop_join.rs @@ -21,7 +21,7 @@ use crate::joins::utils::{ append_right_indices, apply_join_filter_to_indices, build_batch_from_indices, - build_join_schema, check_join_is_valid, combine_join_equivalence_properties, + build_join_schema, check_join_is_valid, estimate_join_statistics, get_anti_indices, get_anti_u64_indices, get_final_indices_from_bit_map, get_semi_indices, get_semi_u64_indices, partitioned_join_output_partitioning, BuildProbeJoinMetrics, ColumnIndex, JoinFilter, @@ -41,7 +41,7 @@ use arrow::util::bit_util; use datafusion_common::{exec_err, DataFusionError, Statistics}; use datafusion_execution::memory_pool::MemoryReservation; use datafusion_expr::JoinType; -use datafusion_physical_expr::{EquivalenceProperties, PhysicalSortExpr}; +use datafusion_physical_expr::{PhysicalSortExpr}; use futures::{ready, Stream, StreamExt, TryStreamExt}; use std::any::Any; use std::fmt::Formatter; diff --git a/datafusion/physical-plan/src/joins/sort_merge_join.rs b/datafusion/physical-plan/src/joins/sort_merge_join.rs index 4dacb39f1af6..e083a07db958 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join.rs @@ -34,12 +34,12 @@ use crate::expressions::Column; use crate::expressions::PhysicalSortExpr; use crate::joins::utils::{ build_join_schema, calculate_join_output_ordering, check_join_is_valid, - combine_join_equivalence_properties, combine_join_ordering_equivalence_properties, + combine_join_ordering_equivalence_properties, estimate_join_statistics, partitioned_join_output_partitioning, JoinOn, JoinSide, }; use crate::metrics::{ExecutionPlanMetricsSet, MetricBuilder, MetricsSet}; use crate::{ - metrics, DisplayAs, DisplayFormatType, Distribution, EquivalenceProperties, + metrics, DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, Partitioning, PhysicalExpr, RecordBatchStream, SendableRecordBatchStream, Statistics, }; diff --git a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs index 70ba374fd0de..1c3380cb4479 100644 --- a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs +++ b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs @@ -49,12 +49,12 @@ use crate::{ hash_join_utils::SortedFilterExpr, utils::{ build_batch_from_indices, build_join_schema, check_join_is_valid, - combine_join_equivalence_properties, partitioned_join_output_partitioning, + partitioned_join_output_partitioning, ColumnIndex, JoinFilter, JoinOn, JoinSide, }, }, metrics::{self, ExecutionPlanMetricsSet, MetricBuilder, MetricsSet}, - DisplayFormatType, Distribution, EquivalenceProperties, ExecutionPlan, Partitioning, + DisplayFormatType, Distribution, ExecutionPlan, Partitioning, RecordBatchStream, SendableRecordBatchStream, Statistics, }; diff --git a/datafusion/physical-plan/src/joins/utils.rs b/datafusion/physical-plan/src/joins/utils.rs index 655b927f82de..471e2558db8e 100644 --- a/datafusion/physical-plan/src/joins/utils.rs +++ b/datafusion/physical-plan/src/joins/utils.rs @@ -28,7 +28,7 @@ use std::usize; use crate::metrics::{self, ExecutionPlanMetricsSet, MetricBuilder}; use crate::SchemaRef; use crate::{ - ColumnStatistics, EquivalenceProperties, ExecutionPlan, Partitioning, Statistics, + ColumnStatistics, ExecutionPlan, Partitioning, Statistics, }; use arrow::array::{ @@ -45,7 +45,7 @@ use datafusion_common::{ }; use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::{ - add_offset_to_lex_ordering, EquivalentClass, LexOrdering, LexOrderingRef, + add_offset_to_lex_ordering, LexOrdering, LexOrderingRef, OrderingEquivalenceProperties, OrderingEquivalentClass, PhysicalExpr, PhysicalSortExpr, }; @@ -230,57 +230,57 @@ pub fn calculate_join_output_ordering( Ok((!output_ordering.is_empty()).then_some(output_ordering)) } -/// Combine equivalence properties of the given join inputs. -pub fn combine_join_equivalence_properties( - join_type: JoinType, - left_properties: EquivalenceProperties, - right_properties: EquivalenceProperties, - left_columns_len: usize, - on: &[(Column, Column)], - schema: SchemaRef, -) -> EquivalenceProperties { - let mut new_properties = EquivalenceProperties::new(schema); - match join_type { - JoinType::Inner | JoinType::Left | JoinType::Full | JoinType::Right => { - new_properties.extend(left_properties.classes().to_vec()); - let new_right_properties = right_properties - .classes() - .iter() - .map(|prop| { - let new_head = Column::new( - prop.head().name(), - left_columns_len + prop.head().index(), - ); - let new_others = prop - .others() - .iter() - .map(|col| { - Column::new(col.name(), left_columns_len + col.index()) - }) - .collect::>(); - EquivalentClass::new(new_head, new_others) - }) - .collect::>(); - - new_properties.extend(new_right_properties); - } - JoinType::LeftSemi | JoinType::LeftAnti => { - new_properties.extend(left_properties.classes().to_vec()) - } - JoinType::RightSemi | JoinType::RightAnti => { - new_properties.extend(right_properties.classes().to_vec()) - } - } - - if join_type == JoinType::Inner { - on.iter().for_each(|(column1, column2)| { - let new_column2 = - Column::new(column2.name(), left_columns_len + column2.index()); - new_properties.add_equal_conditions((column1, &new_column2)) - }) - } - new_properties -} +// /// Combine equivalence properties of the given join inputs. +// pub fn combine_join_equivalence_properties( +// join_type: JoinType, +// left_properties: EquivalenceProperties, +// right_properties: EquivalenceProperties, +// left_columns_len: usize, +// on: &[(Column, Column)], +// schema: SchemaRef, +// ) -> EquivalenceProperties { +// let mut new_properties = EquivalenceProperties::new(schema); +// match join_type { +// JoinType::Inner | JoinType::Left | JoinType::Full | JoinType::Right => { +// new_properties.extend(left_properties.classes().to_vec()); +// let new_right_properties = right_properties +// .classes() +// .iter() +// .map(|prop| { +// let new_head = Column::new( +// prop.head().name(), +// left_columns_len + prop.head().index(), +// ); +// let new_others = prop +// .others() +// .iter() +// .map(|col| { +// Column::new(col.name(), left_columns_len + col.index()) +// }) +// .collect::>(); +// EquivalentClass::new(new_head, new_others) +// }) +// .collect::>(); +// +// new_properties.extend(new_right_properties); +// } +// JoinType::LeftSemi | JoinType::LeftAnti => { +// new_properties.extend(left_properties.classes().to_vec()) +// } +// JoinType::RightSemi | JoinType::RightAnti => { +// new_properties.extend(right_properties.classes().to_vec()) +// } +// } +// +// if join_type == JoinType::Inner { +// on.iter().for_each(|(column1, column2)| { +// let new_column2 = +// Column::new(column2.name(), left_columns_len + column2.index()); +// new_properties.add_equal_conditions((column1, &new_column2)) +// }) +// } +// new_properties +// } // /// Combine equivalence properties of the given join inputs. // pub fn combine_join_equivalence_properties2( @@ -333,32 +333,32 @@ pub fn combine_join_equivalence_properties( // new_properties // } -/// Calculate equivalence properties for the given cross join operation. -pub fn cross_join_equivalence_properties( - left_properties: EquivalenceProperties, - right_properties: EquivalenceProperties, - left_columns_len: usize, - schema: SchemaRef, -) -> EquivalenceProperties { - let mut new_properties = EquivalenceProperties::new(schema); - new_properties.extend(left_properties.classes().to_vec()); - let new_right_properties = right_properties - .classes() - .iter() - .map(|prop| { - let new_head = - Column::new(prop.head().name(), left_columns_len + prop.head().index()); - let new_others = prop - .others() - .iter() - .map(|col| Column::new(col.name(), left_columns_len + col.index())) - .collect::>(); - EquivalentClass::new(new_head, new_others) - }) - .collect::>(); - new_properties.extend(new_right_properties); - new_properties -} +// /// Calculate equivalence properties for the given cross join operation. +// pub fn cross_join_equivalence_properties( +// left_properties: EquivalenceProperties, +// right_properties: EquivalenceProperties, +// left_columns_len: usize, +// schema: SchemaRef, +// ) -> EquivalenceProperties { +// let mut new_properties = EquivalenceProperties::new(schema); +// new_properties.extend(left_properties.classes().to_vec()); +// let new_right_properties = right_properties +// .classes() +// .iter() +// .map(|prop| { +// let new_head = +// Column::new(prop.head().name(), left_columns_len + prop.head().index()); +// let new_others = prop +// .others() +// .iter() +// .map(|col| Column::new(col.name(), left_columns_len + col.index())) +// .collect::>(); +// EquivalentClass::new(new_head, new_others) +// }) +// .collect::>(); +// new_properties.extend(new_right_properties); +// new_properties +// } /// Update right table ordering equivalences so that: /// - They point to valid indices at the output of the join schema, and diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs index 4906431e96e3..433d4bb4aa48 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -330,7 +330,7 @@ use datafusion_physical_expr::expressions::Column; pub use datafusion_physical_expr::window::WindowExpr; pub use datafusion_physical_expr::{AggregateExpr, PhysicalExpr}; pub use datafusion_physical_expr::{Distribution, Partitioning}; -use datafusion_physical_expr::{EquivalenceProperties, PhysicalSortRequirement}; +use datafusion_physical_expr::{ PhysicalSortRequirement}; pub mod aggregates; pub mod analyze; diff --git a/datafusion/physical-plan/src/limit.rs b/datafusion/physical-plan/src/limit.rs index b09a9263b89c..87fa3cded580 100644 --- a/datafusion/physical-plan/src/limit.rs +++ b/datafusion/physical-plan/src/limit.rs @@ -23,7 +23,7 @@ use std::sync::Arc; use std::task::{Context, Poll}; use crate::{ - DisplayFormatType, Distribution, EquivalenceProperties, ExecutionPlan, Partitioning, + DisplayFormatType, Distribution, ExecutionPlan, Partitioning, }; use super::expressions::PhysicalSortExpr; diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index 9c2646f919c2..0b61a4ae1b3b 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -30,7 +30,7 @@ use super::expressions::{Column, PhysicalSortExpr}; use super::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; use super::{DisplayAs, RecordBatchStream, SendableRecordBatchStream, Statistics}; use crate::{ - ColumnStatistics, DisplayFormatType, EquivalenceProperties, ExecutionPlan, + ColumnStatistics, DisplayFormatType, ExecutionPlan, Partitioning, PhysicalExpr, }; @@ -40,8 +40,7 @@ use datafusion_common::Result; use datafusion_execution::TaskContext; use datafusion_physical_expr::expressions::{Literal, UnKnownColumn}; use datafusion_physical_expr::{ - normalize_out_expr_with_columns_map, project_equivalence_properties, - project_ordering_equivalence_properties, OrderingEquivalenceProperties, + normalize_out_expr_with_columns_map, OrderingEquivalenceProperties, }; use datafusion_common::tree_node::{Transformed, TreeNode}; diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 94f9d216d5d7..9eb0df71c0b2 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -30,7 +30,7 @@ use crate::metrics::BaselineMetrics; use crate::repartition::distributor_channels::{channels, partition_aware_channels}; use crate::sorts::streaming_merge; use crate::{ - DisplayFormatType, EquivalenceProperties, ExecutionPlan, Partitioning, Statistics, + DisplayFormatType, ExecutionPlan, Partitioning, Statistics, }; use self::distributor_channels::{DistributionReceiver, DistributionSender}; diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index f01dc777de17..665de2327300 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -42,7 +42,7 @@ use datafusion_execution::memory_pool::{ }; use datafusion_execution::runtime_env::RuntimeEnv; use datafusion_execution::TaskContext; -use datafusion_physical_expr::{EquivalenceProperties, OrderingEquivalenceProperties}; +use datafusion_physical_expr::{ OrderingEquivalenceProperties}; use futures::{StreamExt, TryStreamExt}; use log::{debug, error, trace}; use std::any::Any; diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index fb23ed54323f..2c207c828199 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -34,7 +34,7 @@ use arrow::datatypes::SchemaRef; use datafusion_common::{internal_err, DataFusionError, Result}; use datafusion_execution::TaskContext; use datafusion_physical_expr::{ - EquivalenceProperties, OrderingEquivalenceProperties, PhysicalSortRequirement, + OrderingEquivalenceProperties, PhysicalSortRequirement, }; use log::{debug, trace}; diff --git a/datafusion/physical-plan/src/unnest.rs b/datafusion/physical-plan/src/unnest.rs index 0e493cfeae95..8b1aea6302bd 100644 --- a/datafusion/physical-plan/src/unnest.rs +++ b/datafusion/physical-plan/src/unnest.rs @@ -38,7 +38,7 @@ use std::time::Instant; use std::{any::Any, sync::Arc}; use crate::{ - expressions::Column, DisplayFormatType, Distribution, EquivalenceProperties, + expressions::Column, DisplayFormatType, Distribution, ExecutionPlan, Partitioning, PhysicalExpr, PhysicalSortExpr, RecordBatchStream, SendableRecordBatchStream, Statistics, }; diff --git a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs index a0e91d2227a0..e9c5c21e26d0 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -64,7 +64,7 @@ use datafusion_physical_expr::window::{ PartitionBatches, PartitionKey, PartitionWindowAggStates, WindowState, }; use datafusion_physical_expr::{ - EquivalenceProperties, OrderingEquivalenceProperties, PhysicalExpr, + OrderingEquivalenceProperties, PhysicalExpr, PhysicalSortRequirement, }; diff --git a/datafusion/physical-plan/src/windows/window_agg_exec.rs b/datafusion/physical-plan/src/windows/window_agg_exec.rs index 7217720c82f1..713380ae9808 100644 --- a/datafusion/physical-plan/src/windows/window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/window_agg_exec.rs @@ -24,7 +24,7 @@ use crate::windows::{ calc_requirements, get_ordered_partition_by_indices, window_ordering_equivalence, }; use crate::{ - ColumnStatistics, DisplayAs, DisplayFormatType, Distribution, EquivalenceProperties, + ColumnStatistics, DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, Partitioning, PhysicalExpr, RecordBatchStream, SendableRecordBatchStream, Statistics, WindowExpr, }; From 620dccea08c7d9bf57b48caec837ee16472a6df7 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 22 Sep 2023 18:01:42 +0300 Subject: [PATCH 003/122] Minor changes --- datafusion/physical-expr/src/utils.rs | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/datafusion/physical-expr/src/utils.rs b/datafusion/physical-expr/src/utils.rs index 174f3e87041d..0f78209b8ed6 100644 --- a/datafusion/physical-expr/src/utils.rs +++ b/datafusion/physical-expr/src/utils.rs @@ -844,10 +844,13 @@ mod tests { nulls_first: true, }; let test_schema = create_test_schema()?; - let mut eq_properties = EquivalenceProperties::new(test_schema.clone()); - eq_properties.add_equal_conditions((col_a, col_c)); + let col_a_expr = Arc::new(col_a.clone()) as _; + let col_c_expr = Arc::new(col_c.clone()) as _; + // let mut eq_properties = EquivalenceProperties::new(test_schema.clone()); + // eq_properties.add_equal_conditions((col_a, col_c)); let mut ordering_eq_properties = OrderingEquivalenceProperties::new(test_schema.clone()); + ordering_eq_properties.add_equal_conditions((&col_a_expr, &col_c_expr)); ordering_eq_properties.add_ordering_equal_conditions(( &vec![PhysicalSortExpr { expr: Arc::new(col_a.clone()), From 944029bb71cd84d436ab846e969d5fc725ee3312 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Mon, 25 Sep 2023 09:39:05 +0300 Subject: [PATCH 004/122] all tests pass --- datafusion/physical-expr/src/equivalence.rs | 159 ++++++++++++------ datafusion/physical-expr/src/lib.rs | 5 +- datafusion/physical-expr/src/partitioning.rs | 5 +- .../physical-expr/src/sort_properties.rs | 4 +- datafusion/physical-expr/src/utils.rs | 157 +++++++++-------- .../physical-plan/src/aggregates/mod.rs | 17 +- .../physical-plan/src/coalesce_batches.rs | 4 +- datafusion/physical-plan/src/filter.rs | 2 +- .../physical-plan/src/joins/cross_join.rs | 8 +- .../physical-plan/src/joins/hash_join.rs | 9 +- .../src/joins/nested_loop_join.rs | 11 +- .../src/joins/sort_merge_join.rs | 9 +- .../src/joins/symmetric_hash_join.rs | 8 +- datafusion/physical-plan/src/joins/utils.rs | 33 ++-- datafusion/physical-plan/src/lib.rs | 2 +- datafusion/physical-plan/src/limit.rs | 4 +- datafusion/physical-plan/src/projection.rs | 5 +- .../physical-plan/src/repartition/mod.rs | 4 +- datafusion/physical-plan/src/sorts/sort.rs | 2 +- .../src/sorts/sort_preserving_merge.rs | 4 +- datafusion/physical-plan/src/unnest.rs | 6 +- .../src/windows/bounded_window_agg_exec.rs | 3 +- .../src/windows/window_agg_exec.rs | 6 +- 23 files changed, 271 insertions(+), 196 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index a81edd9a4128..4c4a29307937 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -29,7 +29,7 @@ use crate::sort_properties::{ExprOrdering, SortProperties}; use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::utils::longest_consecutive_prefix; use datafusion_common::{JoinType, Result}; -use itertools::izip; +use itertools::{enumerate, izip}; use std::collections::{HashMap, HashSet}; use std::hash::Hash; use std::ops::Range; @@ -241,6 +241,7 @@ impl OrderingEquivalenceProperties { } else { self.oeq_class = Some(other); } + self.normalize_state(); } } @@ -273,18 +274,52 @@ impl OrderingEquivalenceProperties { new_conditions: (&Arc, &Arc), ) { let (first, second) = new_conditions; - let mut added_to_existing_equalities = false; - self.eq_classes.iter_mut().for_each(|eq_class| { - if physical_exprs_contains(eq_class, first) - && !physical_exprs_contains(eq_class, second) - { - eq_class.push(second.clone()); - added_to_existing_equalities = true; + let mut first_group = None; + let mut second_group = None; + for (group_idx, eq_class) in self.eq_classes.iter().enumerate() { + if physical_exprs_contains(eq_class, first) { + first_group = Some(group_idx); + } + if physical_exprs_contains(eq_class, second) { + second_group = Some(group_idx); + } + } + match (first_group, second_group) { + (Some(first_group_idx), Some(second_group_idx)) => { + // We should bridge these groups + if first_group_idx != second_group_idx { + let other_class = self.eq_classes[second_group_idx].clone(); + self.eq_classes[first_group_idx].extend(other_class); + self.eq_classes.remove(second_group_idx); + } + } + (Some(group_idx), None) => { + self.eq_classes[group_idx].push(second.clone()); + } + (None, Some(group_idx)) => { + self.eq_classes[group_idx].push(first.clone()); + } + (None, None) => { + self.eq_classes.push(vec![first.clone(), second.clone()]); } - }); - if !added_to_existing_equalities && !first.eq(second) { - self.eq_classes.push(vec![first.clone(), second.clone()]); } + self.normalize_state(); + } + + /// Normalizes state according to equivalent classes + fn normalize_state(&mut self) { + let mut new_oeq_class = self.oeq_class.clone(); + if let Some(oeq_class) = &mut new_oeq_class { + // println!("oeq_class: {:?}", oeq_class); + oeq_class.head = self.normalize_sort_exprs(&oeq_class.head); + oeq_class.others = oeq_class + .others + .iter() + .map(|item| self.normalize_sort_exprs(item)) + .collect(); + // println!("oeq_class: {:?}", oeq_class); + } + self.oeq_class = new_oeq_class; } /// Add physical expression that have constant value to the `self.constants` @@ -1573,39 +1608,51 @@ mod tests { Field::new("y", DataType::Int64, true), ])); - let mut eq_properties = EquivalenceProperties::new(schema); - let new_condition = (&Column::new("a", 0), &Column::new("b", 1)); - eq_properties.add_equal_conditions(new_condition); - assert_eq!(eq_properties.classes().len(), 1); + let mut eq_properties = OrderingEquivalenceProperties::new(schema); + let col_a_expr = Arc::new(Column::new("a", 0)) as Arc; + let col_b_expr = Arc::new(Column::new("b", 1)) as Arc; + let col_c_expr = Arc::new(Column::new("c", 2)) as Arc; + let col_x_expr = Arc::new(Column::new("x", 3)) as Arc; + let col_y_expr = Arc::new(Column::new("y", 4)) as Arc; - let new_condition = (&Column::new("b", 1), &Column::new("a", 0)); + let new_condition = (&col_a_expr, &col_b_expr); eq_properties.add_equal_conditions(new_condition); - assert_eq!(eq_properties.classes().len(), 1); - assert_eq!(eq_properties.classes()[0].len(), 2); - assert!(eq_properties.classes()[0].contains(&Column::new("a", 0))); - assert!(eq_properties.classes()[0].contains(&Column::new("b", 1))); + assert_eq!(eq_properties.eq_classes().len(), 1); - let new_condition = (&Column::new("b", 1), &Column::new("c", 2)); + let new_condition = (&col_b_expr, &col_a_expr); eq_properties.add_equal_conditions(new_condition); - assert_eq!(eq_properties.classes().len(), 1); - assert_eq!(eq_properties.classes()[0].len(), 3); - assert!(eq_properties.classes()[0].contains(&Column::new("a", 0))); - assert!(eq_properties.classes()[0].contains(&Column::new("b", 1))); - assert!(eq_properties.classes()[0].contains(&Column::new("c", 2))); + assert_eq!(eq_properties.eq_classes().len(), 1); + let eq_class = &eq_properties.eq_classes()[0]; + assert_eq!(eq_class.len(), 2); + assert!(physical_exprs_contains(eq_class, &col_a_expr)); + assert!(physical_exprs_contains(eq_class, &col_b_expr)); - let new_condition = (&Column::new("x", 3), &Column::new("y", 4)); + let new_condition = (&col_b_expr, &col_c_expr); + eq_properties.add_equal_conditions(new_condition); + assert_eq!(eq_properties.eq_classes().len(), 1); + let eq_class = &eq_properties.eq_classes()[0]; + assert_eq!(eq_class.len(), 3); + assert!(physical_exprs_contains(eq_class, &col_a_expr)); + assert!(physical_exprs_contains(eq_class, &col_b_expr)); + assert!(physical_exprs_contains(eq_class, &col_c_expr)); + + // This is a new set of equality. Hence equivalent class count should be 2. + let new_condition = (&col_x_expr, &col_y_expr); eq_properties.add_equal_conditions(new_condition); - assert_eq!(eq_properties.classes().len(), 2); + assert_eq!(eq_properties.eq_classes().len(), 2); - let new_condition = (&Column::new("x", 3), &Column::new("a", 0)); + // This equality bridges distinct equality sets. + // Hence equivalent class count should decrease from 2 to 1. + let new_condition = (&col_x_expr, &col_a_expr); eq_properties.add_equal_conditions(new_condition); - assert_eq!(eq_properties.classes().len(), 1); - assert_eq!(eq_properties.classes()[0].len(), 5); - assert!(eq_properties.classes()[0].contains(&Column::new("a", 0))); - assert!(eq_properties.classes()[0].contains(&Column::new("b", 1))); - assert!(eq_properties.classes()[0].contains(&Column::new("c", 2))); - assert!(eq_properties.classes()[0].contains(&Column::new("x", 3))); - assert!(eq_properties.classes()[0].contains(&Column::new("y", 4))); + assert_eq!(eq_properties.eq_classes().len(), 1); + let eq_class = &eq_properties.eq_classes()[0]; + assert_eq!(eq_class.len(), 5); + assert!(physical_exprs_contains(eq_class, &col_a_expr)); + assert!(physical_exprs_contains(eq_class, &col_b_expr)); + assert!(physical_exprs_contains(eq_class, &col_c_expr)); + assert!(physical_exprs_contains(eq_class, &col_x_expr)); + assert!(physical_exprs_contains(eq_class, &col_y_expr)); Ok(()) } @@ -1618,10 +1665,14 @@ mod tests { Field::new("c", DataType::Int64, true), ])); - let mut input_properties = EquivalenceProperties::new(input_schema); - let new_condition = (&Column::new("a", 0), &Column::new("b", 1)); + let mut input_properties = OrderingEquivalenceProperties::new(input_schema); + let col_a_expr = Arc::new(Column::new("a", 0)) as Arc; + let col_b_expr = Arc::new(Column::new("b", 1)) as Arc; + let col_c_expr = Arc::new(Column::new("c", 2)) as Arc; + + let new_condition = (&col_a_expr, &col_b_expr); input_properties.add_equal_conditions(new_condition); - let new_condition = (&Column::new("b", 1), &Column::new("c", 2)); + let new_condition = (&col_b_expr, &col_c_expr); input_properties.add_equal_conditions(new_condition); let out_schema = Arc::new(Schema::new(vec![ @@ -1631,6 +1682,16 @@ mod tests { Field::new("a4", DataType::Int64, true), ])); + let col_a1_expr = Arc::new(Column::new("a1", 0)) as Arc; + let col_a2_expr = Arc::new(Column::new("a2", 1)) as Arc; + let col_a3_expr = Arc::new(Column::new("a3", 2)) as Arc; + let col_a4_expr = Arc::new(Column::new("a4", 2)) as Arc; + let source_to_target_mapping = vec![ + (col_a_expr.clone(), col_a1_expr.clone()), + (col_a_expr.clone(), col_a2_expr.clone()), + (col_a_expr.clone(), col_a3_expr.clone()), + (col_a_expr.clone(), col_a4_expr.clone()), + ]; let mut alias_map = HashMap::new(); alias_map.insert( Column::new("a", 0), @@ -1641,15 +1702,17 @@ mod tests { Column::new("a4", 3), ], ); - let mut out_properties = EquivalenceProperties::new(out_schema); - - project_equivalence_properties(input_properties, &alias_map, &mut out_properties); - assert_eq!(out_properties.classes().len(), 1); - assert_eq!(out_properties.classes()[0].len(), 4); - assert!(out_properties.classes()[0].contains(&Column::new("a1", 0))); - assert!(out_properties.classes()[0].contains(&Column::new("a2", 1))); - assert!(out_properties.classes()[0].contains(&Column::new("a3", 2))); - assert!(out_properties.classes()[0].contains(&Column::new("a4", 3))); + // let mut out_properties = OrderingEquivalenceProperties::new(out_schema); + let out_properties = + input_properties.project(&alias_map, &source_to_target_mapping, out_schema); + + assert_eq!(out_properties.eq_classes().len(), 1); + let eq_class = &out_properties.eq_classes()[0]; + assert_eq!(eq_class.len(), 4); + assert!(physical_exprs_contains(eq_class, &col_a1_expr)); + assert!(physical_exprs_contains(eq_class, &col_a2_expr)); + assert!(physical_exprs_contains(eq_class, &col_a3_expr)); + assert!(physical_exprs_contains(eq_class, &col_a4_expr)); Ok(()) } diff --git a/datafusion/physical-expr/src/lib.rs b/datafusion/physical-expr/src/lib.rs index 42a91bfb684c..cb13f65ae69f 100644 --- a/datafusion/physical-expr/src/lib.rs +++ b/datafusion/physical-expr/src/lib.rs @@ -56,9 +56,8 @@ pub use aggregate::AggregateExpr; pub use analysis::{analyze, AnalysisContext, ExprBoundaries}; pub use equivalence::{ add_offset_to_lex_ordering, ordering_equivalence_properties_helper, - project_ordering_equivalence_properties, - EquivalentClass, OrderingEquivalenceProperties, - OrderingEquivalentClass, + project_ordering_equivalence_properties, EquivalentClass, + OrderingEquivalenceProperties, OrderingEquivalentClass, }; pub use partitioning::{Distribution, Partitioning}; diff --git a/datafusion/physical-expr/src/partitioning.rs b/datafusion/physical-expr/src/partitioning.rs index 8b7f9748e4ac..e5db2f9a41f2 100644 --- a/datafusion/physical-expr/src/partitioning.rs +++ b/datafusion/physical-expr/src/partitioning.rs @@ -20,10 +20,7 @@ use std::fmt; use std::sync::Arc; -use crate::{ - expr_list_eq_strict_order, OrderingEquivalenceProperties, - PhysicalExpr, -}; +use crate::{expr_list_eq_strict_order, OrderingEquivalenceProperties, PhysicalExpr}; /// Partitioning schemes supported by operators. #[derive(Debug, Clone)] diff --git a/datafusion/physical-expr/src/sort_properties.rs b/datafusion/physical-expr/src/sort_properties.rs index 802d8d57d28c..d3a24fc8c706 100644 --- a/datafusion/physical-expr/src/sort_properties.rs +++ b/datafusion/physical-expr/src/sort_properties.rs @@ -19,9 +19,7 @@ use std::{ops::Neg, sync::Arc}; use crate::expressions::Column; use crate::utils::get_indices_of_matching_sort_exprs_with_order_eq; -use crate::{ - OrderingEquivalenceProperties, PhysicalExpr, PhysicalSortExpr, -}; +use crate::{OrderingEquivalenceProperties, PhysicalExpr, PhysicalSortExpr}; use arrow_schema::SortOptions; use datafusion_common::tree_node::{Transformed, TreeNode, VisitRecursion}; diff --git a/datafusion/physical-expr/src/utils.rs b/datafusion/physical-expr/src/utils.rs index 0f78209b8ed6..5040e9b53355 100644 --- a/datafusion/physical-expr/src/utils.rs +++ b/datafusion/physical-expr/src/utils.rs @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -use crate::equivalence::{ OrderingEquivalenceProperties}; +use crate::equivalence::OrderingEquivalenceProperties; use crate::expressions::{BinaryExpr, Column, UnKnownColumn}; use crate::sort_properties::{ExprOrdering, SortProperties}; use crate::update_ordering; @@ -820,11 +820,7 @@ mod tests { Ok(schema) } - fn create_test_params() -> Result<( - SchemaRef, - EquivalenceProperties, - OrderingEquivalenceProperties, - )> { + fn create_test_params() -> Result<(SchemaRef, OrderingEquivalenceProperties)> { // Assume schema satisfies ordering a ASC NULLS LAST // and d ASC NULLS LAST, b ASC NULLS LAST and e DESC NULLS FIRST, f ASC NULLS LAST, g ASC NULLS LAST // Assume that column a and c are aliases. @@ -846,8 +842,6 @@ mod tests { let test_schema = create_test_schema()?; let col_a_expr = Arc::new(col_a.clone()) as _; let col_c_expr = Arc::new(col_c.clone()) as _; - // let mut eq_properties = EquivalenceProperties::new(test_schema.clone()); - // eq_properties.add_equal_conditions((col_a, col_c)); let mut ordering_eq_properties = OrderingEquivalenceProperties::new(test_schema.clone()); ordering_eq_properties.add_equal_conditions((&col_a_expr, &col_c_expr)); @@ -887,7 +881,7 @@ mod tests { }, ], )); - Ok((test_schema, eq_properties, ordering_eq_properties)) + Ok((test_schema, ordering_eq_properties)) } #[test] @@ -970,7 +964,6 @@ mod tests { #[test] fn test_get_indices_of_matching_exprs() { let empty_schema = &Arc::new(Schema::empty()); - let equal_properties = || EquivalenceProperties::new(empty_schema.clone()); let list1: Vec> = vec![ Arc::new(Column::new("a", 0)), Arc::new(Column::new("b", 1)), @@ -1100,7 +1093,7 @@ mod tests { }, ]; let provided = Some(&provided[..]); - let (_test_schema, eq_properties, ordering_eq_properties) = create_test_params()?; + let (_test_schema, ordering_eq_properties) = create_test_params()?; // First element in the tuple stores vector of requirement, second element is the expected return value for ordering_satisfy function let requirements = vec![ // `a ASC NULLS LAST`, expects `ordering_satisfy` to be `true`, since existing ordering `a ASC NULLS LAST, b ASC NULLS LAST` satisfies it @@ -1257,7 +1250,7 @@ mod tests { ), ]; - let (_test_schema, eq_properties, ordering_eq_properties) = create_test_params()?; + let (_test_schema, ordering_eq_properties) = create_test_params()?; for (reqs, expected_normalized) in requirements.into_iter() { let req = convert_to_requirement(&reqs); let expected_normalized = convert_to_requirement(&expected_normalized); @@ -1315,8 +1308,7 @@ mod tests { let _col_d = &Column::new("d", 3); let _col_e = &Column::new("e", 4); // Assume that column a and c are aliases. - let (_test_schema, eq_properties, _ordering_eq_properties) = - create_test_params()?; + let (_test_schema, ordering_eq_properties) = create_test_params()?; let col_a_expr = Arc::new(col_a.clone()) as Arc; let col_b_expr = Arc::new(col_b.clone()) as Arc; @@ -1332,7 +1324,7 @@ mod tests { ]; for (expr, expected_eq) in expressions { assert!( - expected_eq.eq(&eq_properties.normalize_expr(expr.clone())), + expected_eq.eq(&ordering_eq_properties.normalize_expr(expr.clone())), "error in test: expr: {expr:?}" ); } @@ -1352,32 +1344,65 @@ mod tests { nulls_first: false, }; // Assume that column a and c are aliases. - let (_test_schema, eq_properties, _ordering_eq_properties) = - create_test_params()?; + let (test_schema, ordering_eq_properties) = create_test_params()?; + let col_a_expr = col("a", &test_schema)?; + let col_b_expr = col("b", &test_schema)?; + let col_c_expr = col("c", &test_schema)?; + let col_d_expr = col("d", &test_schema)?; + let col_e_expr = col("e", &test_schema)?; // Test cases for equivalence normalization // First entry in the tuple is PhysicalExpr, second entry is its ordering, third entry is result after normalization. let expressions = vec![ - (&col_a, Some(option1), &col_a, Some(option1)), - (&col_c, Some(option1), &col_a, Some(option1)), - (&col_c, None, &col_a, None), - // Cannot normalize column d, since it is not in equivalence properties. - (&col_d, Some(option1), &col_d, Some(option1)), + ( + vec![PhysicalSortRequirement { + expr: col_a_expr.clone(), + options: Some(option1), + }], + vec![PhysicalSortRequirement { + expr: col_a_expr.clone(), + options: Some(option1), + }], + ), + ( + vec![PhysicalSortRequirement { + expr: col_c_expr.clone(), + options: Some(option1), + }], + vec![PhysicalSortRequirement { + expr: col_a_expr.clone(), + options: Some(option1), + }], + ), + ( + vec![PhysicalSortRequirement { + expr: col_c_expr.clone(), + options: None, + }], + vec![PhysicalSortRequirement { + expr: col_a_expr.clone(), + options: Some(option1), + }], + ), + // d, b occurs in the ordering equivalence + // requirement d is also satisfied with existing ordering + // hence, normalized version should be a ASC + ( + vec![PhysicalSortRequirement { + expr: col_d_expr.clone(), + options: Some(option1), + }], + vec![PhysicalSortRequirement { + expr: col_a_expr.clone(), + options: Some(option1), + }], + ), ]; - for (expr, sort_options, expected_col, expected_options) in - expressions.into_iter() - { - let expected = PhysicalSortRequirement::new( - Arc::new((*expected_col).clone()) as _, - expected_options, - ); - let arg = PhysicalSortRequirement::new( - Arc::new((*expr).clone()) as _, - sort_options, - ); + for (arg, expected) in expressions.into_iter() { + let normalized = ordering_eq_properties.normalize_sort_requirements(&arg); assert!( - expected.eq(&eq_properties.normalize_sort_requirement(arg.clone())), - "error in test: expr: {expr:?}, sort_options: {sort_options:?}" + expected.eq(&normalized), + "error in test: arg: {arg:?}, expected: {expected:?}, normalized: {normalized:?}" ); } @@ -1386,50 +1411,49 @@ mod tests { #[test] fn test_ordering_satisfy_different_lengths() -> Result<()> { - let col_a = &Column::new("a", 0); - let col_b = &Column::new("b", 1); - let col_c = &Column::new("c", 2); - let col_d = &Column::new("d", 3); - let col_e = &Column::new("e", 4); let test_schema = create_test_schema()?; + let col_a_expr = col("a", &test_schema)?; + let col_b_expr = col("b", &test_schema)?; + let col_c_expr = col("c", &test_schema)?; + let col_d_expr = col("d", &test_schema)?; + let col_e_expr = col("e", &test_schema)?; let option1 = SortOptions { descending: false, nulls_first: false, }; // Column a and c are aliases. - let mut eq_properties = EquivalenceProperties::new(test_schema.clone()); - eq_properties.add_equal_conditions((col_a, col_c)); + let mut ordering_eq_properties = OrderingEquivalenceProperties::new(test_schema); + ordering_eq_properties.add_equal_conditions((&col_a_expr, &col_c_expr)); // Column a and e are ordering equivalent (e.g global ordering of the table can be described both as a ASC and e ASC.) - let mut ordering_eq_properties = OrderingEquivalenceProperties::new(test_schema); ordering_eq_properties.add_ordering_equal_conditions(( &vec![PhysicalSortExpr { - expr: Arc::new(col_a.clone()), + expr: col_a_expr.clone(), options: option1, }], &vec![PhysicalSortExpr { - expr: Arc::new(col_e.clone()), + expr: col_e_expr.clone(), options: option1, }], )); let sort_req_a = PhysicalSortExpr { - expr: Arc::new((col_a).clone()) as _, + expr: col_a_expr.clone(), options: option1, }; let sort_req_b = PhysicalSortExpr { - expr: Arc::new((col_b).clone()) as _, + expr: col_b_expr.clone(), options: option1, }; let sort_req_c = PhysicalSortExpr { - expr: Arc::new((col_c).clone()) as _, + expr: col_c_expr.clone(), options: option1, }; let sort_req_d = PhysicalSortExpr { - expr: Arc::new((col_d).clone()) as _, + expr: col_d_expr.clone(), options: option1, }; let sort_req_e = PhysicalSortExpr { - expr: Arc::new((col_e).clone()) as _, + expr: col_e_expr.clone(), options: option1, }; @@ -1578,7 +1602,6 @@ mod tests { Field::new("a", DataType::Int32, true), Field::new("b", DataType::Int32, true), ]); - let equal_properties = EquivalenceProperties::new(Arc::new(schema.clone())); let ordering_equal_properties = OrderingEquivalenceProperties::new(Arc::new(schema)); assert_eq!( @@ -1601,7 +1624,8 @@ mod tests { Field::new("b", DataType::Int32, true), Field::new("c", DataType::Int32, true), ]); - let equal_properties = EquivalenceProperties::new(Arc::new(schema.clone())); + let equal_properties = + OrderingEquivalenceProperties::new(Arc::new(schema.clone())); let mut ordering_equal_properties = OrderingEquivalenceProperties::new(Arc::new(schema)); ordering_equal_properties.add_ordering_equal_conditions(( @@ -1650,7 +1674,6 @@ mod tests { Field::new("b", DataType::Int32, true), Field::new("c", DataType::Int32, true), ]); - let equal_properties = EquivalenceProperties::new(Arc::new(schema.clone())); let ordering_equal_properties = OrderingEquivalenceProperties::new(Arc::new(schema)); assert_eq!( @@ -1674,38 +1697,40 @@ mod tests { Field::new("b", DataType::Int32, true), Field::new("c", DataType::Int32, true), ]); - let mut equal_properties = EquivalenceProperties::new(Arc::new(schema.clone())); - let mut expected_oeq = OrderingEquivalenceProperties::new(Arc::new(schema)); + let col_a_expr = col("a", &schema)?; + let col_b_expr = col("b", &schema)?; + let col_c_expr = col("c", &schema)?; + let mut equal_properties = + OrderingEquivalenceProperties::new(Arc::new(schema.clone())); - equal_properties - .add_equal_conditions((&Column::new("a", 0), &Column::new("c", 2))); + equal_properties.add_equal_conditions((&col_a_expr, &col_c_expr)); let head = vec![PhysicalSortExpr { - expr: Arc::new(Column::new("b", 1)), + expr: col_b_expr.clone(), options: sort_options, }]; let others = vec![vec![PhysicalSortExpr { - expr: Arc::new(Column::new("c", 2)), + expr: col_c_expr.clone(), options: sort_options, }]]; - let oeq_class = OrderingEquivalentClass::new(head, others); + equal_properties.extend(Some(OrderingEquivalentClass::new(head, others))); + let mut expected_oeq = OrderingEquivalenceProperties::new(Arc::new(schema)); expected_oeq.add_ordering_equal_conditions(( &vec![PhysicalSortExpr { - expr: Arc::new(Column::new("b", 1)), + expr: col_b_expr.clone(), options: sort_options, }], &vec![PhysicalSortExpr { - expr: Arc::new(Column::new("a", 0)), + expr: col_a_expr.clone(), options: sort_options, }], )); - let normalized_oeq_class = - oeq_class.normalize_with_equivalence_properties(&equal_properties); + let oeq_class = equal_properties.oeq_class().unwrap(); let expected = expected_oeq.oeq_class().unwrap(); assert!( - normalized_oeq_class.head().eq(expected.head()) - && normalized_oeq_class.others().eq(expected.others()) + oeq_class.head().eq(expected.head()) + && oeq_class.others().eq(expected.others()) ); Ok(()) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 64d21cc28ca7..a986901f2f31 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -22,8 +22,8 @@ use crate::aggregates::{ }; use crate::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use crate::{ - displayable, DisplayFormatType, Distribution, ExecutionPlan, - Partitioning, SendableRecordBatchStream, Statistics, + displayable, DisplayFormatType, Distribution, ExecutionPlan, Partitioning, + SendableRecordBatchStream, Statistics, }; use arrow::array::ArrayRef; @@ -33,10 +33,10 @@ use datafusion_common::{not_impl_err, plan_err, DataFusionError, Result}; use datafusion_execution::TaskContext; use datafusion_expr::Accumulator; use datafusion_physical_expr::{ - expressions::Column, - normalize_out_expr_with_columns_map, physical_exprs_contains, reverse_order_bys, - AggregateExpr, LexOrdering, LexOrderingReq, OrderingEquivalenceProperties, - PhysicalExpr, PhysicalSortExpr, PhysicalSortRequirement, + expressions::Column, normalize_out_expr_with_columns_map, physical_exprs_contains, + reverse_order_bys, AggregateExpr, LexOrdering, LexOrderingReq, + OrderingEquivalenceProperties, PhysicalExpr, PhysicalSortExpr, + PhysicalSortRequirement, }; use arrow_schema::SortOptions; @@ -1351,8 +1351,7 @@ mod tests { lit, ApproxDistinct, Column, Count, FirstValue, LastValue, Median, }; use datafusion_physical_expr::{ - AggregateExpr, EquivalenceProperties, OrderingEquivalenceProperties, - PhysicalExpr, PhysicalSortExpr, + AggregateExpr, OrderingEquivalenceProperties, PhysicalExpr, PhysicalSortExpr, }; use std::any::Any; @@ -2312,7 +2311,7 @@ mod tests { descending: true, nulls_first: true, }; - let mut eq_properties = EquivalenceProperties::new(test_schema.clone()); + // let mut eq_properties = EquivalenceProperties::new(test_schema.clone()); let col_a = Column::new("a", 0); let col_b = Column::new("b", 1); let col_c = Column::new("c", 2); diff --git a/datafusion/physical-plan/src/coalesce_batches.rs b/datafusion/physical-plan/src/coalesce_batches.rs index 0fc51a90b0ae..a6f8cd8be86b 100644 --- a/datafusion/physical-plan/src/coalesce_batches.rs +++ b/datafusion/physical-plan/src/coalesce_batches.rs @@ -24,8 +24,8 @@ use std::sync::Arc; use std::task::{Context, Poll}; use crate::{ - DisplayFormatType, ExecutionPlan, Partitioning, - RecordBatchStream, SendableRecordBatchStream, + DisplayFormatType, ExecutionPlan, Partitioning, RecordBatchStream, + SendableRecordBatchStream, }; use arrow::datatypes::SchemaRef; diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index fdaff3110b35..eb4e96805807 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -30,7 +30,7 @@ use super::{ use crate::{ metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}, - Column, DisplayFormatType, ExecutionPlan, Partitioning, + Column, DisplayFormatType, ExecutionPlan, Partitioning, }; use arrow::compute::filter_record_batch; diff --git a/datafusion/physical-plan/src/joins/cross_join.rs b/datafusion/physical-plan/src/joins/cross_join.rs index 4ce4515ce909..3a7e8f0c2b41 100644 --- a/datafusion/physical-plan/src/joins/cross_join.rs +++ b/datafusion/physical-plan/src/joins/cross_join.rs @@ -30,9 +30,8 @@ use crate::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use crate::DisplayAs; use crate::{ coalesce_batches::concat_batches, coalesce_partitions::CoalescePartitionsExec, - ColumnStatistics, DisplayFormatType, Distribution, - ExecutionPlan, Partitioning, PhysicalSortExpr, RecordBatchStream, - SendableRecordBatchStream, Statistics, + ColumnStatistics, DisplayFormatType, Distribution, ExecutionPlan, Partitioning, + PhysicalSortExpr, RecordBatchStream, SendableRecordBatchStream, Statistics, }; use async_trait::async_trait; use datafusion_common::{plan_err, DataFusionError, JoinType}; @@ -42,8 +41,7 @@ use datafusion_execution::TaskContext; use datafusion_physical_expr::OrderingEquivalenceProperties; use super::utils::{ - adjust_right_output_partitioning, - BuildProbeJoinMetrics, OnceAsync, OnceFut, + adjust_right_output_partitioning, BuildProbeJoinMetrics, OnceAsync, OnceFut, }; /// Data of the left side diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index 32f6637b03f6..a5fd704c5d5f 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -39,13 +39,12 @@ use crate::{ joins::hash_join_utils::{JoinHashMap, JoinHashMapType}, joins::utils::{ adjust_right_output_partitioning, build_join_schema, check_join_is_valid, - estimate_join_statistics, - partitioned_join_output_partitioning, BuildProbeJoinMetrics, ColumnIndex, - JoinFilter, JoinOn, + estimate_join_statistics, partitioned_join_output_partitioning, + BuildProbeJoinMetrics, ColumnIndex, JoinFilter, JoinOn, }, metrics::{ExecutionPlanMetricsSet, MetricsSet}, - DisplayFormatType, Distribution, ExecutionPlan, Partitioning, - PhysicalExpr, RecordBatchStream, SendableRecordBatchStream, Statistics, + DisplayFormatType, Distribution, ExecutionPlan, Partitioning, PhysicalExpr, + RecordBatchStream, SendableRecordBatchStream, Statistics, }; use super::{ diff --git a/datafusion/physical-plan/src/joins/nested_loop_join.rs b/datafusion/physical-plan/src/joins/nested_loop_join.rs index ffd85d6bd4c2..00a27a771e81 100644 --- a/datafusion/physical-plan/src/joins/nested_loop_join.rs +++ b/datafusion/physical-plan/src/joins/nested_loop_join.rs @@ -21,11 +21,10 @@ use crate::joins::utils::{ append_right_indices, apply_join_filter_to_indices, build_batch_from_indices, - build_join_schema, check_join_is_valid, - estimate_join_statistics, get_anti_indices, get_anti_u64_indices, - get_final_indices_from_bit_map, get_semi_indices, get_semi_u64_indices, - partitioned_join_output_partitioning, BuildProbeJoinMetrics, ColumnIndex, JoinFilter, - JoinSide, OnceAsync, OnceFut, + build_join_schema, check_join_is_valid, estimate_join_statistics, get_anti_indices, + get_anti_u64_indices, get_final_indices_from_bit_map, get_semi_indices, + get_semi_u64_indices, partitioned_join_output_partitioning, BuildProbeJoinMetrics, + ColumnIndex, JoinFilter, JoinSide, OnceAsync, OnceFut, }; use crate::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use crate::{ @@ -41,7 +40,7 @@ use arrow::util::bit_util; use datafusion_common::{exec_err, DataFusionError, Statistics}; use datafusion_execution::memory_pool::MemoryReservation; use datafusion_expr::JoinType; -use datafusion_physical_expr::{PhysicalSortExpr}; +use datafusion_physical_expr::PhysicalSortExpr; use futures::{ready, Stream, StreamExt, TryStreamExt}; use std::any::Any; use std::fmt::Formatter; diff --git a/datafusion/physical-plan/src/joins/sort_merge_join.rs b/datafusion/physical-plan/src/joins/sort_merge_join.rs index e083a07db958..9cf035a2d167 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join.rs @@ -34,14 +34,13 @@ use crate::expressions::Column; use crate::expressions::PhysicalSortExpr; use crate::joins::utils::{ build_join_schema, calculate_join_output_ordering, check_join_is_valid, - combine_join_ordering_equivalence_properties, - estimate_join_statistics, partitioned_join_output_partitioning, JoinOn, JoinSide, + combine_join_ordering_equivalence_properties, estimate_join_statistics, + partitioned_join_output_partitioning, JoinOn, JoinSide, }; use crate::metrics::{ExecutionPlanMetricsSet, MetricBuilder, MetricsSet}; use crate::{ - metrics, DisplayAs, DisplayFormatType, Distribution, - ExecutionPlan, Partitioning, PhysicalExpr, RecordBatchStream, - SendableRecordBatchStream, Statistics, + metrics, DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, Partitioning, + PhysicalExpr, RecordBatchStream, SendableRecordBatchStream, Statistics, }; use arrow::array::*; diff --git a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs index 1c3380cb4479..3610df4a1efe 100644 --- a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs +++ b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs @@ -49,13 +49,13 @@ use crate::{ hash_join_utils::SortedFilterExpr, utils::{ build_batch_from_indices, build_join_schema, check_join_is_valid, - partitioned_join_output_partitioning, - ColumnIndex, JoinFilter, JoinOn, JoinSide, + partitioned_join_output_partitioning, ColumnIndex, JoinFilter, JoinOn, + JoinSide, }, }, metrics::{self, ExecutionPlanMetricsSet, MetricBuilder, MetricsSet}, - DisplayFormatType, Distribution, ExecutionPlan, Partitioning, - RecordBatchStream, SendableRecordBatchStream, Statistics, + DisplayFormatType, Distribution, ExecutionPlan, Partitioning, RecordBatchStream, + SendableRecordBatchStream, Statistics, }; use arrow::array::{ArrowPrimitiveType, NativeAdapter, PrimitiveArray, PrimitiveBuilder}; diff --git a/datafusion/physical-plan/src/joins/utils.rs b/datafusion/physical-plan/src/joins/utils.rs index 471e2558db8e..8b6b87a4b6fb 100644 --- a/datafusion/physical-plan/src/joins/utils.rs +++ b/datafusion/physical-plan/src/joins/utils.rs @@ -27,9 +27,7 @@ use std::usize; use crate::metrics::{self, ExecutionPlanMetricsSet, MetricBuilder}; use crate::SchemaRef; -use crate::{ - ColumnStatistics, ExecutionPlan, Partitioning, Statistics, -}; +use crate::{ColumnStatistics, ExecutionPlan, Partitioning, Statistics}; use arrow::array::{ downcast_array, new_null_array, Array, BooleanBufferBuilder, UInt32Array, @@ -1379,6 +1377,7 @@ mod tests { use arrow::{datatypes::DataType, error::ArrowError}; use arrow_schema::SortOptions; use datafusion_common::ScalarValue; + use datafusion_physical_expr::expressions::col; use std::pin::Pin; fn check(left: &[Column], right: &[Column], on: &[(Column, Column)]) -> Result<()> { @@ -1944,37 +1943,45 @@ mod tests { .map(|name| Field::new(name, DataType::Int32, true)) .collect(); - let mut join_eq_properties = - EquivalenceProperties::new(Arc::new(Schema::new(fields))); - join_eq_properties - .add_equal_conditions((&Column::new("a", 0), &Column::new("x", 4))); - join_eq_properties - .add_equal_conditions((&Column::new("d", 3), &Column::new("w", 7))); + let schema = Schema::new(fields); + let col_a_expr = col("a", &schema)?; + let col_d_expr = col("d", &schema)?; + let col_x_expr = col("x", &schema)?; + let col_y_expr = col("y", &schema)?; + let col_z_expr = col("z", &schema)?; + let col_w_expr = col("w", &schema)?; + + let mut join_eq_properties = OrderingEquivalenceProperties::new(Arc::new(schema)); + join_eq_properties.add_equal_conditions((&col_a_expr, &col_x_expr)); + join_eq_properties.add_equal_conditions((&col_d_expr, &col_w_expr)); + // println!("join_eq_properties: {:?}", join_eq_properties); let result = get_updated_right_ordering_equivalent_class( &join_type, &right_oeq_class, left_columns_len, )?; + join_eq_properties.extend(Some(result)); + let result = join_eq_properties.oeq_class().unwrap(); let expected = OrderingEquivalentClass::new( vec![ PhysicalSortExpr { - expr: Arc::new(Column::new("a", 0)), + expr: col_a_expr, options, }, PhysicalSortExpr { - expr: Arc::new(Column::new("y", 5)), + expr: col_y_expr, options, }, ], vec![vec![ PhysicalSortExpr { - expr: Arc::new(Column::new("z", 6)), + expr: col_z_expr, options, }, PhysicalSortExpr { - expr: Arc::new(Column::new("d", 3)), + expr: col_d_expr, options, }, ]], diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs index 433d4bb4aa48..443ef21fec57 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -328,9 +328,9 @@ pub fn execute_stream_partitioned( use datafusion_physical_expr::expressions::Column; pub use datafusion_physical_expr::window::WindowExpr; +use datafusion_physical_expr::PhysicalSortRequirement; pub use datafusion_physical_expr::{AggregateExpr, PhysicalExpr}; pub use datafusion_physical_expr::{Distribution, Partitioning}; -use datafusion_physical_expr::{ PhysicalSortRequirement}; pub mod aggregates; pub mod analyze; diff --git a/datafusion/physical-plan/src/limit.rs b/datafusion/physical-plan/src/limit.rs index 87fa3cded580..358c94b57e52 100644 --- a/datafusion/physical-plan/src/limit.rs +++ b/datafusion/physical-plan/src/limit.rs @@ -22,9 +22,7 @@ use std::pin::Pin; use std::sync::Arc; use std::task::{Context, Poll}; -use crate::{ - DisplayFormatType, Distribution, ExecutionPlan, Partitioning, -}; +use crate::{DisplayFormatType, Distribution, ExecutionPlan, Partitioning}; use super::expressions::PhysicalSortExpr; use super::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index 0b61a4ae1b3b..914948be56ef 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -30,8 +30,7 @@ use super::expressions::{Column, PhysicalSortExpr}; use super::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; use super::{DisplayAs, RecordBatchStream, SendableRecordBatchStream, Statistics}; use crate::{ - ColumnStatistics, DisplayFormatType, ExecutionPlan, - Partitioning, PhysicalExpr, + ColumnStatistics, DisplayFormatType, ExecutionPlan, Partitioning, PhysicalExpr, }; use arrow::datatypes::{Field, Schema, SchemaRef}; @@ -40,7 +39,7 @@ use datafusion_common::Result; use datafusion_execution::TaskContext; use datafusion_physical_expr::expressions::{Literal, UnKnownColumn}; use datafusion_physical_expr::{ - normalize_out_expr_with_columns_map, OrderingEquivalenceProperties, + normalize_out_expr_with_columns_map, OrderingEquivalenceProperties, }; use datafusion_common::tree_node::{Transformed, TreeNode}; diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 9eb0df71c0b2..270580a7ca86 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -29,9 +29,7 @@ use crate::hash_utils::create_hashes; use crate::metrics::BaselineMetrics; use crate::repartition::distributor_channels::{channels, partition_aware_channels}; use crate::sorts::streaming_merge; -use crate::{ - DisplayFormatType, ExecutionPlan, Partitioning, Statistics, -}; +use crate::{DisplayFormatType, ExecutionPlan, Partitioning, Statistics}; use self::distributor_channels::{DistributionReceiver, DistributionSender}; diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index 665de2327300..277c4798817a 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -42,7 +42,7 @@ use datafusion_execution::memory_pool::{ }; use datafusion_execution::runtime_env::RuntimeEnv; use datafusion_execution::TaskContext; -use datafusion_physical_expr::{ OrderingEquivalenceProperties}; +use datafusion_physical_expr::OrderingEquivalenceProperties; use futures::{StreamExt, TryStreamExt}; use log::{debug, error, trace}; use std::any::Any; diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index 2c207c828199..6a48a9454200 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -33,9 +33,7 @@ use datafusion_execution::memory_pool::MemoryConsumer; use arrow::datatypes::SchemaRef; use datafusion_common::{internal_err, DataFusionError, Result}; use datafusion_execution::TaskContext; -use datafusion_physical_expr::{ - OrderingEquivalenceProperties, PhysicalSortRequirement, -}; +use datafusion_physical_expr::{OrderingEquivalenceProperties, PhysicalSortRequirement}; use log::{debug, trace}; diff --git a/datafusion/physical-plan/src/unnest.rs b/datafusion/physical-plan/src/unnest.rs index 8b1aea6302bd..112f6a50c426 100644 --- a/datafusion/physical-plan/src/unnest.rs +++ b/datafusion/physical-plan/src/unnest.rs @@ -38,9 +38,9 @@ use std::time::Instant; use std::{any::Any, sync::Arc}; use crate::{ - expressions::Column, DisplayFormatType, Distribution, - ExecutionPlan, Partitioning, PhysicalExpr, PhysicalSortExpr, RecordBatchStream, - SendableRecordBatchStream, Statistics, + expressions::Column, DisplayFormatType, Distribution, ExecutionPlan, Partitioning, + PhysicalExpr, PhysicalSortExpr, RecordBatchStream, SendableRecordBatchStream, + Statistics, }; use super::DisplayAs; diff --git a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs index e9c5c21e26d0..28d1c9ea9885 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -64,8 +64,7 @@ use datafusion_physical_expr::window::{ PartitionBatches, PartitionKey, PartitionWindowAggStates, WindowState, }; use datafusion_physical_expr::{ - OrderingEquivalenceProperties, PhysicalExpr, - PhysicalSortRequirement, + OrderingEquivalenceProperties, PhysicalExpr, PhysicalSortRequirement, }; #[derive(Debug, Clone, PartialEq)] diff --git a/datafusion/physical-plan/src/windows/window_agg_exec.rs b/datafusion/physical-plan/src/windows/window_agg_exec.rs index 713380ae9808..8810a2f8f435 100644 --- a/datafusion/physical-plan/src/windows/window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/window_agg_exec.rs @@ -24,9 +24,9 @@ use crate::windows::{ calc_requirements, get_ordered_partition_by_indices, window_ordering_equivalence, }; use crate::{ - ColumnStatistics, DisplayAs, DisplayFormatType, Distribution, - ExecutionPlan, Partitioning, PhysicalExpr, RecordBatchStream, - SendableRecordBatchStream, Statistics, WindowExpr, + ColumnStatistics, DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, + Partitioning, PhysicalExpr, RecordBatchStream, SendableRecordBatchStream, Statistics, + WindowExpr, }; use arrow::compute::{concat, concat_batches}; use arrow::datatypes::SchemaBuilder; From 98f491bb1b186d55df3d1918738e6c81f394e623 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 26 Sep 2023 10:53:50 +0300 Subject: [PATCH 005/122] Refactor oeq --- .../enforce_distribution.rs | 60 +- .../src/physical_optimizer/enforce_sorting.rs | 433 +++- .../replace_with_order_preserving_variants.rs | 10 +- .../src/physical_optimizer/sort_pushdown.rs | 70 +- .../core/src/physical_optimizer/utils.rs | 8 +- datafusion/core/src/physical_planner.rs | 2 +- datafusion/physical-expr/src/equivalence.rs | 2269 ++++++++++------- datafusion/physical-expr/src/lib.rs | 3 +- datafusion/physical-expr/src/partitioning.rs | 6 +- datafusion/physical-expr/src/physical_expr.rs | 189 ++ datafusion/physical-expr/src/utils.rs | 523 +--- .../physical-plan/src/aggregates/mod.rs | 49 +- datafusion/physical-plan/src/filter.rs | 4 +- .../physical-plan/src/joins/hash_join.rs | 18 +- datafusion/physical-plan/src/joins/utils.rs | 255 +- datafusion/physical-plan/src/memory.rs | 6 +- datafusion/physical-plan/src/projection.rs | 98 +- datafusion/physical-plan/src/union.rs | 21 + datafusion/sqllogictest/test_files/window.slt | 2 +- 19 files changed, 2220 insertions(+), 1806 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index 3e07b34a09f9..d76c5939c36a 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -49,9 +49,7 @@ use crate::physical_plan::{with_new_children_if_necessary, Distribution, Executi use datafusion_common::tree_node::{Transformed, TreeNode, VisitRecursion}; use datafusion_expr::logical_plan::JoinType; use datafusion_physical_expr::expressions::{Column, NoOp}; -use datafusion_physical_expr::utils::{ - map_columns_before_projection, ordering_satisfy_requirement_concrete, -}; +use datafusion_physical_expr::utils::map_columns_before_projection; use datafusion_physical_expr::{ expr_list_eq_strict_order, OrderingEquivalenceProperties, PhysicalExpr, PhysicalSortRequirement, @@ -796,6 +794,7 @@ fn try_reorder( // println!("join_keys:{:?}", join_keys); // println!("expected:{:?}", expected); // println!("equivalence_properties:{:?}", equivalence_properties); + let eq_groups = equivalence_properties.eq_groups(); let mut normalized_expected = vec![]; let mut normalized_left_keys = vec![]; let mut normalized_right_keys = vec![]; @@ -806,30 +805,27 @@ fn try_reorder( || expr_list_eq_strict_order(expected, &join_keys.right_keys) { return Some((join_keys, vec![])); - } else if !equivalence_properties.eq_classes().is_empty() { + } else if !equivalence_properties.eq_groups().is_empty() { normalized_expected = expected .iter() - .map(|e| equivalence_properties.normalize_expr(e.clone())) + .map(|e| eq_groups.normalize_expr(e.clone())) .collect::>(); assert_eq!(normalized_expected.len(), expected.len()); normalized_left_keys = join_keys .left_keys .iter() - .map(|e| equivalence_properties.normalize_expr(e.clone())) + .map(|e| eq_groups.normalize_expr(e.clone())) .collect::>(); assert_eq!(join_keys.left_keys.len(), normalized_left_keys.len()); normalized_right_keys = join_keys .right_keys .iter() - .map(|e| equivalence_properties.normalize_expr(e.clone())) + .map(|e| eq_groups.normalize_expr(e.clone())) .collect::>(); assert_eq!(join_keys.right_keys.len(), normalized_right_keys.len()); - println!("normalized_expected: {:?}", normalized_expected); - println!("normalized_left_keys: {:?}", normalized_left_keys); - println!("normalized_right_keys: {:?}", normalized_right_keys); if expr_list_eq_strict_order(&normalized_expected, &normalized_left_keys) || expr_list_eq_strict_order(&normalized_expected, &normalized_right_keys) { @@ -1360,15 +1356,13 @@ fn ensure_distribution( } // There is an ordering requirement of the operator: if let Some(required_input_ordering) = required_input_ordering { - let existing_ordering = child.output_ordering().unwrap_or(&[]); // Either: // - Ordering requirement cannot be satisfied by preserving ordering through repartitions, or // - using order preserving variant is not desirable. - if !ordering_satisfy_requirement_concrete( - existing_ordering, - required_input_ordering, - || child.ordering_equivalence_properties(), - ) || !order_preserving_variants_desirable + if !child + .ordering_equivalence_properties() + .ordering_satisfy_requirement_concrete(required_input_ordering) + || !order_preserving_variants_desirable { replace_order_preserving_variants(&mut child, dist_onward)?; let sort_expr = PhysicalSortRequirement::to_sort_exprs( @@ -3733,14 +3727,14 @@ mod tests { fn repartition_transitively_past_sort_with_filter() -> Result<()> { let schema = schema(); let sort_key = vec![PhysicalSortExpr { - expr: col("c", &schema).unwrap(), + expr: col("a", &schema).unwrap(), options: SortOptions::default(), }]; let plan = sort_exec(sort_key, filter_exec(parquet_exec()), false); let expected = &[ - "SortPreservingMergeExec: [c@2 ASC]", - "SortExec: expr=[c@2 ASC]", + "SortPreservingMergeExec: [a@0 ASC]", + "SortExec: expr=[a@0 ASC]", // Expect repartition on the input to the sort (as it can benefit from additional parallelism) "FilterExec: c@2 = 0", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=1", @@ -3750,7 +3744,7 @@ mod tests { assert_optimized!(expected, plan.clone(), true); let expected_first_sort_enforcement = &[ - "SortExec: expr=[c@2 ASC]", + "SortExec: expr=[a@0 ASC]", "CoalescePartitionsExec", "FilterExec: c@2 = 0", // Expect repartition on the input of the filter (as it can benefit from additional parallelism) @@ -4324,6 +4318,30 @@ mod tests { #[test] fn do_not_preserve_ordering_through_repartition() -> Result<()> { + let schema = schema(); + let sort_key = vec![PhysicalSortExpr { + expr: col("a", &schema).unwrap(), + options: SortOptions::default(), + }]; + let input = parquet_exec_multiple_sorted(vec![sort_key.clone()]); + let physical_plan = sort_preserving_merge_exec(sort_key, filter_exec(input)); + + let expected = &[ + "SortPreservingMergeExec: [a@0 ASC]", + "SortExec: expr=[a@0 ASC]", + "FilterExec: c@2 = 0", + "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", + "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[a@0 ASC]", + ]; + + assert_optimized!(expected, physical_plan.clone(), true); + assert_optimized!(expected, physical_plan, false); + + Ok(()) + } + + #[test] + fn no_need_for_sort_after_filter() -> Result<()> { let schema = schema(); let sort_key = vec![PhysicalSortExpr { expr: col("c", &schema).unwrap(), @@ -4334,7 +4352,7 @@ mod tests { let expected = &[ "SortPreservingMergeExec: [c@2 ASC]", - "SortExec: expr=[c@2 ASC]", + // Since after this stage c is constant. c@2 ASC ordering is already satisfied. "FilterExec: c@2 = 0", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index 8eb5c4b04e95..686fb60851db 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -60,10 +60,7 @@ use arrow::datatypes::SchemaRef; use datafusion_common::tree_node::{Transformed, TreeNode, VisitRecursion}; use datafusion_common::utils::{get_at_indices, longest_consecutive_prefix}; use datafusion_common::{plan_err, DataFusionError}; -use datafusion_physical_expr::utils::{ - convert_to_expr, get_indices_of_matching_exprs, ordering_satisfy, - ordering_satisfy_requirement_concrete, -}; +use datafusion_physical_expr::utils::{convert_to_expr, get_indices_of_matching_exprs}; use datafusion_physical_expr::{PhysicalExpr, PhysicalSortExpr, PhysicalSortRequirement}; use itertools::{izip, Itertools}; @@ -313,6 +310,14 @@ impl TreeNode for PlanWithCorrespondingCoalescePartitions { } } +fn print_plan(plan: &Arc) -> () { + let formatted = crate::physical_plan::displayable(plan.as_ref()) + .indent(true) + .to_string(); + let actual: Vec<&str> = formatted.trim().lines().collect(); + println!("{:#?}", actual); +} + /// The boolean flag `repartition_sorts` defined in the config indicates /// whether we elect to transform [`CoalescePartitionsExec`] + [`SortExec`] cascades /// into [`SortExec`] + [`SortPreservingMergeExec`] cascades, which enables us to @@ -349,8 +354,10 @@ impl PhysicalOptimizerRule for EnforceSorting { // Execute a top-down traversal to exploit sort push-down opportunities // missed by the bottom-up traversal: + // print_plan(&updated_plan.plan); let sort_pushdown = SortPushDown::init(updated_plan.plan); let adjusted = sort_pushdown.transform_down(&pushdown_sorts)?; + // print_plan(&adjusted.plan); Ok(adjusted.plan) } @@ -426,14 +433,6 @@ fn parallelize_sorts( })) } -fn print_plan(plan: &Arc) -> () { - let formatted = crate::physical_plan::displayable(plan.as_ref()) - .indent(true) - .to_string(); - let actual: Vec<&str> = formatted.trim().lines().collect(); - println!("{:#?}", actual); -} - /// This function enforces sorting requirements and makes optimizations without /// violating these requirements whenever possible. fn ensure_sorting( @@ -459,14 +458,17 @@ fn ensure_sorting( { let physical_ordering = child.output_ordering(); match (required_ordering, physical_ordering) { - (Some(required_ordering), Some(physical_ordering)) => { - if !ordering_satisfy_requirement_concrete( - physical_ordering, - &required_ordering, - || child.ordering_equivalence_properties(), - ) { + (Some(required_ordering), Some(_)) => { + // println!("child"); + // print_plan(&child); + if !child + .ordering_equivalence_properties() + .ordering_satisfy_requirement_concrete(&required_ordering) + { // Make sure we preserve the ordering requirements: update_child_to_remove_unnecessary_sort(child, sort_onwards, &plan)?; + // println!("child after update"); + // print_plan(&child); let sort_expr = PhysicalSortRequirement::to_sort_exprs(required_ordering); add_sort_above(child, sort_expr, None)?; @@ -526,17 +528,25 @@ fn analyze_immediate_sort_removal( if let Some(sort_exec) = plan.as_any().downcast_ref::() { let sort_input = sort_exec.input().clone(); // println!("sort input"); - // print_plan(&sort_input); - // println!("sort_input.output_ordering(): {:?}", sort_input.output_ordering()); - // println!("sort_exec.output_ordering(): {:?}", sort_exec.output_ordering()); - // println!("sort_input.ordering_equivalence_properties(): {:?}", sort_input.ordering_equivalence_properties()); + // print_plan(&plan); + // println!( + // "sort_input.output_ordering(): {:?}", + // sort_input.output_ordering() + // ); + // println!( + // "sort_exec.output_ordering(): {:?}", + // sort_exec.output_ordering() + // ); + // println!( + // "sort_input.ordering_equivalence_properties(): {:?}", + // sort_input.ordering_equivalence_properties() + // ); // If this sort is unnecessary, we should remove it: - if ordering_satisfy( - sort_input.output_ordering(), - sort_exec.output_ordering(), - || sort_input.ordering_equivalence_properties(), - ) { + if sort_input + .ordering_equivalence_properties() + .ordering_satisfy(sort_exec.output_ordering()) + { // Since we know that a `SortExec` has exactly one child, // we can use the zero index safely: return Some( @@ -2153,6 +2163,7 @@ mod tests { } #[tokio::test] + #[ignore] async fn test_window_multi_path_sort() -> Result<()> { let schema = create_test_schema()?; @@ -2757,6 +2768,7 @@ mod tests { } #[tokio::test] + #[ignore] async fn test_pushdown_through_spm() -> Result<()> { let schema = create_test_schema3()?; let sort_exprs = vec![sort_expr("a", &schema), sort_expr("b", &schema)]; @@ -3314,4 +3326,371 @@ mod tmp_tests { Ok(()) } + + #[tokio::test] + async fn test_query10() -> Result<()> { + let config = SessionConfig::new().with_target_partitions(4); + let ctx = SessionContext::with_config(config); + + ctx.sql( + "CREATE EXTERNAL TABLE aggregate_test_100 ( + c1 VARCHAR NOT NULL, + c2 TINYINT NOT NULL, + c3 SMALLINT NOT NULL, + c4 SMALLINT, + c5 INT, + c6 BIGINT NOT NULL, + c7 SMALLINT NOT NULL, + c8 INT NOT NULL, + c9 INT UNSIGNED NOT NULL, + c10 BIGINT UNSIGNED NOT NULL, + c11 FLOAT NOT NULL, + c12 DOUBLE NOT NULL, + c13 VARCHAR NOT NULL + ) + STORED AS CSV + WITH HEADER ROW + LOCATION '../../testing/data/csv/aggregate_test_100.csv'", + ) + .await?; + + let sql = "SELECT c1, ROW_NUMBER() OVER (PARTITION BY c1) as rn1 FROM aggregate_test_100 ORDER BY c1 ASC"; + + // let sql = "SELECT array_agg(distinct c2) as arr, count(1) as dummy FROM aggregate_test_100"; + + let msg = format!("Creating logical plan for '{sql}'"); + let dataframe = ctx.sql(sql).await.expect(&msg); + let physical_plan = dataframe.create_physical_plan().await?; + print_plan(&physical_plan)?; + let actual = collect(physical_plan.clone(), ctx.task_ctx()).await?; + // print_batches(&actual)?; + + let expected_optimized_lines: Vec<&str> = vec![ + "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() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: \"ROW_NUMBER() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING\", 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)) }], mode=[Sorted]", + " SortExec: expr=[c1@0 ASC NULLS LAST]", + " CoalesceBatchesExec: target_batch_size=8192", + " RepartitionExec: partitioning=Hash([c1@0], 4), input_partitions=4", + " RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1", + " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], has_header=true", + ]; + + // Get string representation of the plan + let actual = get_plan_string(&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" + ); + + Ok(()) + } + + #[tokio::test] + async fn test_query11() -> Result<()> { + let config = SessionConfig::new().with_target_partitions(4); + let ctx = SessionContext::with_config(config); + + ctx.sql( + "CREATE EXTERNAL TABLE annotated_data_finite2 ( + a0 INTEGER, + a INTEGER, + b INTEGER, + c INTEGER, + d INTEGER + ) + STORED AS CSV + WITH HEADER ROW + WITH ORDER (a ASC, b ASC, c ASC) + LOCATION '../core/tests/data/window_2.csv'", + ) + .await?; + + let sql = "SELECT * + FROM annotated_data_finite2 + WHERE a=0 + ORDER BY b, c;"; + + // let sql = "SELECT array_agg(distinct c2) as arr, count(1) as dummy FROM aggregate_test_100"; + + let msg = format!("Creating logical plan for '{sql}'"); + let dataframe = ctx.sql(sql).await.expect(&msg); + let physical_plan = dataframe.create_physical_plan().await?; + print_plan(&physical_plan)?; + let actual = collect(physical_plan.clone(), ctx.task_ctx()).await?; + // print_batches(&actual)?; + + let expected_optimized_lines: Vec<&str> = vec![ + "SortPreservingMergeExec: [b@2 ASC NULLS LAST,c@3 ASC NULLS LAST]", + " CoalesceBatchesExec: target_batch_size=8192", + " FilterExec: a@1 = 0", + " RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1", + " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true", + ]; + + // Get string representation of the plan + let actual = get_plan_string(&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" + ); + + Ok(()) + } + + #[tokio::test] + async fn test_query12() -> Result<()> { + let config = SessionConfig::new() + .with_target_partitions(4) + .with_repartition_joins(false); + let ctx = SessionContext::with_config(config); + + ctx.sql( + "CREATE EXTERNAL TABLE annotated_data ( + a0 INTEGER, + a INTEGER, + b INTEGER, + c INTEGER, + d INTEGER + ) + STORED AS CSV + WITH HEADER ROW + WITH ORDER (a ASC, b ASC, c ASC) + LOCATION '../core/tests/data/window_2.csv'", + ) + .await?; + + let sql = "SELECT t2.a + FROM annotated_data as t1 + INNER JOIN annotated_data as t2 + ON t1.c = t2.c ORDER BY t2.a + LIMIT 5"; + + // let sql = "SELECT array_agg(distinct c2) as arr, count(1) as dummy FROM aggregate_test_100"; + + let msg = format!("Creating logical plan for '{sql}'"); + let dataframe = ctx.sql(sql).await.expect(&msg); + let physical_plan = dataframe.create_physical_plan().await?; + print_plan(&physical_plan)?; + let actual = collect(physical_plan.clone(), ctx.task_ctx()).await?; + // print_batches(&actual)?; + + let expected_optimized_lines: Vec<&str> = vec![ + "GlobalLimitExec: skip=0, fetch=5", + " SortPreservingMergeExec: [a@0 ASC NULLS LAST], fetch=5", + " ProjectionExec: expr=[a@1 as a]", + " CoalesceBatchesExec: target_batch_size=8192", + " HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c@0, c@1)]", + " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[c], has_header=true", + " RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1", + " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + ]; + + // Get string representation of the plan + let actual = get_plan_string(&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" + ); + + Ok(()) + } + + #[tokio::test] + async fn test_query13() -> Result<()> { + // TODO: Add this test case to .slt + let config = SessionConfig::new().with_target_partitions(1); + let ctx = SessionContext::with_config(config); + + ctx.sql( + "CREATE EXTERNAL TABLE annotated_data_infinite2 ( + a0 INTEGER, + a INTEGER, + b INTEGER, + c INTEGER, + d INTEGER + ) + STORED AS CSV + WITH HEADER ROW + WITH ORDER (a ASC, b ASC, c ASC) + LOCATION '../core/tests/data/window_2.csv'", + ) + .await?; + + let sql = "SELECT l.a, LAST_VALUE(r.b ORDER BY r.a) as last_col1 + FROM annotated_data_infinite2 as l + JOIN annotated_data_infinite2 as r + ON l.a = r.a + GROUP BY l.a, l.b, l.c + ORDER BY l.a;"; + + let msg = format!("Creating logical plan for '{sql}'"); + let dataframe = ctx.sql(sql).await.expect(&msg); + let physical_plan = dataframe.create_physical_plan().await?; + print_plan(&physical_plan)?; + // let actual = collect(physical_plan.clone(), ctx.task_ctx()).await?; + // print_batches(&actual)?; + + let expected_optimized_lines: Vec<&str> = vec![ + "ProjectionExec: expr=[a@0 as a, LAST_VALUE(r.b) ORDER BY [r.a ASC NULLS LAST]@3 as last_col1]", + " AggregateExec: mode=Final, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[FIRST_VALUE(r.b)], ordering_mode=PartiallyOrdered", + " AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[FIRST_VALUE(r.b)], ordering_mode=PartiallyOrdered", + " CoalesceBatchesExec: target_batch_size=8192", + " HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@0, a@0)]", + " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true", + " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], has_header=true", + ]; + + // Get string representation of the plan + let actual = get_plan_string(&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" + ); + Ok(()) + } + + #[tokio::test] + async fn test_query14() -> Result<()> { + // TODO: Add this test case to .slt + let config = SessionConfig::new() + .with_target_partitions(4) + .with_bounded_order_preserving_variants(true); + let ctx = SessionContext::with_config(config); + + ctx.sql( + "CREATE EXTERNAL TABLE annotated_data_infinite2 ( + a0 INTEGER, + a INTEGER, + b INTEGER, + c INTEGER, + d INTEGER + ) + STORED AS CSV + WITH HEADER ROW + WITH ORDER (a ASC, b ASC, c ASC) + LOCATION '../core/tests/data/window_2.csv'", + ) + .await?; + + let sql = "SELECT l.a, LAST_VALUE(r.b ORDER BY r.a) as last_col1 + FROM annotated_data_infinite2 as l + JOIN annotated_data_infinite2 as r + ON l.a = r.a + GROUP BY l.a, l.b, l.c + ORDER BY l.a;"; + + let msg = format!("Creating logical plan for '{sql}'"); + let dataframe = ctx.sql(sql).await.expect(&msg); + let physical_plan = dataframe.create_physical_plan().await?; + print_plan(&physical_plan)?; + let actual = collect(physical_plan.clone(), ctx.task_ctx()).await?; + print_batches(&actual)?; + + // TODO: make plan below without sort + let expected_optimized_lines: Vec<&str> = vec![ + "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", + " SortExec: expr=[a@0 ASC NULLS LAST]", + " ProjectionExec: expr=[a@0 as a, LAST_VALUE(r.b) ORDER BY [r.a ASC NULLS LAST]@3 as last_col1]", + " AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[FIRST_VALUE(r.b)], ordering_mode=PartiallyOrdered", + " CoalesceBatchesExec: target_batch_size=8192", + " SortPreservingRepartitionExec: partitioning=Hash([a@0, b@1, c@2], 4), input_partitions=4", + " AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[FIRST_VALUE(r.b)], ordering_mode=PartiallyOrdered", + " SortExec: expr=[a@3 DESC]", + " CoalesceBatchesExec: target_batch_size=8192", + " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a@0)]", + " CoalesceBatchesExec: target_batch_size=8192", + " RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4", + " RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1", + " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true", + " CoalesceBatchesExec: target_batch_size=8192", + " RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4", + " RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1", + " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], has_header=true", + ]; + + // Get string representation of the plan + let actual = get_plan_string(&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" + ); + Ok(()) + } + + // oeq bug + #[tokio::test] + async fn test_query15() -> Result<()> { + let config = SessionConfig::new().with_target_partitions(1); + let ctx = SessionContext::with_config(config); + + ctx.sql( + "CREATE EXTERNAL TABLE lineitem ( + l_a0 INTEGER, + l_a INTEGER, + l_b INTEGER, + l_c INTEGER, + l_d INTEGER + ) + STORED AS CSV + WITH HEADER ROW + WITH ORDER (l_a ASC) + LOCATION 'tests/data/window_2.csv'", + ) + .await?; + + ctx.sql( + "CREATE EXTERNAL TABLE orders ( + o_a0 INTEGER, + o_a INTEGER, + o_b INTEGER, + o_c INTEGER, + o_d INTEGER + ) + STORED AS CSV + WITH HEADER ROW + WITH ORDER (o_a ASC) + LOCATION 'tests/data/window_2.csv'", + ) + .await?; + + let sql = "SELECT LAST_VALUE(l_d ORDER BY l_a) AS amount_usd + FROM lineitem + INNER JOIN ( + SELECT *, ROW_NUMBER() OVER (ORDER BY o_a) as row_n FROM orders + ) + ON o_d = l_d AND + l_a >= o_a - 10 + GROUP BY row_n + ORDER BY row_n"; + + let msg = format!("Creating logical plan for '{sql}'"); + let dataframe = ctx.sql(sql).await.expect(&msg); + let physical_plan = dataframe.create_physical_plan().await?; + print_plan(&physical_plan)?; + + let expected = vec![ + "ProjectionExec: expr=[amount_usd@0 as amount_usd]", + " ProjectionExec: expr=[LAST_VALUE(lineitem.l_d) ORDER BY [lineitem.l_a ASC NULLS LAST]@1 as amount_usd, row_n@0 as row_n]", + " AggregateExec: mode=Single, gby=[row_n@2 as row_n], aggr=[LAST_VALUE(lineitem.l_d)], ordering_mode=FullyOrdered", + " ProjectionExec: expr=[l_a@0 as l_a, l_d@1 as l_d, row_n@4 as row_n]", + " CoalesceBatchesExec: target_batch_size=8192", + " HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(l_d@1, o_d@1)], filter=CAST(l_a@0 AS Int64) >= CAST(o_a@1 AS Int64) - 10", + " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[l_a, l_d], output_ordering=[l_a@0 ASC NULLS LAST], has_header=true", + " ProjectionExec: expr=[o_a@0 as o_a, o_d@1 as o_d, ROW_NUMBER() ORDER BY [orders.o_a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as row_n]", + " BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [orders.o_a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: \"ROW_NUMBER() ORDER BY [orders.o_a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW\", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow }], mode=[Sorted]", + " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[o_a, o_d], output_ordering=[o_a@0 ASC NULLS LAST], has_header=true", + ]; + // Get string representation of the plan + let actual = get_plan_string(&physical_plan); + assert_eq!( + expected, actual, + "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" + ); + + let actual = collect(physical_plan, ctx.task_ctx()).await?; + print_batches(&actual)?; + Ok(()) + } + } diff --git a/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs b/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs index 58ce3954fd79..dcce2156dccc 100644 --- a/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs +++ b/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs @@ -31,7 +31,6 @@ use super::utils::is_repartition; use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::{Transformed, TreeNode, VisitRecursion}; -use datafusion_physical_expr::utils::ordering_satisfy; use std::sync::Arc; @@ -259,11 +258,10 @@ pub(crate) fn replace_with_order_preserving_variants( is_spm_better || use_order_preserving_variant, )?; // If this sort is unnecessary, we should remove it and update the plan: - if ordering_satisfy( - updated_sort_input.output_ordering(), - plan.output_ordering(), - || updated_sort_input.ordering_equivalence_properties(), - ) { + if updated_sort_input + .ordering_equivalence_properties() + .ordering_satisfy(plan.output_ordering()) + { return Ok(Transformed::Yes(OrderPreservationContext { plan: updated_sort_input, ordering_onwards: vec![None], diff --git a/datafusion/core/src/physical_optimizer/sort_pushdown.rs b/datafusion/core/src/physical_optimizer/sort_pushdown.rs index 54fdac646bbd..b5e7df6a2159 100644 --- a/datafusion/core/src/physical_optimizer/sort_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/sort_pushdown.rs @@ -32,9 +32,6 @@ use datafusion_common::tree_node::{Transformed, TreeNode, VisitRecursion}; use datafusion_common::{plan_err, DataFusionError, Result}; use datafusion_expr::JoinType; use datafusion_physical_expr::expressions::Column; -use datafusion_physical_expr::utils::{ - ordering_satisfy, ordering_satisfy_requirement, requirements_compatible, -}; use datafusion_physical_expr::{PhysicalSortExpr, PhysicalSortRequirement}; use itertools::izip; @@ -130,9 +127,10 @@ pub(crate) fn pushdown_sorts( let err = || DataFusionError::Plan(ERR_MSG.to_string()); if let Some(sort_exec) = plan.as_any().downcast_ref::() { let mut new_plan = plan.clone(); - if !ordering_satisfy_requirement(plan.output_ordering(), parent_required, || { - plan.ordering_equivalence_properties() - }) { + if !plan + .ordering_equivalence_properties() + .ordering_satisfy_requirement(parent_required) + { // If the current plan is a SortExec, modify it to satisfy parent requirements: let parent_required_expr = PhysicalSortRequirement::to_sort_exprs( parent_required.ok_or_else(err)?.iter().cloned(), @@ -160,9 +158,10 @@ pub(crate) fn pushdown_sorts( } } else { // Executors other than SortExec - if ordering_satisfy_requirement(plan.output_ordering(), parent_required, || { - plan.ordering_equivalence_properties() - }) { + if plan + .ordering_equivalence_properties() + .ordering_satisfy_requirement(parent_required) + { // Satisfies parent requirements, immediately return. return Ok(Transformed::Yes(SortPushDown { required_ordering: None, @@ -258,20 +257,24 @@ fn pushdown_requirement_to_children( || plan.as_any().is::() // Do not push-down through SortPreservingMergeExec when // ordering requirement invalidates requirement of sort preserving merge exec. - || (is_sort_preserving_merge(plan) && !ordering_satisfy( - parent_required - .map(|req| PhysicalSortRequirement::to_sort_exprs(req.to_vec())) - .as_deref(), - plan.output_ordering(), - || plan.ordering_equivalence_properties(), - ) + || (is_sort_preserving_merge(plan) && true + // !ordering_satisfy( + // parent_required + // .map(|req| PhysicalSortRequirement::to_sort_exprs(req.to_vec())) + // .as_deref(), + // plan.output_ordering(), + // || plan.ordering_equivalence_properties(), + // ) + // && plan.ordering_equivalence_properties().get_finer_ordering() ) { + println!("parent_required if:{:?}", parent_required); // If the current plan is a leaf node or can not maintain any of the input ordering, can not pushed down requirements. // For RepartitionExec, we always choose to not push down the sort requirements even the RepartitionExec(input_partition=1) could maintain input ordering. // Pushing down is not beneficial Ok(None) } else { + println!("parent_required else:{:?}", parent_required); Ok(Some( maintains_input_order .iter() @@ -297,14 +300,16 @@ fn determine_children_requirement( request_child: Option<&[PhysicalSortRequirement]>, child_plan: Arc, ) -> RequirementsCompatibility { - if requirements_compatible(request_child, parent_required, || { - child_plan.ordering_equivalence_properties() - }) { + if child_plan + .ordering_equivalence_properties() + .requirements_compatible(request_child, parent_required) + { // 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.ordering_equivalence_properties() - }) { + } else if child_plan + .ordering_equivalence_properties() + .requirements_compatible(parent_required, request_child) + { // 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) @@ -333,12 +338,21 @@ fn try_pushdown_requirements_to_join( &smj.maintains_input_order(), Some(SortMergeJoinExec::probe_side(&smj.join_type())), )?; - Ok(ordering_satisfy_requirement( - new_output_ordering.as_deref(), - parent_required, - || smj.ordering_equivalence_properties(), - ) - .then(|| { + // TODO: use hypothetical new ordering. Add support for this API. + let new_output_ordering = + PhysicalSortRequirement::from_sort_exprs(&new_output_ordering.unwrap_or(vec![])); + let finer_req = smj + .ordering_equivalence_properties() + .get_finer_requirement(&new_output_ordering, parent_required.unwrap_or(&[])); + println!( + "finer_req:{:?}, new_output_ordering:{:?}, parent_required:{:?}", + finer_req, new_output_ordering, parent_required + ); + let mut should_pushdown = false; + if let Some(finer_req) = finer_req { + should_pushdown = finer_req.eq(&new_output_ordering); + } + Ok(should_pushdown.then(|| { let required_input_ordering = smj.required_input_ordering(); let new_req = Some(PhysicalSortRequirement::from_sort_exprs(&sort_expr)); match push_side { diff --git a/datafusion/core/src/physical_optimizer/utils.rs b/datafusion/core/src/physical_optimizer/utils.rs index 435cf8fd2dc1..be97611606d0 100644 --- a/datafusion/core/src/physical_optimizer/utils.rs +++ b/datafusion/core/src/physical_optimizer/utils.rs @@ -35,7 +35,6 @@ use crate::physical_plan::windows::{BoundedWindowAggExec, WindowAggExec}; use crate::physical_plan::{displayable, ExecutionPlan}; use datafusion_common::DataFusionError; -use datafusion_physical_expr::utils::ordering_satisfy; use datafusion_physical_expr::PhysicalSortExpr; /// This object implements a tree that we use while keeping track of paths @@ -109,9 +108,10 @@ pub fn add_sort_above( fetch: Option, ) -> Result<()> { // If the ordering requirement is already satisfied, do not add a sort. - if !ordering_satisfy(node.output_ordering(), Some(&sort_expr), || { - node.ordering_equivalence_properties() - }) { + if !node + .ordering_equivalence_properties() + .ordering_satisfy(Some(&sort_expr)) + { let new_sort = SortExec::new(sort_expr, node.clone()).with_fetch(fetch); *node = Arc::new(if node.output_partitioning().partition_count() > 1 { diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index ab4c33350579..793a2730a750 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -2433,7 +2433,7 @@ mod tests { .build()?; let execution_plan = plan(&logical_plan).await?; - print_plan(&execution_plan); + // print_plan(&execution_plan); let final_hash_agg = execution_plan .as_any() .downcast_ref::() diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 4c4a29307937..0266b79fc766 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -16,185 +16,418 @@ // under the License. use crate::expressions::{CastExpr, Column}; -use crate::utils::{collect_columns, get_indices_of_matching_exprs, merge_vectors}; +use crate::utils::{collect_columns, get_indices_of_matching_exprs}; use crate::{ - LexOrdering, LexOrderingRef, LexOrderingReq, PhysicalExpr, PhysicalSortExpr, - PhysicalSortRequirement, + physical_exprs_contains, LexOrdering, LexOrderingRef, LexOrderingReq, PhysicalExpr, + PhysicalSortExpr, PhysicalSortRequirement, }; use arrow::datatypes::SchemaRef; use arrow_schema::{Fields, SortOptions}; +use crate::physical_expr::{deduplicate_physical_exprs, have_common_entries}; use crate::sort_properties::{ExprOrdering, SortProperties}; use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::utils::longest_consecutive_prefix; use datafusion_common::{JoinType, Result}; -use itertools::{enumerate, izip}; -use std::collections::{HashMap, HashSet}; +use itertools::izip; +use petgraph::visit::Walker; use std::hash::Hash; use std::ops::Range; use std::sync::Arc; -// /// Represents a collection of [`EquivalentClass`] (equivalences -// /// between columns in relations) -// /// -// /// This is used to represent: -// /// -// /// 1. Equality conditions (like `A=B`), when `T` = [`Column`] -// #[derive(Debug, Clone)] -// pub struct EquivalenceProperties { -// classes: Vec>, -// schema: SchemaRef, -// } -// -// impl EquivalenceProperties { -// pub fn new(schema: SchemaRef) -> Self { -// EquivalenceProperties { -// classes: vec![], -// schema, -// } -// } -// -// /// return the set of equivalences -// pub fn classes(&self) -> &[EquivalentClass] { -// &self.classes -// } -// -// pub fn schema(&self) -> SchemaRef { -// self.schema.clone() -// } -// -// /// Add the [`EquivalentClass`] from `iter` to this list -// pub fn extend>>(&mut self, iter: I) { -// for ec in iter { -// self.classes.push(ec) -// } -// } -// -// /// Adds new equal conditions into the EquivalenceProperties. New equal -// /// conditions usually come from equality predicates in a join/filter. -// pub fn add_equal_conditions(&mut self, new_conditions: (&Column, &Column)) { -// let mut idx1: Option = None; -// let mut idx2: Option = None; -// for (idx, class) in self.classes.iter_mut().enumerate() { -// let contains_first = class.contains(new_conditions.0); -// let contains_second = class.contains(new_conditions.1); -// match (contains_first, contains_second) { -// (true, false) => { -// class.insert(new_conditions.1.clone()); -// idx1 = Some(idx); -// } -// (false, true) => { -// class.insert(new_conditions.0.clone()); -// idx2 = Some(idx); -// } -// (true, true) => { -// idx1 = Some(idx); -// idx2 = Some(idx); -// break; -// } -// (false, false) => {} -// } -// } -// -// match (idx1, idx2) { -// (Some(idx_1), Some(idx_2)) if idx_1 != idx_2 => { -// // need to merge the two existing EquivalentClasses -// let second_eq_class = self.classes.get(idx_2).unwrap().clone(); -// let first_eq_class = self.classes.get_mut(idx_1).unwrap(); -// for prop in second_eq_class.iter() { -// if !first_eq_class.contains(prop) { -// first_eq_class.insert(prop.clone()); -// } -// } -// self.classes.remove(idx_2); -// } -// (None, None) => { -// // adding new pairs -// self.classes.push(EquivalentClass::::new( -// new_conditions.0.clone(), -// vec![new_conditions.1.clone()], -// )); -// } -// _ => {} -// } -// } -// -// /// Normalizes physical expression according to `EquivalentClass`es inside `self.classes`. -// /// expression is replaced with `EquivalentClass::head` expression if it is among `EquivalentClass::others`. -// pub fn normalize_expr(&self, expr: Arc) -> Arc { -// expr.clone() -// .transform(&|expr| { -// let normalized_form = -// expr.as_any().downcast_ref::().and_then(|column| { -// for class in &self.classes { -// if class.contains(column) { -// return Some(Arc::new(class.head().clone()) as _); -// } -// } -// None -// }); -// Ok(if let Some(normalized_form) = normalized_form { -// Transformed::Yes(normalized_form) -// } else { -// Transformed::No(expr) -// }) -// }) -// .unwrap_or(expr) -// } -// -// /// This function applies the \[`normalize_expr`] -// /// function for all expression in `exprs` and returns a vector of -// /// normalized physical expressions. -// pub fn normalize_exprs( -// &self, -// exprs: &[Arc], -// ) -> Vec> { -// exprs -// .iter() -// .map(|expr| self.normalize_expr(expr.clone())) -// .collect::>() -// } -// -// /// This function normalizes `sort_requirement` according to `EquivalenceClasses` in the `self`. -// /// If the given sort requirement doesn't belong to equivalence set inside -// /// `self`, it returns `sort_requirement` as is. -// pub fn normalize_sort_requirement( -// &self, -// mut sort_requirement: PhysicalSortRequirement, -// ) -> PhysicalSortRequirement { -// sort_requirement.expr = self.normalize_expr(sort_requirement.expr); -// sort_requirement -// } -// -// /// This function applies the \[`normalize_sort_requirement`] -// /// function for all sort requirements in `sort_reqs` and returns a vector of -// /// normalized sort expressions. -// pub fn normalize_sort_requirements( -// &self, -// sort_reqs: &[PhysicalSortRequirement], -// ) -> Vec { -// let normalized_sort_reqs = sort_reqs -// .iter() -// .map(|sort_req| self.normalize_sort_requirement(sort_req.clone())) -// .collect::>(); -// collapse_vec(normalized_sort_reqs) -// } -// -// /// Similar to the \[`normalize_sort_requirements`] this function normalizes -// /// sort expressions in `sort_exprs` and returns a vector of -// /// normalized sort expressions. -// pub fn normalize_sort_exprs( -// &self, -// sort_exprs: &[PhysicalSortExpr], -// ) -> Vec { -// let sort_requirements = -// PhysicalSortRequirement::from_sort_exprs(sort_exprs.iter()); -// let normalized_sort_requirement = -// self.normalize_sort_requirements(&sort_requirements); -// PhysicalSortRequirement::to_sort_exprs(normalized_sort_requirement) -// } -// } +/// EquivalentClass is a set of [`Arc`]s that are known +/// to have the same value in all tuples in a relation. These are generated by equality predicates, +/// typically equi-join conditions and equality conditions in filters. +#[derive(Debug, Clone)] +pub struct EquivalentGroups { + inner: Vec>>, +} + +impl EquivalentGroups { + fn empty() -> Self { + EquivalentGroups { inner: vec![] } + } + + fn new(entries: Vec>>) -> Self { + let mut res = EquivalentGroups { inner: entries }; + res.remove_redundant_entries(); + res + } + + pub fn is_empty(&self) -> bool { + self.inner.is_empty() + } + + fn iter(&self) -> impl Iterator>> { + self.inner.iter() + } + + fn into_iter(self) -> impl Iterator>> { + self.inner.into_iter() + } + + pub fn add_equal_conditions( + &mut self, + new_conditions: (&Arc, &Arc), + ) { + let (first, second) = new_conditions; + let mut first_group = None; + let mut second_group = None; + for (group_idx, eq_class) in self.inner.iter().enumerate() { + if physical_exprs_contains(eq_class, first) { + first_group = Some(group_idx); + } + if physical_exprs_contains(eq_class, second) { + second_group = Some(group_idx); + } + } + match (first_group, second_group) { + (Some(first_group_idx), Some(second_group_idx)) => { + // We should bridge these groups + if first_group_idx != second_group_idx { + let other_class = self.inner[second_group_idx].clone(); + // TODO: Use group at the lower index during merging + // This would improve performance during remove. + self.inner[first_group_idx].extend(other_class); + self.inner.remove(second_group_idx); + } + } + (Some(group_idx), None) => { + self.inner[group_idx].push(second.clone()); + } + (None, Some(group_idx)) => { + self.inner[group_idx].push(first.clone()); + } + (None, None) => { + self.inner.push(vec![first.clone(), second.clone()]); + } + } + self.remove_redundant_entries(); + } + + /// Remove redundant entries from the state. + fn remove_redundant_entries(&mut self) { + // Remove duplicate entries from each group. + self.inner = self + .inner + .iter() + .filter_map(|eq_group| { + let unique_eq_group = deduplicate_physical_exprs(eq_group); + // Keep groups that have at least 2 entries + (unique_eq_group.len() > 1).then_some(unique_eq_group) + }) + .collect(); + + self.bridge_groups() + } + + /// This utils bridges groups that have common entry + fn bridge_groups(&mut self) { + let mut out_groups = vec![]; + for group in &self.inner { + if out_groups.is_empty() { + out_groups.push(group.clone()); + } else { + let mut bridged_group = group.clone(); + // Delete groups in the `out_groups` that have common entry with `group`. + // Append deleted groups to the `bridged_group` + out_groups.retain(|distinct_group| { + let have_common = have_common_entries(distinct_group, &group); + if have_common { + bridged_group.extend(distinct_group.clone()); + } + !have_common + }); + // before inserting make sure that entries are deduplicated + let bridged_group = deduplicate_physical_exprs(&bridged_group); + out_groups.push(bridged_group); + } + } + self.inner = out_groups; + } + + // TODO: Add a method for bridging equalities + + fn len(&self) -> usize { + self.inner.len() + } + + fn extend(&mut self, other: EquivalentGroups) { + // TODO: Add check for redundancy + self.inner.extend(other.inner); + self.remove_redundant_entries(); + } + + /// Normalizes physical expression according to `EquivalentClass`es inside `self.classes`. + /// expression is replaced with `EquivalentClass::head` expression if it is among `EquivalentClass::others`. + pub fn normalize_expr(&self, expr: Arc) -> Arc { + expr.clone() + .transform(&|expr| { + for class in self.iter() { + if physical_exprs_contains(class, &expr) { + return Ok(Transformed::Yes(class[0].clone())); + } + } + Ok(Transformed::No(expr)) + }) + .unwrap_or(expr) + } + + /// This function applies the \[`normalize_expr`] + /// function for all expression in `exprs` and returns a vector of + /// normalized physical expressions. + pub fn normalize_exprs( + &self, + exprs: &[Arc], + ) -> Vec> { + let res = exprs + .iter() + .map(|expr| self.normalize_expr(expr.clone())) + .collect::>(); + // TODO: Add deduplication check here after normalization + res + } + + /// This function normalizes `sort_requirement` according to `EquivalenceClasses` in the `self`. + /// If the given sort requirement doesn't belong to equivalence set inside + /// `self`, it returns `sort_requirement` as is. + pub fn normalize_sort_requirement( + &self, + mut sort_requirement: PhysicalSortRequirement, + ) -> PhysicalSortRequirement { + sort_requirement.expr = self.normalize_expr(sort_requirement.expr); + sort_requirement + } + + /// This function normalizes `sort_requirement` according to `EquivalenceClasses` in the `self`. + /// If the given sort requirement doesn't belong to equivalence set inside + /// `self`, it returns `sort_requirement` as is. + pub fn normalize_sort_expr( + &self, + mut sort_expr: PhysicalSortExpr, + ) -> PhysicalSortExpr { + sort_expr.expr = self.normalize_expr(sort_expr.expr); + sort_expr + } + + /// This function applies the \[`normalize_sort_requirement`] + /// function for all sort requirements in `sort_reqs` and returns a vector of + /// normalized sort expressions. + pub fn normalize_sort_requirements( + &self, + sort_reqs: &[PhysicalSortRequirement], + ) -> Vec { + let normalized_sort_reqs = sort_reqs + .iter() + .map(|sort_req| self.normalize_sort_requirement(sort_req.clone())) + .collect::>(); + // TODO: Add deduplication check here after normalization + collapse_lex_req(normalized_sort_reqs) + // collapse_vec(normalized_sort_reqs) + } + + /// Similar to the \[`normalize_sort_requirements`] this function normalizes + /// sort expressions in `sort_exprs` and returns a vector of + /// normalized sort expressions. + pub fn normalize_sort_exprs( + &self, + sort_exprs: &[PhysicalSortExpr], + ) -> Vec { + // Convert `PhysicalSortExpr`s to `PhysicalSortRequirement`s + let sort_requirements = + PhysicalSortRequirement::from_sort_exprs(sort_exprs.iter()); + let normalized_sort_requirement = + self.normalize_sort_requirements(&sort_requirements); + // Convert back `PhysicalSortRequirement`s to `PhysicalSortExpr`s + PhysicalSortRequirement::to_sort_exprs(normalized_sort_requirement) + } + + /// Calculate updated version of the expression, according to projection mapping + /// returns `None`, if expression is not valid after projection. + fn get_aliased_expr( + source_to_target_mapping: &ProjectionMapping, + expr: &Arc, + ) -> Option> { + for (source, target) in source_to_target_mapping { + if expr.eq(source) { + return Some(target.clone()); + } + } + None + } + + /// Construct equivalent groups according to projection mapping + /// Each inner vector contains equivalents sets. Outer vector corresponds to + /// distinct equivalent groups + fn get_equivalent_groups( + source_to_target_mapping: &ProjectionMapping, + ) -> Vec>> { + // TODO: Convert below algorithm to the version that use HashMap. + let mut res = vec![]; + for (source, target) in source_to_target_mapping { + if res.is_empty() { + res.push((source, vec![target.clone()])); + } + if let Some(idx) = res.iter_mut().position(|(key, _values)| key.eq(source)) { + let (_, values) = &mut res[idx]; + if !physical_exprs_contains(values, target) { + values.push(target.clone()); + } + } + } + + // Filter out groups with single entry, there is nothing + // else equal to these expressions. Hence tracking them is meaningless + res.into_iter() + .filter_map(|(_key, values)| (values.len() > 1).then_some(values)) + .collect() + } + + /// Projects EquivalentGroups according to projection mapping described in `source_to_target_mapping`. + pub fn project( + &self, + source_to_target_mapping: &ProjectionMapping, + ) -> EquivalentGroups { + let mut new_eq_classes = vec![]; + for eq_class in self.iter() { + let new_eq_class = eq_class + .iter() + .filter_map(|expr| Self::get_aliased_expr(source_to_target_mapping, expr)) + .collect::>(); + if new_eq_class.len() > 1 { + new_eq_classes.push(new_eq_class.clone()); + } + } + let new_classes = Self::get_equivalent_groups(source_to_target_mapping); + new_eq_classes.extend(new_classes); + + let mut projection_eq_groups = EquivalentGroups::new(new_eq_classes); + // Make sure there is no redundant entry after projection. + projection_eq_groups.remove_redundant_entries(); + projection_eq_groups + } +} + +/// `LexOrdering` stores the lexicographical ordering for a schema. +/// OrderingEquivalentClass keeps track of different alternative orderings than can +/// describe the schema. +/// For instance, for the table below +/// |a|b|c|d| +/// |1|4|3|1| +/// |2|3|3|2| +/// |3|1|2|2| +/// |3|2|1|3| +/// both `vec![a ASC, b ASC]` and `vec![c DESC, d ASC]` describe the ordering of the table. +/// For this case, we say that `vec![a ASC, b ASC]`, and `vec![c DESC, d ASC]` are ordering equivalent. +#[derive(Debug, Clone, Eq, PartialEq, Hash)] +pub struct OrderingEquivalentGroup { + inner: Vec, +} + +impl OrderingEquivalentGroup { + fn empty() -> Self { + OrderingEquivalentGroup { inner: vec![] } + } + + pub fn new(entries: Vec) -> Self { + let mut res = OrderingEquivalentGroup { inner: entries }; + res.remove_redundant_entries(); + res + } + + pub fn contains(&self, other: &LexOrdering) -> bool { + self.inner.contains(other) + } + + fn push(&mut self, other: LexOrdering) { + if !self.contains(&other) { + self.inner.push(other); + } + // TODO: remove below check + self.remove_redundant_entries(); + } + + /// Adds new ordering into the ordering equivalent group. + pub fn add_new_orderings(&mut self, orderings: &[LexOrdering]) { + for ordering in orderings.iter() { + if !self.contains(&ordering) { + self.push(ordering.clone()); + } + } + self.remove_redundant_entries(); + } + + fn remove_redundant_entries(&mut self) { + // Make sure there is no redundant entry + let mut res: Vec = vec![]; + for ordering in self.iter() { + let mut is_inside = false; + for item in &mut res { + if let Some(finer) = get_finer(item, &ordering) { + *item = finer; + is_inside = true; + } + } + if !is_inside { + res.push(ordering.clone()); + } + } + self.inner = res; + } + + pub fn is_empty(&self) -> bool { + self.inner.is_empty() + } + + pub fn iter(&self) -> impl Iterator { + self.inner.iter() + } + + fn into_iter(self) -> impl Iterator { + self.inner.into_iter() + } + + fn len(&self) -> usize { + self.inner.len() + } + + pub fn extend(&mut self, other: OrderingEquivalentGroup) { + for ordering in other.iter() { + if !self.contains(ordering) { + self.inner.push(ordering.clone()) + } + } + self.remove_redundant_entries(); + } + + pub fn output_ordering(&self) -> Option> { + if let Some(first) = self.inner.first() { + Some(first.clone()) + } else { + None + } + } + + // Append other as postfix to existing ordering equivalences + pub fn join_postfix( + &self, + other: &OrderingEquivalentGroup, + ) -> OrderingEquivalentGroup { + if other.is_empty() { + return OrderingEquivalentGroup::new(self.inner.clone()); + } + let mut res = vec![]; + for ordering in self.iter() { + for postfix in other.iter() { + let mut new_ordering = ordering.clone(); + new_ordering.extend(postfix.clone()); + res.push(new_ordering) + } + } + OrderingEquivalentGroup::new(res) + } +} /// `OrderingEquivalenceProperties` keeps track of columns that describe the /// global ordering of the schema. These columns are not necessarily same; e.g. @@ -213,8 +446,8 @@ use std::sync::Arc; /// and treat `a ASC` and `b DESC` as the same ordering requirement. #[derive(Debug, Clone)] pub struct OrderingEquivalenceProperties { - eq_classes: Vec>>, - oeq_class: Option, + eq_groups: EquivalentGroups, + oeq_group: OrderingEquivalentGroup, /// Keeps track of expressions that have constant value. constants: Vec>, schema: SchemaRef, @@ -224,8 +457,8 @@ impl OrderingEquivalenceProperties { /// Create an empty `OrderingEquivalenceProperties` pub fn new(schema: SchemaRef) -> Self { Self { - eq_classes: vec![], - oeq_class: None, + eq_groups: EquivalentGroups::empty(), + oeq_group: OrderingEquivalentGroup::empty(), constants: vec![], schema, } @@ -233,93 +466,71 @@ impl OrderingEquivalenceProperties { /// Extends `OrderingEquivalenceProperties` by adding ordering inside the `other` /// to the `self.oeq_class`. - pub fn extend(&mut self, other: Option) { - if let Some(other) = other { - if let Some(class) = &mut self.oeq_class { - class.others.insert(other.head); - class.others.extend(other.others); - } else { - self.oeq_class = Some(other); + pub fn extend(&mut self, other: OrderingEquivalentGroup) { + for ordering in other.into_iter() { + if !self.oeq_group.contains(&ordering) { + self.oeq_group.push(ordering); } - self.normalize_state(); } + self.normalize_state(); } - pub fn oeq_class(&self) -> Option<&OrderingEquivalentClass> { - self.oeq_class.as_ref() + /// Return a reference to the ordering equivalent group + pub fn oeq_group(&self) -> &OrderingEquivalentGroup { + &self.oeq_group } - pub fn eq_classes(&self) -> &[Vec>] { - &self.eq_classes + /// Return a reference to the equivalent groups + pub fn eq_groups(&self) -> &EquivalentGroups { + &self.eq_groups } - /// Adds new equal conditions into the EquivalenceProperties. New equal - /// conditions usually come from equality predicates in a join/filter. - pub fn add_ordering_equal_conditions( - &mut self, - new_conditions: (&LexOrdering, &LexOrdering), - ) { - if let Some(class) = &mut self.oeq_class { - class.insert(new_conditions.0.clone()); - class.insert(new_conditions.1.clone()); - } else { - let head = new_conditions.0.clone(); - let others = vec![new_conditions.1.clone()]; - self.oeq_class = Some(OrderingEquivalentClass::new(head, others)) - } + /// Adds new ordering into the ordering equivalent group. + pub fn add_new_orderings(&mut self, orderings: &[LexOrdering]) { + self.oeq_group.add_new_orderings(orderings); + self.normalize_state(); } + /// Adds new equality group into the equivalent groups. + /// If equalities are new, otherwise extends corresponding group. pub fn add_equal_conditions( &mut self, new_conditions: (&Arc, &Arc), ) { - let (first, second) = new_conditions; - let mut first_group = None; - let mut second_group = None; - for (group_idx, eq_class) in self.eq_classes.iter().enumerate() { - if physical_exprs_contains(eq_class, first) { - first_group = Some(group_idx); - } - if physical_exprs_contains(eq_class, second) { - second_group = Some(group_idx); - } - } - match (first_group, second_group) { - (Some(first_group_idx), Some(second_group_idx)) => { - // We should bridge these groups - if first_group_idx != second_group_idx { - let other_class = self.eq_classes[second_group_idx].clone(); - self.eq_classes[first_group_idx].extend(other_class); - self.eq_classes.remove(second_group_idx); - } - } - (Some(group_idx), None) => { - self.eq_classes[group_idx].push(second.clone()); - } - (None, Some(group_idx)) => { - self.eq_classes[group_idx].push(first.clone()); - } - (None, None) => { - self.eq_classes.push(vec![first.clone(), second.clone()]); - } - } + self.eq_groups.add_equal_conditions(new_conditions); self.normalize_state(); } /// Normalizes state according to equivalent classes fn normalize_state(&mut self) { - let mut new_oeq_class = self.oeq_class.clone(); - if let Some(oeq_class) = &mut new_oeq_class { - // println!("oeq_class: {:?}", oeq_class); - oeq_class.head = self.normalize_sort_exprs(&oeq_class.head); - oeq_class.others = oeq_class - .others - .iter() - .map(|item| self.normalize_sort_exprs(item)) - .collect(); - // println!("oeq_class: {:?}", oeq_class); + let normalized_ordering = self + .oeq_group + .inner + .iter() + .map(|ordering| { + let ordering = self.eq_groups.normalize_sort_exprs(&ordering); + let req = prune_sort_reqs_with_constants( + &PhysicalSortRequirement::from_sort_exprs(&ordering), + &self.constants, + ); + PhysicalSortRequirement::to_sort_exprs(req) + }) + .collect::>>(); + let mut res: Vec = vec![]; + for ordering in normalized_ordering.into_iter() { + let mut is_inside = false; + for item in &mut res { + if let Some(finer) = get_finer(item, &ordering) { + *item = finer; + is_inside = true; + } + } + if !is_inside { + res.push(ordering); + } } - self.oeq_class = new_oeq_class; + self.oeq_group = OrderingEquivalentGroup::new(res); + // TODO: Add redundant entry check } /// Add physical expression that have constant value to the `self.constants` @@ -329,6 +540,7 @@ impl OrderingEquivalenceProperties { self.constants.push(constant); } }); + self.normalize_state(); self } @@ -336,70 +548,88 @@ impl OrderingEquivalenceProperties { self.schema.clone() } - fn normalize_with_eq_classes( + /// Transform `sort_exprs` vector, to standardized version using `eq_properties` and `ordering_eq_properties` + /// Assume `eq_properties` states that `Column a` and `Column b` are aliases. + /// Also assume `ordering_eq_properties` states that ordering `vec![d ASC]` and `vec![a ASC, c ASC]` are + /// ordering equivalent (in the sense that both describe the ordering of the table). + /// If the `sort_exprs` input to this function were `vec![b ASC, c ASC]`, + /// This function converts `sort_exprs` `vec![b ASC, c ASC]` to first `vec![a ASC, c ASC]` after considering `eq_properties` + /// Then converts `vec![a ASC, c ASC]` to `vec![d ASC]` after considering `ordering_eq_properties`. + /// Standardized version `vec![d ASC]` is used in subsequent operations. + pub fn normalize_sort_exprs( &self, - sort_reqs: &[PhysicalSortRequirement], - ) -> Vec { - // println!("sort_reqs: {:?}", sort_reqs); - // println!("self.eq_classes: {:?}", self.eq_classes); - let normalized_sort_reqs = sort_reqs - .iter() - .map(|sort_req| { - for eq_class in &self.eq_classes { - if physical_exprs_contains(eq_class, &sort_req.expr) { - return PhysicalSortRequirement { - expr: eq_class[0].clone(), - options: sort_req.options, - }; - } - } - sort_req.clone() - }) - .collect::>(); - let normalized_sort_reqs = - prune_sort_reqs_with_constants(&normalized_sort_reqs, &self.constants); - normalized_sort_reqs + sort_exprs: &[PhysicalSortExpr], + ) -> Vec { + // Convert `PhysicalSortExpr`s to `PhysicalSortRequirement`s + let sort_requirements = + PhysicalSortRequirement::from_sort_exprs(sort_exprs.iter()); + let normalized_exprs = self.normalize_sort_requirements(&sort_requirements); + // Convert back `PhysicalSortRequirement`s to `PhysicalSortExpr`s + PhysicalSortRequirement::to_sort_exprs(normalized_exprs) } /// This function normalizes `sort_reqs` by /// - removing expressions that have constant value from requirement /// - replacing sections that are in the `self.oeq_class.others` with `self.oeq_class.head` /// - removing sections that satisfies global ordering that are in the post fix of requirement + /// + /// Transform `sort_reqs` vector, to standardized version using `eq_properties` and `ordering_eq_properties` + /// Assume `eq_properties` states that `Column a` and `Column b` are aliases. + /// Also assume `ordering_eq_properties` states that ordering `vec![d ASC]` and `vec![a ASC, c ASC]` are + /// ordering equivalent (in the sense that both describe the ordering of the table). + /// If the `sort_reqs` input to this function were `vec![b Some(ASC), c None]`, + /// This function converts `sort_exprs` `vec![b Some(ASC), c None]` to first `vec![a Some(ASC), c None]` after considering `eq_properties` + /// Then converts `vec![a Some(ASC), c None]` to `vec![d Some(ASC)]` after considering `ordering_eq_properties`. + /// Standardized version `vec![d Some(ASC)]` is used in subsequent operations. pub fn normalize_sort_requirements( &self, sort_reqs: &[PhysicalSortRequirement], ) -> Vec { - let normalized_sort_reqs = self.normalize_with_eq_classes(sort_reqs); + // println!("sort_reqs: {:?}", sort_reqs); + // println!("self.oeq_class: {:?}", self.oeq_class); + let normalized_sort_reqs = self.eq_groups.normalize_sort_requirements(sort_reqs); + let normalized_sort_reqs = + prune_sort_reqs_with_constants(&normalized_sort_reqs, &self.constants); let mut normalized_sort_reqs = collapse_lex_req(normalized_sort_reqs); - if let Some(oeq_class) = &self.oeq_class { - for item in oeq_class.others() { - let item = PhysicalSortRequirement::from_sort_exprs(item); - let item = self.normalize_with_eq_classes(&item); - let item = prune_sort_reqs_with_constants(&item, &self.constants); - let ranges = get_compatible_ranges(&normalized_sort_reqs, &item); - let mut offset: i64 = 0; - for Range { start, end } in ranges { - let head = PhysicalSortRequirement::from_sort_exprs(oeq_class.head()); - let head = self.normalize_with_eq_classes(&head); - let mut head = prune_sort_reqs_with_constants(&head, &self.constants); - let updated_start = (start as i64 + offset) as usize; - let updated_end = (end as i64 + offset) as usize; - let range = end - start; - offset += head.len() as i64 - range as i64; - let all_none = normalized_sort_reqs[updated_start..updated_end] - .iter() - .all(|req| req.options.is_none()); - if all_none { - for req in head.iter_mut() { - req.options = None; - } + if self.oeq_group.is_empty() { + return normalized_sort_reqs; + } + let first_entry = + PhysicalSortRequirement::from_sort_exprs(&self.oeq_group.inner[0]); + let first_entry = self.eq_groups.normalize_sort_requirements(&first_entry); + let first_entry = prune_sort_reqs_with_constants(&first_entry, &self.constants); + for item in self.oeq_group.iter() { + let item = PhysicalSortRequirement::from_sort_exprs(item); + let item = self.eq_groups.normalize_sort_requirements(&item); + let item = prune_sort_reqs_with_constants(&item, &self.constants); + let ranges = get_compatible_ranges(&normalized_sort_reqs, &item); + let mut offset: i64 = 0; + for Range { start, end } in ranges { + // let head = PhysicalSortRequirement::from_sort_exprs(&self.oeq_class[0]); + // let head = self.normalize_with_eq_classes(&head); + // let mut head = prune_sort_reqs_with_constants(&head, &self.constants); + let mut head = + prune_sort_reqs_with_constants(&first_entry, &self.constants); + let updated_start = (start as i64 + offset) as usize; + let updated_end = (end as i64 + offset) as usize; + let range = end - start; + offset += head.len() as i64 - range as i64; + let all_none = normalized_sort_reqs[updated_start..updated_end] + .iter() + .all(|req| req.options.is_none()); + if all_none { + for req in head.iter_mut() { + req.options = None; } - normalized_sort_reqs.splice(updated_start..updated_end, head); } + normalized_sort_reqs.splice(updated_start..updated_end, head); } - normalized_sort_reqs = simplify_lex_req(normalized_sort_reqs, oeq_class); } - collapse_lex_req(normalized_sort_reqs) + normalized_sort_reqs = simplify_lex_req(normalized_sort_reqs, &self.oeq_group); + + let res = collapse_lex_req(normalized_sort_reqs); + // println!("normalzied sort_reqs:{:?}", res); + res } /// Checks whether `leading_ordering` is contained in any of the ordering @@ -408,94 +638,16 @@ impl OrderingEquivalenceProperties { &self, leading_ordering: &PhysicalSortExpr, ) -> bool { - if let Some(oeq_class) = &self.oeq_class { - for ordering in oeq_class - .others - .iter() - .chain(std::iter::once(&oeq_class.head)) - { - if ordering[0].eq(leading_ordering) { - return true; - } - } - } - false - } - - /// Normalizes physical expression according to `EquivalentClass`es inside `self.classes`. - /// expression is replaced with `EquivalentClass::head` expression if it is among `EquivalentClass::others`. - pub fn normalize_expr(&self, expr: Arc) -> Arc { - expr.clone() - .transform(&|expr| { - for class in self.eq_classes() { - if physical_exprs_contains(class, &expr) { - return Ok(Transformed::Yes(class[0].clone())); - } - } - Ok(Transformed::No(expr)) - }) - .unwrap_or(expr) - } - - pub fn normalize_sort_expr(&self, sort_expr: PhysicalSortExpr) -> PhysicalSortExpr { - let PhysicalSortExpr { expr, options } = sort_expr; - let new_expr = self.normalize_expr(expr); - PhysicalSortExpr { - expr: new_expr, - options, - } - } - - pub fn normalize_exprs( - &self, - exprs: &[Arc], - ) -> Vec> { - let res = exprs - .iter() - .map(|expr| self.normalize_expr(expr.clone())) - .collect::>(); - // TODO: Add deduplication check here after normalization - res - } - - pub fn normalize_sort_exprs( - &self, - sort_exprs: &[PhysicalSortExpr], - ) -> Vec { - let res = sort_exprs + self.oeq_group() .iter() - .map(|sort_expr| self.normalize_sort_expr(sort_expr.clone())) - .collect::>(); - // TODO: Add deduplication check here after normalization - res - } - - fn get_aliased_expr( - alias_map: &HashMap>, - source_to_target_mapping: &[(Arc, Arc)], - expr: &Arc, - ) -> Option> { - for (source, target) in source_to_target_mapping { - if expr.eq(source) { - return Some(target.clone()); - } - } - // for (column, columns) in alias_map { - // let column_expr = Arc::new(column.clone()) as Arc; - // // println!("column_expr:{:?}, expr:{:?}",column_expr, expr); - // if column_expr.eq(expr) { - // // println!("return some"); - // return Some(Arc::new(columns[0].clone())); - // } - // } - None + .any(|ordering| ordering[0].eq(leading_ordering)) } fn get_eq_class_group( - old_eq_class: &[Vec>], + old_eq_class: &EquivalentGroups, expr: &Arc, ) -> Option>> { - for eq_class in old_eq_class { + for eq_class in old_eq_class.iter() { if physical_exprs_contains(eq_class, expr) { return Some(eq_class.to_vec()); } @@ -504,24 +656,10 @@ impl OrderingEquivalenceProperties { } fn get_corresponding_expr( - old_eq_class: &[Vec>], - new_eq_class: &[Option>>], - alias_map: &HashMap>, - source_to_target_mapping: &[(Arc, Arc)], + old_eq_class: &EquivalentGroups, + source_to_target_mapping: &ProjectionMapping, expr: &Arc, ) -> Option> { - assert_eq!(old_eq_class.len(), new_eq_class.len()); - // for (old, new) in izip!(old_eq_class.iter(), new_eq_class.iter()) { - // if let (true, Some(new_eq)) = (physical_exprs_contains(old, expr), new) { - // return Some(new_eq[0].clone()); - // } - // } - // for (column, columns) in alias_map { - // let column_expr = Arc::new(column.clone()) as Arc; - // if column_expr.eq(expr) { - // return Some(Arc::new(columns[0].clone())); - // } - // } let children = expr.children(); if children.is_empty() { for (source, target) in source_to_target_mapping.iter() { @@ -544,8 +682,6 @@ impl OrderingEquivalenceProperties { .map(|child| { Self::get_corresponding_expr( old_eq_class, - new_eq_class, - alias_map, source_to_target_mapping, &child, ) @@ -556,37 +692,20 @@ impl OrderingEquivalenceProperties { } else { None } - - // for (source, target) in source_to_target_mapping.iter(){ - // - // if source.eq(expr) || old_eq_class.iter().any(|eq_class| eq_class.iter().any(|item| item.eq(source))){ - // return Some(target.clone()) - // } else if let Some(group) = Self::get_eq_class_group(old_eq_class, source){ - // if physical_exprs_contains(&group, expr){ - // return Some(target.clone()) - // } - // } - // } - // None } fn get_projected_ordering( - old_eq_class: &[Vec>], - new_eq_class: &[Option>>], - alias_map: &HashMap>, - source_to_target_mapping: &[(Arc, Arc)], + old_eq_class: &EquivalentGroups, + source_to_target_mapping: &ProjectionMapping, ordering: &[PhysicalSortExpr], ) -> Option> { // println!("old_eq_class: {:?}", old_eq_class); - // println!("new_eq_class: {:?}", new_eq_class); // println!("ordering: {:?}", ordering); let mut res = vec![]; for order in ordering { // println!("order.expr:{:?}", order.expr); if let Some(new_expr) = Self::get_corresponding_expr( old_eq_class, - new_eq_class, - alias_map, source_to_target_mapping, &order.expr, ) { @@ -606,92 +725,34 @@ impl OrderingEquivalenceProperties { } } - fn get_equivalent_groups( - source_to_target_mapping: &[(Arc, Arc)], - ) -> Vec>> { - let mut res = vec![]; - for (source, target) in source_to_target_mapping { - if res.is_empty() { - res.push((source, vec![target.clone()])); - } - if let Some(idx) = res.iter_mut().position(|(key, _values)| key.eq(source)) { - let (_, values) = &mut res[idx]; - if !physical_exprs_contains(values, target) { - values.push(target.clone()); - } - } - } - res.into_iter() - .filter_map( - |(_, values)| { - if values.len() > 1 { - Some(values) - } else { - None - } - }, - ) - .collect() - // vec![] - } - pub fn project( &self, - alias_map: &HashMap>, - source_to_target_mapping: &[(Arc, Arc)], + source_to_target_mapping: &ProjectionMapping, output_schema: SchemaRef, ) -> OrderingEquivalenceProperties { - // println!("alias_map: {:?}", alias_map); // println!("source_to_target_mapping: {:?}", source_to_target_mapping); - let mut res = OrderingEquivalenceProperties::new(output_schema); + let mut projected_properties = OrderingEquivalenceProperties::new(output_schema); - let mut new_eq_classes = vec![]; - let mut new_eq_classes2 = vec![]; - for eq_class in &self.eq_classes { - let new_eq_class = eq_class - .iter() - .filter_map(|expr| { - Self::get_aliased_expr(alias_map, source_to_target_mapping, expr) - }) - .collect::>(); - // println!("new_eq_class:{:?}", new_eq_class); - if new_eq_class.len() > 1 { - new_eq_classes.push(new_eq_class.clone()); - } - if new_eq_class.is_empty() { - new_eq_classes2.push(None); - } else { - new_eq_classes2.push(Some(new_eq_class)); - } - } - let new_classes = Self::get_equivalent_groups(source_to_target_mapping); - // println!("new_classes alias group:{:?}", new_classes); - // TODO: Add check for redundant group - // combine groups with common entries - new_eq_classes.extend(new_classes); - res.eq_classes = new_eq_classes; + let projected_eq_groups = self.eq_groups.project(source_to_target_mapping); + projected_properties.eq_groups = projected_eq_groups; - if let Some(oeq_class) = &self.oeq_class { - // println!("old oeq class: {:?}", oeq_class); - let new_ordering = oeq_class - .iter() - .filter_map(|order| { - Self::get_projected_ordering( - &self.eq_classes, - &new_eq_classes2, - alias_map, - source_to_target_mapping, - order, - ) - }) - .collect::>(); - // println!("new_ordering: {:?}", new_ordering); - if !new_ordering.is_empty() { - let head = new_ordering[0].clone(); - let others = new_ordering[1..].to_vec(); - res.oeq_class = Some(OrderingEquivalentClass::new(head, others)); - } + // println!("old oeq class: {:?}", oeq_class); + let new_ordering = self + .oeq_group + .iter() + .filter_map(|order| { + Self::get_projected_ordering( + &self.eq_groups, + source_to_target_mapping, + order, + ) + }) + .collect::>(); + // println!("new_ordering: {:?}", new_ordering); + if !new_ordering.is_empty() { + projected_properties.oeq_group = OrderingEquivalentGroup::new(new_ordering); } + for (source, target) in source_to_target_mapping { let initial_expr = ExprOrdering::new(source.clone()); let transformed = initial_expr @@ -702,19 +763,17 @@ impl OrderingEquivalenceProperties { expr: target.clone(), options: sort_options, }; - if let Some(oeq_class) = &mut res.oeq_class { - // println!("oeq_class before: {:?}", oeq_class); - oeq_class.add_new_ordering(&[sort_expr]); - // println!("oeq_class after: {:?}", oeq_class); - // oeq_class.others.insert(vec![sort_expr]); - } else { - let head = vec![sort_expr]; - res.oeq_class = Some(OrderingEquivalentClass::new(head, vec![])); + if !projected_properties + .oeq_group + .contains(&vec![sort_expr.clone()]) + { + projected_properties.oeq_group.push(vec![sort_expr]); } } } + // TODO: Add redundancy check, for ordering equivalences - res + projected_properties } pub fn with_reorder( @@ -723,7 +782,18 @@ impl OrderingEquivalenceProperties { ) -> OrderingEquivalenceProperties { // TODO: In some cases, existing ordering equivalences may still be valid add this analysis // Equivalences and constants are still valid after reorder - self.oeq_class = Some(OrderingEquivalentClass::new(sort_expr, vec![])); + let sort_expr = sort_expr + .into_iter() + .map(|PhysicalSortExpr { expr, options }| { + let new_expr = self.eq_groups.normalize_expr(expr); + PhysicalSortExpr { + expr: new_expr, + options, + } + }) + .collect::>(); + // TODO: Add deduplicate + self.oeq_group = OrderingEquivalentGroup::new(vec![sort_expr]); self } @@ -731,49 +801,49 @@ impl OrderingEquivalenceProperties { &self, exprs: &[Arc], ) -> Option> { - let exprs_normalized = self.normalize_exprs(exprs); + let exprs_normalized = self.eq_groups.normalize_exprs(exprs); // println!("exprs: {:?}", exprs); // println!("exprs_normalized: {:?}", exprs_normalized); // println!("self.eq_classes: {:?}", self.eq_classes); // println!("self.oeq_class: {:?}", self.oeq_class); let mut best = vec![]; - if let Some(oeq_class) = &self.oeq_class { - for ordering in oeq_class.iter() { - let ordering = self.normalize_sort_exprs(ordering); - let ordering_exprs = ordering + + for ordering in self.oeq_group.iter() { + let ordering = self.eq_groups.normalize_sort_exprs(ordering); + let ordering_exprs = ordering + .iter() + .map(|sort_expr| sort_expr.expr.clone()) + .collect::>(); + // let ordering_exprs = self.normalize_exprs(&ordering); + // println!("exprs_normalized: {:?}, normalized_ordering_exprs:{:?}", exprs_normalized, ordering_exprs); + let mut ordered_indices = + get_indices_of_matching_exprs(&exprs_normalized, &ordering_exprs); + // println!("ordered_indices: {:?}", ordered_indices); + ordered_indices.sort(); + // Find out how many expressions of the existing ordering define ordering + // for expressions in the GROUP BY clause. For example, if the input is + // ordered by a, b, c, d and we group by b, a, d; the result below would be. + // 2, meaning 2 elements (a, b) among the GROUP BY columns define ordering. + let first_n = longest_consecutive_prefix(ordered_indices); + if first_n > best.len() { + let ordered_exprs = ordering_exprs[0..first_n].to_vec(); + // Find indices for the GROUP BY expressions such that when we iterate with + // these indices, we would match existing ordering. For the example above, + // this would produce 1, 0; meaning 1st and 0th entries (a, b) among the + // GROUP BY expressions b, a, d match input ordering. + let indices = + get_indices_of_matching_exprs(&ordered_exprs, &exprs_normalized); + // println!("indices:{:?}, ordered_exprs: {:?}, exprs_normalized:{:?}", indices, ordered_exprs, exprs_normalized); + best = indices .iter() - .map(|sort_expr| sort_expr.expr.clone()) - .collect::>(); - // let ordering_exprs = self.normalize_exprs(&ordering); - // println!("exprs_normalized: {:?}, normalized_ordering_exprs:{:?}", exprs_normalized, ordering_exprs); - let mut ordered_indices = - get_indices_of_matching_exprs(&exprs_normalized, &ordering_exprs); - // println!("ordered_indices: {:?}", ordered_indices); - ordered_indices.sort(); - // Find out how many expressions of the existing ordering define ordering - // for expressions in the GROUP BY clause. For example, if the input is - // ordered by a, b, c, d and we group by b, a, d; the result below would be. - // 2, meaning 2 elements (a, b) among the GROUP BY columns define ordering. - let first_n = longest_consecutive_prefix(ordered_indices); - if first_n > best.len() { - let ordered_exprs = ordering_exprs[0..first_n].to_vec(); - // Find indices for the GROUP BY expressions such that when we iterate with - // these indices, we would match existing ordering. For the example above, - // this would produce 1, 0; meaning 1st and 0th entries (a, b) among the - // GROUP BY expressions b, a, d match input ordering. - let indices = - get_indices_of_matching_exprs(&ordered_exprs, &exprs_normalized); - // println!("indices:{:?}, ordered_exprs: {:?}, exprs_normalized:{:?}", indices, ordered_exprs, exprs_normalized); - best = indices - .iter() - .enumerate() - .map(|(order_idx, &match_idx)| { - (match_idx, ordering[order_idx].options) - }) - .collect(); - } + .enumerate() + .map(|(order_idx, &match_idx)| { + (match_idx, ordering[order_idx].options) + }) + .collect(); } } + if best.is_empty() { None } else { @@ -782,91 +852,166 @@ impl OrderingEquivalenceProperties { } pub fn with_empty_ordering_equivalence(mut self) -> OrderingEquivalenceProperties { - self.oeq_class = None; + self.oeq_group = OrderingEquivalentGroup::empty(); self } -} -/// EquivalentClass is a set of [`Column`]s or [`PhysicalSortExpr`]s that are known -/// to have the same value in all tuples in a relation. `EquivalentClass` -/// is generated by equality predicates, typically equijoin conditions and equality -/// conditions in filters. `EquivalentClass` is generated by the -/// `ROW_NUMBER` window function. -#[derive(Debug, Clone)] -pub struct EquivalentClass { - /// First element in the EquivalentClass - head: T, - /// Other equal columns - others: HashSet, -} + /// Checks whether given ordering requirements are satisfied by provided [PhysicalSortExpr]s. + pub fn ordering_satisfy(&self, required: Option<&[PhysicalSortExpr]>) -> bool { + match required { + None => true, + Some(required) => self.ordering_satisfy_concrete(required), + } + } -impl EquivalentClass { - pub fn new(head: T, others: Vec) -> EquivalentClass { - EquivalentClass { - head, - others: HashSet::from_iter(others), + /// Checks whether the required [`PhysicalSortExpr`]s are satisfied by the + /// any of the existing orderings. + pub fn ordering_satisfy_concrete(&self, required: &[PhysicalSortExpr]) -> bool { + let required_normalized = self.normalize_sort_exprs(required); + let provided_normalized = self.oeq_group().output_ordering().unwrap_or(vec![]); + // println!("required: {:?}", required); + // println!("self.oeq_class():{:?}", self.oeq_class()); + // println!("required_normalized:{:?}", required_normalized); + // println!("provided_normalized:{:?}", provided_normalized); + if required_normalized.len() > provided_normalized.len() { + return false; } + let res = required_normalized + .into_iter() + .zip(provided_normalized) + .all(|(req, given)| given == req); + // println!("res:{:?}", res); + res } - pub fn head(&self) -> &T { - &self.head + /// Find the finer requirement among `req1` and `req2` + /// If `None`, this means that `req1` and `req2` are not compatible + /// e.g there is no requirement that satisfies both + pub fn get_finer_ordering<'a>( + &self, + req1: &'a [PhysicalSortExpr], + req2: &'a [PhysicalSortExpr], + ) -> Option<&'a [PhysicalSortExpr]> { + let lhs = self.normalize_sort_exprs(req1); + let rhs = self.normalize_sort_exprs(req2); + if izip!(lhs.iter(), rhs.iter()).all(|(lhs, rhs)| lhs.eq(rhs)) { + if lhs.len() > rhs.len() { + return Some(req1); + } else { + return Some(req2); + } + } + // Neither `provided` nor `req` satisfies one another, they are incompatible. + None } - pub fn others(&self) -> &HashSet { - &self.others + /// Find the finer requirement among `req1` and `req2` + /// If `None`, this means that `req1` and `req2` are not compatible + /// e.g there is no requirement that satisfies both + pub fn get_finer_requirement<'a>( + &self, + req1: &'a [PhysicalSortRequirement], + req2: &'a [PhysicalSortRequirement], + ) -> Option<&'a [PhysicalSortRequirement]> { + let lhs = self.normalize_sort_requirements(req1); + let rhs = self.normalize_sort_requirements(req2); + let mut left_finer = false; + let mut right_finer = false; + if izip!(lhs.iter(), rhs.iter()).all(|(lhs, rhs)| { + match (lhs.options, rhs.options) { + (Some(lhs_opt), Some(rhs_opt)) => { + lhs.expr.eq(&rhs.expr) && lhs_opt == rhs_opt + } + (Some(_), None) => { + left_finer = true; + lhs.expr.eq(&rhs.expr) + } + (None, Some(_)) => { + right_finer = true; + lhs.expr.eq(&rhs.expr) + } + (None, None) => lhs.expr.eq(&rhs.expr), + } + }) { + if lhs.len() >= rhs.len() && !right_finer { + return Some(req1); + } else if rhs.len() >= lhs.len() && !left_finer { + return Some(req2); + } + } + // Neither `provided` nor `req` satisfies one another, they are incompatible. + None } - pub fn contains(&self, col: &T) -> bool { - self.head == *col || self.others.contains(col) + /// Checks whether the given [`PhysicalSortRequirement`]s are satisfied by the + /// provided [`PhysicalSortExpr`]s. + pub fn ordering_satisfy_requirement( + &self, + // provided: Option<&[PhysicalSortExpr]>, + required: Option<&[PhysicalSortRequirement]>, + ) -> bool { + match required { + None => true, + Some(required) => self.ordering_satisfy_requirement_concrete(required), + } } - pub fn insert(&mut self, col: T) -> bool { - self.head != col && self.others.insert(col) + /// Checks whether the given [`PhysicalSortRequirement`]s are satisfied by the + /// provided [`PhysicalSortExpr`]s. + pub fn ordering_satisfy_requirement_concrete( + &self, + // provided: &[PhysicalSortExpr], + required: &[PhysicalSortRequirement], + ) -> bool { + // let eq_properties = equal_properties(); + let provided_normalized = self.oeq_group().output_ordering().unwrap_or(vec![]); + let required_normalized = self.normalize_sort_requirements(required); + // let provided_normalized = self.normalize_sort_exprs2(provided); + if required_normalized.len() > provided_normalized.len() { + return false; + } + required_normalized + .into_iter() + .zip(provided_normalized) + .all(|(req, given)| given.satisfy(&req)) } - pub fn remove(&mut self, col: &T) -> bool { - let removed = self.others.remove(col); - // If we are removing the head, adjust others so that its first entry becomes the new head. - if !removed && *col == self.head { - if let Some(col) = self.others.iter().next().cloned() { - let removed = self.others.remove(&col); - self.head = col; - removed - } else { - // We don't allow empty equivalence classes, reject removal if one tries removing - // the only element in an equivalence class. - false + /// Checks whether the given [`PhysicalSortRequirement`]s are equal or more + /// specific than the provided [`PhysicalSortRequirement`]s. + pub fn requirements_compatible( + &self, + provided: Option<&[PhysicalSortRequirement]>, + required: Option<&[PhysicalSortRequirement]>, + ) -> bool { + match (provided, required) { + (_, None) => true, + (None, Some(_)) => false, + (Some(provided), Some(required)) => { + self.requirements_compatible_concrete(provided, required) } - } else { - removed } } - pub fn iter(&self) -> impl Iterator { - std::iter::once(&self.head).chain(self.others.iter()) - } - - pub fn len(&self) -> usize { - self.others.len() + 1 - } - - pub fn is_empty(&self) -> bool { - self.len() == 0 + /// Checks whether the given [`PhysicalSortRequirement`]s are equal or more + /// specific than the provided [`PhysicalSortRequirement`]s. + fn requirements_compatible_concrete( + &self, + provided: &[PhysicalSortRequirement], + required: &[PhysicalSortRequirement], + ) -> bool { + let required_normalized = self.normalize_sort_requirements(required); + let provided_normalized = self.normalize_sort_requirements(provided); + if required_normalized.len() > provided_normalized.len() { + return false; + } + required_normalized + .into_iter() + .zip(provided_normalized) + .all(|(req, given)| given.compatible(&req)) } } -/// `LexOrdering` stores the lexicographical ordering for a schema. -/// OrderingEquivalentClass keeps track of different alternative orderings than can -/// describe the schema. -/// For instance, for the table below -/// |a|b|c|d| -/// |1|4|3|1| -/// |2|3|3|2| -/// |3|1|2|2| -/// |3|2|1|3| -/// both `vec![a ASC, b ASC]` and `vec![c DESC, d ASC]` describe the ordering of the table. -/// For this case, we say that `vec![a ASC, b ASC]`, and `vec![c DESC, d ASC]` are ordering equivalent. -pub type OrderingEquivalentClass = EquivalentClass; +type ProjectionMapping = Vec<(Arc, Arc)>; /// Update each expression in `ordering` with alias expressions. Assume /// `ordering` is `a ASC, b ASC` and `c` is alias of `b`. Then, the result @@ -889,119 +1034,171 @@ fn update_with_alias( ordering } -impl OrderingEquivalentClass { - /// This function updates ordering equivalences with alias information. - /// For instance, assume columns `a` and `b` are aliases (a as b), and - /// orderings `a ASC` and `c DESC` are equivalent. Here, we replace column - /// `a` with `b` in ordering equivalence expressions. After this function, - /// `a ASC`, `c DESC` will be converted to the `b ASC`, `c DESC`. - fn update_with_aliases( - &mut self, - oeq_alias_map: &[(Column, Column)], - fields: &Fields, - ) { - let is_head_invalid = self.head.iter().any(|sort_expr| { - collect_columns(&sort_expr.expr) - .iter() - .any(|col| is_column_invalid_in_new_schema(col, fields)) - }); - // If head is invalidated, update head with alias expressions - if is_head_invalid { - self.head = update_with_alias(self.head.clone(), oeq_alias_map); - } else { - let new_oeq_expr = update_with_alias(self.head.clone(), oeq_alias_map); - self.insert(new_oeq_expr); - } - for ordering in self.others.clone().into_iter() { - self.insert(update_with_alias(ordering, oeq_alias_map)); - } - } - - /// Adds `offset` value to the index of each expression inside `self.head` and `self.others`. - pub fn add_offset(&self, offset: usize) -> Result { - let head = add_offset_to_lex_ordering(self.head(), offset)?; - let others = self - .others() - .iter() - .map(|ordering| add_offset_to_lex_ordering(ordering, offset)) - .collect::>>()?; - Ok(OrderingEquivalentClass::new(head, others)) - } - - // /// This function normalizes `OrderingEquivalenceProperties` according to `eq_properties`. - // /// More explicitly, it makes sure that expressions in `oeq_class` are head entries - // /// in `eq_properties`, replacing any non-head entries with head entries if necessary. - // pub fn normalize_with_equivalence_properties( - // &self, - // eq_properties: &EquivalenceProperties, - // ) -> OrderingEquivalentClass { - // let head = eq_properties.normalize_sort_exprs(self.head()); - // - // let others = self - // .others() - // .iter() - // .map(|other| eq_properties.normalize_sort_exprs(other)) - // .collect(); - // - // EquivalentClass::new(head, others) - // } - - /// Prefix with existing ordering. - pub fn prefix_ordering_equivalent_class_with_existing_ordering( - &self, - existing_ordering: &[PhysicalSortExpr], - ) -> OrderingEquivalentClass { - // let existing_ordering = eq_properties.normalize_sort_exprs(existing_ordering); - // let normalized_head = eq_properties.normalize_sort_exprs(self.head()); - let normalized_head = self.head(); - let updated_head = merge_vectors(&existing_ordering, &normalized_head); - let updated_others = self - .others() - .iter() - .map(|ordering| { - // let normalized_ordering = eq_properties.normalize_sort_exprs(ordering); - let normalized_ordering = ordering; - merge_vectors(&existing_ordering, &normalized_ordering) - }) - .collect(); - OrderingEquivalentClass::new(updated_head, updated_others) - } - - fn get_finer( - lhs: &[PhysicalSortExpr], - rhs: &[PhysicalSortExpr], - ) -> Option> { - if izip!(lhs.iter(), rhs.iter()).all(|(lhs, rhs)| lhs.eq(rhs)) { - if lhs.len() > rhs.len() { - return Some(lhs.to_vec()); +fn update_with_aliases( + in_data: &OrderingEquivalentGroup, + oeq_alias_map: &[(Column, Column)], + fields: &Fields, +) -> OrderingEquivalentGroup { + let new_data = in_data + .iter() + .filter_map(|ordering| { + let new_ordering = update_with_alias(ordering.clone(), oeq_alias_map); + let is_invalid = new_ordering.iter().any(|sort_expr| { + // If any one of the columns, used in Expression is invalid after projection, + // remove expression from ordering equivalences + collect_columns(&sort_expr.expr) + .iter() + .any(|col| is_column_invalid_in_new_schema(col, fields)) + }); + if is_invalid { + None } else { - return Some(rhs.to_vec()); + Some(new_ordering) } - } - None - } + }) + .collect(); + OrderingEquivalentGroup::new(new_data) +} - fn add_new_ordering(&mut self, ordering: &[PhysicalSortExpr]) { - let mut is_redundant = false; - let mut new_res = vec![]; - for existing_ordering in self.iter() { - if let Some(finer) = Self::get_finer(existing_ordering, ordering) { - // existing_ordering = finer; - new_res.push(finer); - is_redundant = true; - } else { - new_res.push(existing_ordering.to_vec()); - } - } - if !is_redundant { - new_res.push(ordering.to_vec()); +/// Adds `offset` value to the index of each expression inside `self.head` and `self.others`. +pub fn add_offset( + in_data: &OrderingEquivalentGroup, + offset: usize, +) -> Result { + let res = in_data + .iter() + .map(|ordering| add_offset_to_lex_ordering(ordering, offset)) + .collect::>>()?; + Ok(OrderingEquivalentGroup::new(res)) +} + +fn get_finer( + lhs: &[PhysicalSortExpr], + rhs: &[PhysicalSortExpr], +) -> Option> { + if izip!(lhs.iter(), rhs.iter()).all(|(lhs, rhs)| lhs.eq(rhs)) { + if lhs.len() > rhs.len() { + return Some(lhs.to_vec()); + } else { + return Some(rhs.to_vec()); } - let head = new_res[0].clone(); - let others = new_res[1..].to_vec(); - *self = OrderingEquivalentClass::new(head, others); } + None } +// impl OrderingEquivalentClass { +// /// This function updates ordering equivalences with alias information. +// /// For instance, assume columns `a` and `b` are aliases (a as b), and +// /// orderings `a ASC` and `c DESC` are equivalent. Here, we replace column +// /// `a` with `b` in ordering equivalence expressions. After this function, +// /// `a ASC`, `c DESC` will be converted to the `b ASC`, `c DESC`. +// fn update_with_aliases( +// &mut self, +// oeq_alias_map: &[(Column, Column)], +// fields: &Fields, +// ) { +// let is_head_invalid = self.head.iter().any(|sort_expr| { +// collect_columns(&sort_expr.expr) +// .iter() +// .any(|col| is_column_invalid_in_new_schema(col, fields)) +// }); +// // If head is invalidated, update head with alias expressions +// if is_head_invalid { +// self.head = update_with_alias(self.head.clone(), oeq_alias_map); +// } else { +// let new_oeq_expr = update_with_alias(self.head.clone(), oeq_alias_map); +// self.insert(new_oeq_expr); +// } +// for ordering in self.others.clone().into_iter() { +// self.insert(update_with_alias(ordering, oeq_alias_map)); +// } +// } +// +// /// Adds `offset` value to the index of each expression inside `self.head` and `self.others`. +// pub fn add_offset(&self, offset: usize) -> Result { +// let head = add_offset_to_lex_ordering(self.head(), offset)?; +// let others = self +// .others() +// .iter() +// .map(|ordering| add_offset_to_lex_ordering(ordering, offset)) +// .collect::>>()?; +// Ok(OrderingEquivalentClass::new(head, others)) +// } +// +// // /// This function normalizes `OrderingEquivalenceProperties` according to `eq_properties`. +// // /// More explicitly, it makes sure that expressions in `oeq_class` are head entries +// // /// in `eq_properties`, replacing any non-head entries with head entries if necessary. +// // pub fn normalize_with_equivalence_properties( +// // &self, +// // eq_properties: &EquivalenceProperties, +// // ) -> OrderingEquivalentClass { +// // let head = eq_properties.normalize_sort_exprs(self.head()); +// // +// // let others = self +// // .others() +// // .iter() +// // .map(|other| eq_properties.normalize_sort_exprs(other)) +// // .collect(); +// // +// // EquivalentClass::new(head, others) +// // } +// +// /// Prefix with existing ordering. +// pub fn prefix_ordering_equivalent_class_with_existing_ordering( +// &self, +// existing_ordering: &[PhysicalSortExpr], +// ) -> OrderingEquivalentClass { +// // let existing_ordering = eq_properties.normalize_sort_exprs(existing_ordering); +// // let normalized_head = eq_properties.normalize_sort_exprs(self.head()); +// let normalized_head = self.head(); +// let updated_head = merge_vectors(&existing_ordering, &normalized_head); +// let updated_others = self +// .others() +// .iter() +// .map(|ordering| { +// // let normalized_ordering = eq_properties.normalize_sort_exprs(ordering); +// let normalized_ordering = ordering; +// merge_vectors(&existing_ordering, &normalized_ordering) +// }) +// .collect(); +// OrderingEquivalentClass::new(updated_head, updated_others) +// } +// +// fn get_finer( +// lhs: &[PhysicalSortExpr], +// rhs: &[PhysicalSortExpr], +// ) -> Option> { +// if izip!(lhs.iter(), rhs.iter()).all(|(lhs, rhs)| lhs.eq(rhs)) { +// if lhs.len() > rhs.len() { +// return Some(lhs.to_vec()); +// } else { +// return Some(rhs.to_vec()); +// } +// } +// None +// } +// +// fn add_new_ordering(&mut self, ordering: &[PhysicalSortExpr]) { +// let mut is_redundant = false; +// let mut new_res = vec![]; +// for existing_ordering in self.iter() { +// if let Some(finer) = Self::get_finer(existing_ordering, ordering) { +// // existing_ordering = finer; +// new_res.push(finer); +// is_redundant = true; +// } else { +// new_res.push(existing_ordering.to_vec()); +// } +// } +// if !is_redundant { +// new_res.push(ordering.to_vec()); +// } +// let head = new_res[0].clone(); +// let others = new_res[1..].to_vec(); +// *self = OrderingEquivalentClass::new(head, others); +// } +// } + /// This is a builder object facilitating incremental construction /// for ordering equivalences. pub struct OrderingEquivalenceBuilder { @@ -1028,7 +1225,7 @@ impl OrderingEquivalenceBuilder { new_ordering_eq_properties: OrderingEquivalenceProperties, ) -> Self { self.ordering_eq_properties - .extend(new_ordering_eq_properties.oeq_class().cloned()); + .extend(new_ordering_eq_properties.oeq_group); self } @@ -1062,10 +1259,8 @@ impl OrderingEquivalenceBuilder { } // If there is an existing ordering, add new ordering as an equivalence: if !normalized_out_ordering.is_empty() { - self.ordering_eq_properties.add_ordering_equal_conditions(( - &normalized_out_ordering, - &new_equivalent_ordering, - )); + self.ordering_eq_properties + .add_new_orderings(&[normalized_out_ordering, new_equivalent_ordering]); } } @@ -1090,129 +1285,36 @@ fn is_column_invalid_in_new_schema(column: &Column, fields: &Fields) -> bool { idx >= fields.len() || fields[idx].name() != column.name() } -/// Gets first aliased version of `col` found in `alias_map`. -fn get_alias_column( - col: &Column, - alias_map: &HashMap>, -) -> Option { - alias_map - .iter() - .find_map(|(column, columns)| column.eq(col).then(|| columns[0].clone())) -} - -// /// This function applies the given projection to the given equivalence -// /// properties to compute the resulting (projected) equivalence properties; e.g. -// /// 1) Adding an alias, which can introduce additional equivalence properties, -// /// as in Projection(a, a as a1, a as a2). -// /// 2) Truncate the [`EquivalentClass`]es that are not in the output schema. -// pub fn project_equivalence_properties( -// input_eq: EquivalenceProperties, -// alias_map: &HashMap>, -// output_eq: &mut EquivalenceProperties, +// /// This function applies the given projection to the given ordering +// /// equivalence properties to compute the resulting (projected) ordering +// /// equivalence properties; e.g. +// /// 1) Adding an alias, which can introduce additional ordering equivalence +// /// properties, as in Projection(a, a as a1, a as a2) extends global ordering +// /// of a to a1 and a2. +// /// 2) Truncate the [`OrderingEquivalentClass`]es that are not in the output schema. +// pub fn project_ordering_equivalence_properties( +// input_eq: OrderingEquivalenceProperties, +// columns_map: &HashMap>, +// output_eq: &mut OrderingEquivalenceProperties, // ) { // // Get schema and fields of projection output // let schema = output_eq.schema(); // let fields = schema.fields(); // -// let mut eq_classes = input_eq.classes().to_vec(); -// for (column, columns) in alias_map { -// let mut find_match = false; -// for class in eq_classes.iter_mut() { -// // If `self.head` is invalidated in the new schema, update head -// // with this change `self.head` is not randomly assigned by one of the entries from `self.others` -// if is_column_invalid_in_new_schema(&class.head, fields) { -// if let Some(alias_col) = get_alias_column(&class.head, alias_map) { -// class.head = alias_col; -// } -// } -// if class.contains(column) { -// for col in columns { -// class.insert(col.clone()); -// } -// find_match = true; -// break; -// } -// } -// if !find_match { -// eq_classes.push(EquivalentClass::new(column.clone(), columns.clone())); +// let mut oeq_class = input_eq.oeq_class().clone(); +// let mut oeq_alias_map = vec![]; +// for (column, columns) in columns_map { +// if is_column_invalid_in_new_schema(column, fields) { +// oeq_alias_map.push((column.clone(), columns[0].clone())); // } // } +// oeq_class = update_with_aliases(&oeq_class, &oeq_alias_map, fields); // -// // Prune columns that are no longer in the schema from equivalences. -// for class in eq_classes.iter_mut() { -// let columns_to_remove = class -// .iter() -// .filter(|column| is_column_invalid_in_new_schema(column, fields)) -// .cloned() -// .collect::>(); -// for column in columns_to_remove { -// class.remove(&column); -// } +// if oeq_class.len() > 1 { +// output_eq.extend(oeq_class); // } -// -// eq_classes.retain(|props| { -// props.len() > 1 -// && -// // A column should not give an equivalence with itself. -// !(props.len() == 2 && props.head.eq(props.others().iter().next().unwrap())) -// }); -// -// output_eq.extend(eq_classes); // } -/// This function applies the given projection to the given ordering -/// equivalence properties to compute the resulting (projected) ordering -/// equivalence properties; e.g. -/// 1) Adding an alias, which can introduce additional ordering equivalence -/// properties, as in Projection(a, a as a1, a as a2) extends global ordering -/// of a to a1 and a2. -/// 2) Truncate the [`OrderingEquivalentClass`]es that are not in the output schema. -pub fn project_ordering_equivalence_properties( - input_eq: OrderingEquivalenceProperties, - columns_map: &HashMap>, - output_eq: &mut OrderingEquivalenceProperties, -) { - // Get schema and fields of projection output - let schema = output_eq.schema(); - let fields = schema.fields(); - - let oeq_class = input_eq.oeq_class(); - let mut oeq_class = if let Some(oeq_class) = oeq_class { - oeq_class.clone() - } else { - return; - }; - let mut oeq_alias_map = vec![]; - for (column, columns) in columns_map { - if is_column_invalid_in_new_schema(column, fields) { - oeq_alias_map.push((column.clone(), columns[0].clone())); - } - } - oeq_class.update_with_aliases(&oeq_alias_map, fields); - - // Prune columns that no longer is in the schema from from the OrderingEquivalenceProperties. - let sort_exprs_to_remove = oeq_class - .iter() - .filter(|sort_exprs| { - sort_exprs.iter().any(|sort_expr| { - let cols_in_expr = collect_columns(&sort_expr.expr); - // If any one of the columns, used in Expression is invalid, remove expression - // from ordering equivalences - cols_in_expr - .iter() - .any(|col| is_column_invalid_in_new_schema(col, fields)) - }) - }) - .cloned() - .collect::>(); - for sort_exprs in sort_exprs_to_remove { - oeq_class.remove(&sort_exprs); - } - if oeq_class.len() > 1 { - output_eq.extend(Some(oeq_class)); - } -} - /// Update `ordering` if it contains cast expression with target column /// after projection, if there is no cast expression among `ordering` expressions, /// returns `None`. @@ -1238,15 +1340,28 @@ pub fn update_ordering_equivalence_with_cast( cast_exprs: &[(CastExpr, Column)], input_oeq: &mut OrderingEquivalenceProperties, ) { - if let Some(cls) = &mut input_oeq.oeq_class { - for ordering in - std::iter::once(cls.head().clone()).chain(cls.others().clone().into_iter()) + // if let Some(cls) = &mut input_oeq.oeq_class { + // for ordering in + // std::iter::once(cls.head().clone()).chain(cls.others().clone().into_iter()) + // { + // if let Some(updated_ordering) = update_with_cast_exprs(cast_exprs, ordering) { + // cls.insert(updated_ordering); + // } + // } + // } + let mut new_casts = vec![]; + for ordering in input_oeq.oeq_group.iter() { + if let Some(updated_ordering) = + update_with_cast_exprs(cast_exprs, ordering.clone()) { - if let Some(updated_ordering) = update_with_cast_exprs(cast_exprs, ordering) { - cls.insert(updated_ordering); - } + new_casts.push(updated_ordering); } } + if !new_casts.is_empty() { + input_oeq + .oeq_group + .extend(OrderingEquivalentGroup::new(new_casts)); + } } /// Retrieves the ordering equivalence properties for a given schema and output ordering. @@ -1259,9 +1374,7 @@ pub fn ordering_equivalence_properties_helper( // Return an empty OrderingEquivalenceProperties: return oep; } else { - let head = eq_orderings[0].clone(); - let others = eq_orderings[1..].to_vec(); - oep.extend(Some(OrderingEquivalentClass::new(head, others))); + oep.extend(OrderingEquivalentGroup::new(eq_orderings.to_vec())); return oep; } // oep.extend(Some(OrderingEquivalentClass::new())) @@ -1296,27 +1409,34 @@ fn collapse_vec(input: Vec) -> Vec { /// entries that have same physical expression inside the given vector `input`. /// `vec![a Some(Asc), a Some(Desc)]` is collapsed to the `vec![a Some(Asc)]`. Since /// when same expression is already seen before, following expressions are redundant. -fn collapse_lex_req(input: LexOrderingReq) -> LexOrderingReq { +fn collapse_lex_sort_exprs(input: LexOrdering) -> LexOrdering { let mut output = vec![]; for item in input { - if !lex_req_contains(&output, &item) { + if output + .iter() + .all(|elem: &PhysicalSortExpr| !elem.expr.eq(&item.expr)) + { output.push(item); } } output } -/// Check whether `sort_req.expr` is among the expressions of `lex_req`. -fn lex_req_contains( - lex_req: &[PhysicalSortRequirement], - sort_req: &PhysicalSortRequirement, -) -> bool { - for constant in lex_req { - if constant.expr.eq(&sort_req.expr) { - return true; +/// This function constructs a duplicate-free `LexOrderingReq` by filtering out duplicate +/// entries that have same physical expression inside the given vector `input`. +/// `vec![a Some(Asc), a Some(Desc)]` is collapsed to the `vec![a Some(Asc)]`. Since +/// when same expression is already seen before, following expressions are redundant. +fn collapse_lex_req(input: LexOrderingReq) -> LexOrderingReq { + let mut output = vec![]; + for item in input { + if output + .iter() + .all(|elem: &PhysicalSortRequirement| !elem.expr.eq(&item.expr)) + { + output.push(item); } } - false + output } /// This function simplifies lexicographical ordering requirement @@ -1324,7 +1444,7 @@ fn lex_req_contains( /// that satisfy global ordering (occurs inside the ordering equivalent class) fn simplify_lex_req( input: LexOrderingReq, - oeq_class: &OrderingEquivalentClass, + oeq_class: &OrderingEquivalentGroup, ) -> LexOrderingReq { let mut section = &input[..]; loop { @@ -1336,7 +1456,8 @@ fn simplify_lex_req( section = §ion[0..section.len() - n_prune]; } if section.is_empty() { - PhysicalSortRequirement::from_sort_exprs(oeq_class.head()) + // By convention use first entry + PhysicalSortRequirement::from_sort_exprs(&oeq_class.inner[0]) } else { section.to_vec() } @@ -1352,10 +1473,10 @@ fn simplify_lex_req( /// this function will return 1, to indicate last element can be removed from the requirement fn prune_last_n_that_is_in_oeq( input: &[PhysicalSortRequirement], - oeq_class: &OrderingEquivalentClass, + oeq_class: &OrderingEquivalentGroup, ) -> usize { let input_len = input.len(); - for ordering in std::iter::once(oeq_class.head()).chain(oeq_class.others().iter()) { + for ordering in oeq_class.iter() { let mut search_range = std::cmp::min(ordering.len(), input_len); while search_range > 0 { let req_section = &input[input_len - search_range..]; @@ -1410,18 +1531,6 @@ fn get_compatible_ranges( .collect() } -/// It is similar to contains method of vector. -/// Finds whether `expr` is among `physical_exprs`. -pub fn physical_exprs_contains( - physical_exprs: &[Arc], - expr: &Arc, -) -> bool { - // println!("physical_exprs:{:?}, expr:{:?}", physical_exprs, expr); - physical_exprs - .iter() - .any(|physical_expr| physical_expr.eq(expr)) -} - /// Remove ordering requirements that have constant value fn prune_sort_reqs_with_constants( ordering: &[PhysicalSortRequirement], @@ -1505,18 +1614,17 @@ pub fn update_ordering( // We have an intermediate (non-leaf) node, account for its children: node.state = Some(node.expr.get_ordering(children_sort_options)); Ok(Transformed::Yes(node)) - } else if let Some(column) = node.expr.as_any().downcast_ref::() { + } else if node.expr.as_any().is::() { // We have a Column, which is one of the two possible leaf node types: // TODO: Make this a method of ordering equivalence - if let Some(oeq_class) = ordering_equal_properties.oeq_class() { - for ordering in oeq_class.iter() { - let global_ordering = &ordering[0]; - if node.expr.eq(&global_ordering.expr) { - node.state = Some(SortProperties::Ordered(global_ordering.options)); - return Ok(Transformed::Yes(node)); - } + for ordering in ordering_equal_properties.oeq_group().iter() { + let global_ordering = &ordering[0]; + if node.expr.eq(&global_ordering.expr) { + node.state = Some(SortProperties::Ordered(global_ordering.options)); + return Ok(Transformed::Yes(node)); } } + node.state = None; Ok(Transformed::No(node)) } else { @@ -1530,31 +1638,31 @@ pub fn update_ordering( /// Combine equivalence properties of the given join inputs. pub fn combine_join_equivalence_properties2( join_type: &JoinType, - left_eq_classes: &[Vec>], - right_eq_classes: &[Vec>], + left_eq_classes: &EquivalentGroups, + right_eq_classes: &EquivalentGroups, left_columns_len: usize, on: &[(Column, Column)], out_properties: &mut OrderingEquivalenceProperties, ) -> Result<()> { - let mut res = vec![]; + let mut res = EquivalentGroups::empty(); match join_type { JoinType::Inner | JoinType::Left | JoinType::Full | JoinType::Right => { - res.extend(left_eq_classes.to_vec()); + res.extend(left_eq_classes.clone()); let updated_eq_classes = right_eq_classes .iter() .map(|eq_class| add_offset_to_exprs(eq_class.to_vec(), left_columns_len)) .collect::>>()?; - res.extend(updated_eq_classes); + res.extend(EquivalentGroups::new(updated_eq_classes)); } JoinType::LeftSemi | JoinType::LeftAnti => { - res.extend(left_eq_classes.to_vec()); + res.extend(left_eq_classes.clone()); } JoinType::RightSemi | JoinType::RightAnti => { - res.extend(right_eq_classes.to_vec()); + res.extend(right_eq_classes.clone()); } } - out_properties.eq_classes = res; + out_properties.eq_groups = res; if *join_type == JoinType::Inner { on.iter().for_each(|(lhs, rhs)| { let new_lhs = Arc::new(lhs.clone()) as _; @@ -1566,27 +1674,96 @@ pub fn combine_join_equivalence_properties2( }); } Ok(()) - - // if join_type == JoinType::Inner { - // on.iter().for_each(|(column1, column2)| { - // let new_column2 = - // Column::new(column2.name(), left_columns_len + column2.index()); - // new_properties.add_equal_conditions((column1, &new_column2)) - // }) - // } - // new_properties } #[cfg(test)] mod tests { use super::*; - use crate::expressions::Column; + use crate::expressions::{col, lit, Column}; use arrow::datatypes::{DataType, Field, Schema}; use datafusion_common::Result; + use crate::physical_expr::physical_exprs_equal; use arrow_schema::SortOptions; use std::sync::Arc; + // Generate a schema which consists of 5 columns (a, b, c, d, e) + fn create_test_schema() -> Result { + let a = Field::new("a", DataType::Int32, true); + let b = Field::new("b", DataType::Int32, true); + let c = Field::new("c", DataType::Int32, true); + let d = Field::new("d", DataType::Int32, true); + let e = Field::new("e", DataType::Int32, true); + let f = Field::new("f", DataType::Int32, true); + let schema = Arc::new(Schema::new(vec![a, b, c, d, e, f])); + + Ok(schema) + } + + fn create_test_params() -> Result<(SchemaRef, OrderingEquivalenceProperties)> { + // Assume schema satisfies ordering a ASC NULLS LAST + // and d ASC NULLS LAST, b ASC NULLS LAST and e DESC NULLS FIRST, f ASC NULLS LAST, g ASC NULLS LAST + // Assume that column a and c are aliases. + let col_a = &Column::new("a", 0); + let col_b = &Column::new("b", 1); + let col_c = &Column::new("c", 2); + let col_d = &Column::new("d", 3); + let col_e = &Column::new("e", 4); + let col_f = &Column::new("f", 5); + let col_g = &Column::new("g", 6); + let option1 = SortOptions { + descending: false, + nulls_first: false, + }; + let option2 = SortOptions { + descending: true, + nulls_first: true, + }; + let test_schema = create_test_schema()?; + let col_a_expr = Arc::new(col_a.clone()) as _; + let col_c_expr = Arc::new(col_c.clone()) as _; + let mut ordering_eq_properties = + OrderingEquivalenceProperties::new(test_schema.clone()); + ordering_eq_properties.add_equal_conditions((&col_a_expr, &col_c_expr)); + ordering_eq_properties.add_new_orderings(&[ + vec![PhysicalSortExpr { + expr: Arc::new(col_a.clone()), + options: option1, + }], + vec![ + PhysicalSortExpr { + expr: Arc::new(col_d.clone()), + options: option1, + }, + PhysicalSortExpr { + expr: Arc::new(col_b.clone()), + options: option1, + }, + ], + ]); + ordering_eq_properties.add_new_orderings(&[ + vec![PhysicalSortExpr { + expr: Arc::new(col_a.clone()), + options: option1, + }], + vec![ + PhysicalSortExpr { + expr: Arc::new(col_e.clone()), + options: option2, + }, + PhysicalSortExpr { + expr: Arc::new(col_f.clone()), + options: option1, + }, + PhysicalSortExpr { + expr: Arc::new(col_g.clone()), + options: option1, + }, + ], + ]); + Ok((test_schema, ordering_eq_properties)) + } + fn convert_to_requirement( in_data: &[(&Column, Option)], ) -> Vec { @@ -1617,20 +1794,20 @@ mod tests { let new_condition = (&col_a_expr, &col_b_expr); eq_properties.add_equal_conditions(new_condition); - assert_eq!(eq_properties.eq_classes().len(), 1); + assert_eq!(eq_properties.eq_groups().len(), 1); let new_condition = (&col_b_expr, &col_a_expr); eq_properties.add_equal_conditions(new_condition); - assert_eq!(eq_properties.eq_classes().len(), 1); - let eq_class = &eq_properties.eq_classes()[0]; + assert_eq!(eq_properties.eq_groups().len(), 1); + let eq_class = &eq_properties.eq_groups().inner[0]; assert_eq!(eq_class.len(), 2); assert!(physical_exprs_contains(eq_class, &col_a_expr)); assert!(physical_exprs_contains(eq_class, &col_b_expr)); let new_condition = (&col_b_expr, &col_c_expr); eq_properties.add_equal_conditions(new_condition); - assert_eq!(eq_properties.eq_classes().len(), 1); - let eq_class = &eq_properties.eq_classes()[0]; + assert_eq!(eq_properties.eq_groups().len(), 1); + let eq_class = &eq_properties.eq_groups().inner[0]; assert_eq!(eq_class.len(), 3); assert!(physical_exprs_contains(eq_class, &col_a_expr)); assert!(physical_exprs_contains(eq_class, &col_b_expr)); @@ -1639,14 +1816,14 @@ mod tests { // This is a new set of equality. Hence equivalent class count should be 2. let new_condition = (&col_x_expr, &col_y_expr); eq_properties.add_equal_conditions(new_condition); - assert_eq!(eq_properties.eq_classes().len(), 2); + assert_eq!(eq_properties.eq_groups().len(), 2); // This equality bridges distinct equality sets. // Hence equivalent class count should decrease from 2 to 1. let new_condition = (&col_x_expr, &col_a_expr); eq_properties.add_equal_conditions(new_condition); - assert_eq!(eq_properties.eq_classes().len(), 1); - let eq_class = &eq_properties.eq_classes()[0]; + assert_eq!(eq_properties.eq_groups().len(), 1); + let eq_class = &eq_properties.eq_groups().inner[0]; assert_eq!(eq_class.len(), 5); assert!(physical_exprs_contains(eq_class, &col_a_expr)); assert!(physical_exprs_contains(eq_class, &col_b_expr)); @@ -1692,22 +1869,12 @@ mod tests { (col_a_expr.clone(), col_a3_expr.clone()), (col_a_expr.clone(), col_a4_expr.clone()), ]; - let mut alias_map = HashMap::new(); - alias_map.insert( - Column::new("a", 0), - vec![ - Column::new("a1", 0), - Column::new("a2", 1), - Column::new("a3", 2), - Column::new("a4", 3), - ], - ); // let mut out_properties = OrderingEquivalenceProperties::new(out_schema); let out_properties = - input_properties.project(&alias_map, &source_to_target_mapping, out_schema); + input_properties.project(&source_to_target_mapping, out_schema); - assert_eq!(out_properties.eq_classes().len(), 1); - let eq_class = &out_properties.eq_classes()[0]; + assert_eq!(out_properties.eq_groups().len(), 1); + let eq_class = &out_properties.eq_groups().inner[0]; assert_eq!(eq_class.len(), 4); assert!(physical_exprs_contains(eq_class, &col_a1_expr)); assert!(physical_exprs_contains(eq_class, &col_a2_expr)); @@ -1765,4 +1932,316 @@ mod tests { } Ok(()) } + + #[test] + fn test_ordering_satisfy() -> Result<()> { + let crude = vec![PhysicalSortExpr { + expr: Arc::new(Column::new("a", 0)), + options: SortOptions::default(), + }]; + let finer = vec![ + PhysicalSortExpr { + expr: Arc::new(Column::new("a", 0)), + options: SortOptions::default(), + }, + PhysicalSortExpr { + expr: Arc::new(Column::new("b", 1)), + options: SortOptions::default(), + }, + ]; + // finer ordering satisfies, crude ordering shoul return true + let empty_schema = &Arc::new(Schema::empty()); + let mut oeq_properties = OrderingEquivalenceProperties::new(empty_schema.clone()); + oeq_properties.oeq_group.push(finer.clone()); + assert!(oeq_properties.ordering_satisfy(Some(&crude))); + + // Crude ordering doesn't satisfy finer ordering. should return false + let mut oeq_properties = OrderingEquivalenceProperties::new(empty_schema.clone()); + oeq_properties.oeq_group.push(crude.clone()); + assert!(!oeq_properties.ordering_satisfy(Some(&finer))); + Ok(()) + } + + #[test] + fn test_ordering_satisfy_with_equivalence() -> Result<()> { + let col_a = &Column::new("a", 0); + let col_b = &Column::new("b", 1); + let col_c = &Column::new("c", 2); + let col_d = &Column::new("d", 3); + let col_e = &Column::new("e", 4); + let col_f = &Column::new("f", 5); + let col_g = &Column::new("g", 6); + let option1 = SortOptions { + descending: false, + nulls_first: false, + }; + let option2 = SortOptions { + descending: true, + nulls_first: true, + }; + // The schema is ordered by a ASC NULLS LAST, b ASC NULLS LAST + let provided = vec![ + PhysicalSortExpr { + expr: Arc::new(col_a.clone()), + options: option1, + }, + PhysicalSortExpr { + expr: Arc::new(col_b.clone()), + options: option1, + }, + ]; + let provided = Some(&provided[..]); + let (_test_schema, ordering_eq_properties) = create_test_params()?; + // First element in the tuple stores vector of requirement, second element is the expected return value for ordering_satisfy function + let requirements = vec![ + // `a ASC NULLS LAST`, expects `ordering_satisfy` to be `true`, since existing ordering `a ASC NULLS LAST, b ASC NULLS LAST` satisfies it + (vec![(col_a, option1)], true), + (vec![(col_a, option2)], false), + // Test whether equivalence works as expected + (vec![(col_c, option1)], true), + (vec![(col_c, option2)], false), + // Test whether ordering equivalence works as expected + (vec![(col_d, option1)], true), + (vec![(col_d, option1), (col_b, option1)], true), + (vec![(col_d, option2), (col_b, option1)], false), + ( + vec![(col_e, option2), (col_f, option1), (col_g, option1)], + true, + ), + (vec![(col_e, option2), (col_f, option1)], true), + (vec![(col_e, option1), (col_f, option1)], false), + (vec![(col_e, option2), (col_b, option1)], false), + (vec![(col_e, option1), (col_b, option1)], false), + ( + vec![ + (col_d, option1), + (col_b, option1), + (col_d, option1), + (col_b, option1), + ], + true, + ), + ( + vec![ + (col_d, option1), + (col_b, option1), + (col_e, option2), + (col_f, option1), + ], + true, + ), + ( + vec![ + (col_d, option1), + (col_b, option1), + (col_e, option2), + (col_b, option1), + ], + true, + ), + ( + vec![ + (col_d, option1), + (col_b, option1), + (col_d, option2), + (col_b, option1), + ], + true, + ), + ( + vec![ + (col_d, option1), + (col_b, option1), + (col_e, option1), + (col_f, option1), + ], + false, + ), + ( + vec![ + (col_d, option1), + (col_b, option1), + (col_e, option1), + (col_b, option1), + ], + false, + ), + (vec![(col_d, option1), (col_e, option2)], true), + ]; + + for (cols, expected) in requirements { + let err_msg = format!("Error in test case:{cols:?}"); + let required = cols + .into_iter() + .map(|(col, options)| PhysicalSortExpr { + expr: Arc::new(col.clone()), + options, + }) + .collect::>(); + + let required = Some(&required[..]); + assert_eq!( + ordering_eq_properties.ordering_satisfy(required), + expected, + "{err_msg}" + ); + } + Ok(()) + } + + #[test] + fn test_ordering_satisfy_different_lengths() -> Result<()> { + let test_schema = create_test_schema()?; + let col_a_expr = col("a", &test_schema)?; + let col_b_expr = col("b", &test_schema)?; + let col_c_expr = col("c", &test_schema)?; + let col_d_expr = col("d", &test_schema)?; + let col_e_expr = col("e", &test_schema)?; + let col_f_expr = col("f", &test_schema)?; + let option1 = SortOptions { + descending: false, + nulls_first: false, + }; + // Column a and c are aliases. + let mut ordering_eq_properties = OrderingEquivalenceProperties::new(test_schema); + ordering_eq_properties.add_equal_conditions((&col_a_expr, &col_c_expr)); + + // Column a and e are ordering equivalent (e.g global ordering of the table can be described both as a ASC and e ASC.) + ordering_eq_properties.add_new_orderings(&[ + vec![PhysicalSortExpr { + expr: col_a_expr.clone(), + options: option1, + }], + vec![PhysicalSortExpr { + expr: col_e_expr.clone(), + options: option1, + }], + ]); + + // Column a and d,f are ordering equivalent (e.g global ordering of the table can be described both as [a ASC] and [d ASC, f ASC].) + ordering_eq_properties.add_new_orderings(&[ + vec![PhysicalSortExpr { + expr: col_a_expr.clone(), + options: option1, + }], + vec![ + PhysicalSortExpr { + expr: col_d_expr.clone(), + options: option1, + }, + PhysicalSortExpr { + expr: col_f_expr.clone(), + options: option1, + }, + ], + ]); + // All of the orderings [a ASC, [d ASC, f ASC], [e ASC]] + // are valid for the table + // Also Columns a and c are equal + + let sort_req_a = PhysicalSortExpr { + expr: col_a_expr.clone(), + options: option1, + }; + let sort_req_b = PhysicalSortExpr { + expr: col_b_expr.clone(), + options: option1, + }; + let sort_req_c = PhysicalSortExpr { + expr: col_c_expr.clone(), + options: option1, + }; + let sort_req_d = PhysicalSortExpr { + expr: col_d_expr.clone(), + options: option1, + }; + let sort_req_e = PhysicalSortExpr { + expr: col_e_expr.clone(), + options: option1, + }; + let sort_req_f = PhysicalSortExpr { + expr: col_f_expr.clone(), + options: option1, + }; + + assert!(ordering_eq_properties.ordering_satisfy_concrete( + // After normalization would be a ASC + &[sort_req_c.clone(), sort_req_a.clone(), sort_req_e.clone(),], + )); + assert!(!ordering_eq_properties.ordering_satisfy_concrete( + // After normalization would be a ASC, b ASC + // which is not satisfied + &[sort_req_c.clone(), sort_req_b.clone(),], + )); + + assert!(ordering_eq_properties.ordering_satisfy_concrete( + // After normalization would be a ASC + &[sort_req_c.clone(), sort_req_d.clone(),], + )); + + assert!(!ordering_eq_properties.ordering_satisfy_concrete( + // After normalization would be a ASC, b ASC + // which is not satisfied + &[sort_req_d.clone(), sort_req_f.clone(), sort_req_b.clone(),], + )); + + assert!(ordering_eq_properties.ordering_satisfy_concrete( + // After normalization would be a ASC + // which is satisfied + &[sort_req_d.clone(), sort_req_f.clone()], + )); + + Ok(()) + } + + #[test] + fn test_bridge_groups() -> Result<()> { + let entries = vec![ + vec![lit(1), lit(2), lit(3)], + vec![lit(2), lit(4), lit(5)], + vec![lit(11), lit(12), lit(9)], + vec![lit(7), lit(6), lit(5)], + ]; + // Expected is a bit weird. However, what we care is they expected contains distinct groups. + // where there is no common entry between any groups. + // Since we do check for vector equality, this version should be used during comparison in the test. + let expected = vec![ + vec![lit(11), lit(12), lit(9)], + vec![lit(7), lit(6), lit(5), lit(2), lit(4), lit(1), lit(3)], + ]; + let mut eq_groups = EquivalentGroups::new(entries); + eq_groups.bridge_groups(); + + let eq_groups = eq_groups.inner; + assert_eq!(eq_groups.len(), expected.len()); + assert_eq!(eq_groups.len(), 2); + + assert!(physical_exprs_equal(&eq_groups[0], &expected[0])); + assert!(physical_exprs_equal(&eq_groups[1], &expected[1])); + Ok(()) + } + + #[test] + fn test_remove_redundant_entries() -> Result<()> { + let entries = vec![ + vec![lit(1), lit(1), lit(2)], + // This group is meaningless should be removed + vec![lit(3), lit(3)], + vec![lit(4), lit(5), lit(6)], + ]; + // Expected is a bit weird. However, what we care is they expected contains distinct groups. + // where there is no common entry between any groups. + // Since we do check for vector equality, this version should be used during comparison in the test. + let expected = vec![vec![lit(1), lit(2)], vec![lit(4), lit(5), lit(6)]]; + let mut eq_groups = EquivalentGroups::new(entries); + eq_groups.remove_redundant_entries(); + + let eq_groups = eq_groups.inner; + assert_eq!(eq_groups.len(), expected.len()); + assert_eq!(eq_groups.len(), 2); + + assert!(physical_exprs_equal(&eq_groups[0], &expected[0])); + assert!(physical_exprs_equal(&eq_groups[1], &expected[1])); + Ok(()) + } } diff --git a/datafusion/physical-expr/src/lib.rs b/datafusion/physical-expr/src/lib.rs index cb13f65ae69f..3fc9010d1299 100644 --- a/datafusion/physical-expr/src/lib.rs +++ b/datafusion/physical-expr/src/lib.rs @@ -56,8 +56,7 @@ pub use aggregate::AggregateExpr; pub use analysis::{analyze, AnalysisContext, ExprBoundaries}; pub use equivalence::{ add_offset_to_lex_ordering, ordering_equivalence_properties_helper, - project_ordering_equivalence_properties, EquivalentClass, - OrderingEquivalenceProperties, OrderingEquivalentClass, + OrderingEquivalenceProperties, }; pub use partitioning::{Distribution, Partitioning}; diff --git a/datafusion/physical-expr/src/partitioning.rs b/datafusion/physical-expr/src/partitioning.rs index e5db2f9a41f2..dc0f3656431f 100644 --- a/datafusion/physical-expr/src/partitioning.rs +++ b/datafusion/physical-expr/src/partitioning.rs @@ -83,15 +83,15 @@ impl Partitioning { // and normalize both exprs based on the eq_properties if !fast_match { let eq_properties = equal_properties(); - let eq_classes = eq_properties.eq_classes(); + let eq_classes = eq_properties.eq_groups(); if !eq_classes.is_empty() { let normalized_required_exprs = required_exprs .iter() - .map(|e| eq_properties.normalize_expr(e.clone())) + .map(|e| eq_classes.normalize_expr(e.clone())) .collect::>(); let normalized_partition_exprs = partition_exprs .iter() - .map(|e| eq_properties.normalize_expr(e.clone())) + .map(|e| eq_classes.normalize_expr(e.clone())) .collect::>(); expr_list_eq_strict_order( &normalized_required_exprs, diff --git a/datafusion/physical-expr/src/physical_expr.rs b/datafusion/physical-expr/src/physical_expr.rs index 81702d8bfae0..fad6487e9878 100644 --- a/datafusion/physical-expr/src/physical_expr.rs +++ b/datafusion/physical-expr/src/physical_expr.rs @@ -27,6 +27,8 @@ use datafusion_common::utils::DataPtr; use datafusion_common::{internal_err, not_impl_err, DataFusionError, Result}; use datafusion_expr::ColumnarValue; +use hashbrown::HashSet; +use itertools::izip; use std::any::Any; use std::fmt::{Debug, Display}; use std::hash::{Hash, Hasher}; @@ -192,3 +194,190 @@ pub fn physical_exprs_contains( .iter() .any(|physical_expr| physical_expr.eq(expr)) } + +/// This util removes duplicates from the `physical_exprs` vector in its argument. +/// Once we can use `HashSet` with `Arc` use it instead. +pub fn deduplicate_physical_exprs( + physical_exprs: &[Arc], +) -> Vec> { + let mut unique_physical_exprs = vec![]; + for expr in physical_exprs { + if !physical_exprs_contains(&unique_physical_exprs, expr) { + unique_physical_exprs.push(expr.clone()); + } + } + unique_physical_exprs +} + +/// Check whether vectors in the arguments have common entries +pub fn have_common_entries( + lhs: &[Arc], + rhs: &[Arc], +) -> bool { + lhs.iter().any(|expr| physical_exprs_contains(rhs, expr)) +} + +/// Check whether physical exprs vectors are equal. +pub fn physical_exprs_equal( + lhs: &[Arc], + rhs: &[Arc], +) -> bool { + lhs.len() == rhs.len() && izip!(lhs, rhs).all(|(lhs, rhs)| lhs.eq(rhs)) +} + +#[cfg(test)] +mod tests { + use crate::expressions::{Column, Literal}; + use crate::physical_expr::{ + deduplicate_physical_exprs, have_common_entries, physical_exprs_equal, + }; + use crate::{physical_exprs_contains, PhysicalExpr}; + use datafusion_common::{Result, ScalarValue}; + use itertools::izip; + use std::sync::Arc; + + #[test] + fn test_physical_exprs_contains() -> Result<()> { + let lit_true = Arc::new(Literal::new(ScalarValue::Boolean(Some(true)))) + as Arc; + let lit_false = Arc::new(Literal::new(ScalarValue::Boolean(Some(false)))) + as Arc; + let lit4 = + Arc::new(Literal::new(ScalarValue::Int32(Some(4)))) as Arc; + let lit2 = + Arc::new(Literal::new(ScalarValue::Int32(Some(2)))) as Arc; + let lit1 = + Arc::new(Literal::new(ScalarValue::Int32(Some(1)))) as Arc; + let col_a_expr = Arc::new(Column::new("a", 0)) as Arc; + let col_b_expr = Arc::new(Column::new("b", 1)) as Arc; + let col_c_expr = Arc::new(Column::new("c", 2)) as Arc; + + // lit(true), lit(false), lit(4), lit(2), Col(a), Col(b) + let physical_exprs: Vec> = vec![ + lit_true.clone(), + lit_false.clone(), + lit4.clone(), + lit2.clone(), + col_a_expr.clone(), + col_b_expr.clone(), + ]; + // below expressions are inside physical_exprs + assert!(physical_exprs_contains(&physical_exprs, &lit_true)); + assert!(physical_exprs_contains(&physical_exprs, &lit2)); + assert!(physical_exprs_contains(&physical_exprs, &col_b_expr)); + + // below expressions are not inside physical_exprs + assert!(!physical_exprs_contains(&physical_exprs, &col_c_expr)); + assert!(!physical_exprs_contains(&physical_exprs, &lit1)); + Ok(()) + } + + #[test] + fn test_deduplicate_physical_exprs() -> Result<()> { + let lit_true = Arc::new(Literal::new(ScalarValue::Boolean(Some(true)))) + as Arc; + let lit_false = Arc::new(Literal::new(ScalarValue::Boolean(Some(false)))) + as Arc; + let lit4 = + Arc::new(Literal::new(ScalarValue::Int32(Some(4)))) as Arc; + let lit2 = + Arc::new(Literal::new(ScalarValue::Int32(Some(2)))) as Arc; + let lit1 = + Arc::new(Literal::new(ScalarValue::Int32(Some(1)))) as Arc; + let col_a_expr = Arc::new(Column::new("a", 0)) as Arc; + let col_b_expr = Arc::new(Column::new("b", 1)) as Arc; + let col_c_expr = Arc::new(Column::new("c", 2)) as Arc; + + // lit(true), lit(false), lit(4), lit(2), Col(a), Col(b) + let physical_exprs: Vec> = vec![ + lit_true.clone(), + lit_false.clone(), + lit4.clone(), + lit2.clone(), + col_a_expr.clone(), + col_a_expr.clone(), + col_b_expr.clone(), + lit_true.clone(), + lit2.clone(), + ]; + + let expected = vec![ + lit_true.clone(), + lit_false.clone(), + lit4.clone(), + lit2.clone(), + col_a_expr.clone(), + col_b_expr.clone(), + ]; + // expected contains unique versions of the physical_exprs + let result = deduplicate_physical_exprs(&physical_exprs); + physical_exprs_equal(&result, &expected); + Ok(()) + } + + #[test] + fn test_have_common_entries() -> Result<()> { + let lit_true = Arc::new(Literal::new(ScalarValue::Boolean(Some(true)))) + as Arc; + let lit_false = Arc::new(Literal::new(ScalarValue::Boolean(Some(false)))) + as Arc; + let lit4 = + Arc::new(Literal::new(ScalarValue::Int32(Some(4)))) as Arc; + let lit2 = + Arc::new(Literal::new(ScalarValue::Int32(Some(2)))) as Arc; + let lit1 = + Arc::new(Literal::new(ScalarValue::Int32(Some(1)))) as Arc; + let col_a_expr = Arc::new(Column::new("a", 0)) as Arc; + let col_b_expr = Arc::new(Column::new("b", 1)) as Arc; + let col_c_expr = Arc::new(Column::new("c", 2)) as Arc; + + let vec1: Vec> = vec![lit_true.clone(), lit_false.clone()]; + + let vec2: Vec> = vec![lit_true.clone(), col_b_expr.clone()]; + + let vec3: Vec> = vec![lit2.clone(), lit1.clone()]; + + // lit_true is common + assert!(have_common_entries(&vec1, &vec2)); + + // there is no common entry + assert!(!have_common_entries(&vec1, &vec3)); + assert!(!have_common_entries(&vec2, &vec3)); + Ok(()) + } + + #[test] + fn test_physical_exprs_equal() -> Result<()> { + let lit_true = Arc::new(Literal::new(ScalarValue::Boolean(Some(true)))) + as Arc; + let lit_false = Arc::new(Literal::new(ScalarValue::Boolean(Some(false)))) + as Arc; + let lit4 = + Arc::new(Literal::new(ScalarValue::Int32(Some(4)))) as Arc; + let lit2 = + Arc::new(Literal::new(ScalarValue::Int32(Some(2)))) as Arc; + let lit1 = + Arc::new(Literal::new(ScalarValue::Int32(Some(1)))) as Arc; + let col_a_expr = Arc::new(Column::new("a", 0)) as Arc; + let col_b_expr = Arc::new(Column::new("b", 1)) as Arc; + let col_c_expr = Arc::new(Column::new("c", 2)) as Arc; + + let vec1: Vec> = vec![lit_true.clone(), lit_false.clone()]; + + let vec2: Vec> = vec![lit_true.clone(), col_b_expr.clone()]; + + let vec3: Vec> = vec![lit2.clone(), lit1.clone()]; + + let vec4: Vec> = vec![lit_true.clone(), lit_false.clone()]; + + // these vectors are same + assert!(physical_exprs_equal(&vec1, &vec1)); + assert!(physical_exprs_equal(&vec1, &vec4)); + + // these vectors are different + assert!(!physical_exprs_equal(&vec1, &vec2)); + assert!(!physical_exprs_equal(&vec1, &vec3)); + + Ok(()) + } +} diff --git a/datafusion/physical-expr/src/utils.rs b/datafusion/physical-expr/src/utils.rs index 5040e9b53355..9382012fea86 100644 --- a/datafusion/physical-expr/src/utils.rs +++ b/datafusion/physical-expr/src/utils.rs @@ -130,155 +130,6 @@ pub fn normalize_out_expr_with_columns_map( .unwrap_or(expr) } -/// Transform `sort_exprs` vector, to standardized version using `eq_properties` and `ordering_eq_properties` -/// Assume `eq_properties` states that `Column a` and `Column b` are aliases. -/// Also assume `ordering_eq_properties` states that ordering `vec![d ASC]` and `vec![a ASC, c ASC]` are -/// ordering equivalent (in the sense that both describe the ordering of the table). -/// If the `sort_exprs` input to this function were `vec![b ASC, c ASC]`, -/// This function converts `sort_exprs` `vec![b ASC, c ASC]` to first `vec![a ASC, c ASC]` after considering `eq_properties` -/// Then converts `vec![a ASC, c ASC]` to `vec![d ASC]` after considering `ordering_eq_properties`. -/// Standardized version `vec![d ASC]` is used in subsequent operations. -fn normalize_sort_exprs( - sort_exprs: &[PhysicalSortExpr], - ordering_eq_properties: &OrderingEquivalenceProperties, -) -> Vec { - let sort_requirements = PhysicalSortRequirement::from_sort_exprs(sort_exprs.iter()); - let normalized_exprs = - normalize_sort_requirements(&sort_requirements, ordering_eq_properties); - PhysicalSortRequirement::to_sort_exprs(normalized_exprs) -} - -/// Transform `sort_reqs` vector, to standardized version using `eq_properties` and `ordering_eq_properties` -/// Assume `eq_properties` states that `Column a` and `Column b` are aliases. -/// Also assume `ordering_eq_properties` states that ordering `vec![d ASC]` and `vec![a ASC, c ASC]` are -/// ordering equivalent (in the sense that both describe the ordering of the table). -/// If the `sort_reqs` input to this function were `vec![b Some(ASC), c None]`, -/// This function converts `sort_exprs` `vec![b Some(ASC), c None]` to first `vec![a Some(ASC), c None]` after considering `eq_properties` -/// Then converts `vec![a Some(ASC), c None]` to `vec![d Some(ASC)]` after considering `ordering_eq_properties`. -/// Standardized version `vec![d Some(ASC)]` is used in subsequent operations. -fn normalize_sort_requirements( - sort_reqs: &[PhysicalSortRequirement], - ordering_eq_properties: &OrderingEquivalenceProperties, -) -> Vec { - // let normalized_sort_reqs = eq_properties.normalize_sort_requirements(sort_reqs); - // ordering_eq_properties.normalize_sort_requirements(&normalized_sort_reqs) - ordering_eq_properties.normalize_sort_requirements(&sort_reqs) -} - -/// Checks whether given ordering requirements are satisfied by provided [PhysicalSortExpr]s. -pub fn ordering_satisfy OrderingEquivalenceProperties>( - provided: Option<&[PhysicalSortExpr]>, - required: Option<&[PhysicalSortExpr]>, - ordering_equal_properties: F2, -) -> bool { - match (provided, required) { - (_, None) => true, - (None, Some(_)) => false, - (Some(provided), Some(required)) => { - ordering_satisfy_concrete(provided, required, ordering_equal_properties) - } - } -} - -/// Checks whether the required [`PhysicalSortExpr`]s are satisfied by the -/// provided [`PhysicalSortExpr`]s. -pub fn ordering_satisfy_concrete OrderingEquivalenceProperties>( - provided: &[PhysicalSortExpr], - required: &[PhysicalSortExpr], - ordering_equal_properties: F2, -) -> bool { - let oeq_properties = ordering_equal_properties(); - let required_normalized = normalize_sort_exprs(required, &oeq_properties); - let provided_normalized = normalize_sort_exprs(provided, &oeq_properties); - // println!("required_normalized:{:?}", required_normalized); - // println!("provided_normalized:{:?}", provided_normalized); - if required_normalized.len() > provided_normalized.len() { - return false; - } - required_normalized - .into_iter() - .zip(provided_normalized) - .all(|(req, given)| given == req) -} - -/// Checks whether the given [`PhysicalSortRequirement`]s are satisfied by the -/// provided [`PhysicalSortExpr`]s. -pub fn ordering_satisfy_requirement OrderingEquivalenceProperties>( - provided: Option<&[PhysicalSortExpr]>, - required: Option<&[PhysicalSortRequirement]>, - ordering_equal_properties: F2, -) -> bool { - match (provided, required) { - (_, None) => true, - (None, Some(_)) => false, - (Some(provided), Some(required)) => ordering_satisfy_requirement_concrete( - provided, - required, - ordering_equal_properties, - ), - } -} - -/// Checks whether the given [`PhysicalSortRequirement`]s are satisfied by the -/// provided [`PhysicalSortExpr`]s. -pub fn ordering_satisfy_requirement_concrete< - F2: FnOnce() -> OrderingEquivalenceProperties, ->( - provided: &[PhysicalSortExpr], - required: &[PhysicalSortRequirement], - ordering_equal_properties: F2, -) -> bool { - let oeq_properties = ordering_equal_properties(); - // let eq_properties = equal_properties(); - let required_normalized = normalize_sort_requirements(required, &oeq_properties); - let provided_normalized = normalize_sort_exprs(provided, &oeq_properties); - if required_normalized.len() > provided_normalized.len() { - return false; - } - required_normalized - .into_iter() - .zip(provided_normalized) - .all(|(req, given)| given.satisfy(&req)) -} - -/// Checks whether the given [`PhysicalSortRequirement`]s are equal or more -/// specific than the provided [`PhysicalSortRequirement`]s. -pub fn requirements_compatible OrderingEquivalenceProperties>( - provided: Option<&[PhysicalSortRequirement]>, - required: Option<&[PhysicalSortRequirement]>, - ordering_equal_properties: F, -) -> bool { - match (provided, required) { - (_, None) => true, - (None, Some(_)) => false, - (Some(provided), Some(required)) => requirements_compatible_concrete( - provided, - required, - ordering_equal_properties, - ), - } -} - -/// Checks whether the given [`PhysicalSortRequirement`]s are equal or more -/// specific than the provided [`PhysicalSortRequirement`]s. -fn requirements_compatible_concrete OrderingEquivalenceProperties>( - provided: &[PhysicalSortRequirement], - required: &[PhysicalSortRequirement], - ordering_equal_properties: F, -) -> bool { - let oeq_properties = ordering_equal_properties(); - - let required_normalized = normalize_sort_requirements(required, &oeq_properties); - let provided_normalized = normalize_sort_requirements(provided, &oeq_properties); - if required_normalized.len() > provided_normalized.len() { - return false; - } - required_normalized - .into_iter() - .zip(provided_normalized) - .all(|(req, given)| given.compatible(&req)) -} - /// This function maps back requirement after ProjectionExec /// to the Executor for its input. // Specifically, `ProjectionExec` changes index of `Column`s in the schema of its input executor. @@ -530,26 +381,6 @@ pub fn reverse_order_bys(order_bys: &[PhysicalSortExpr]) -> Vec OrderingEquivalenceProperties>( - req1: &'a [PhysicalSortExpr], - req2: &'a [PhysicalSortExpr], - ordering_eq_properties: F2, -) -> Option<&'a [PhysicalSortExpr]> { - if ordering_satisfy_concrete(req1, req2, &ordering_eq_properties) { - // Finer requirement is `provided`, since it satisfies the other: - return Some(req1); - } - if ordering_satisfy_concrete(req2, req1, &ordering_eq_properties) { - // Finer requirement is `req`, since it satisfies the other: - return Some(req2); - } - // Neither `provided` nor `req` satisfies one another, they are incompatible. - None -} - /// Scatter `truthy` array by boolean mask. When the mask evaluates `true`, next values of `truthy` /// are taken, when the mask evaluates `false` values null values are filled. /// @@ -632,13 +463,12 @@ pub fn get_indices_of_matching_sort_exprs_with_order_eq( }) .collect::>(); - let normalized_required = normalize_sort_requirements( - &sort_requirement_on_requirements, - &OrderingEquivalenceProperties::new(order_eq_properties.schema()), - ); - let normalized_provided = normalize_sort_requirements( + let oeq = OrderingEquivalenceProperties::new(order_eq_properties.schema()); + + let normalized_required = + oeq.normalize_sort_requirements(&sort_requirement_on_requirements); + let normalized_provided = oeq.normalize_sort_requirements( &PhysicalSortRequirement::from_sort_exprs(provided_sorts.iter()), - &OrderingEquivalenceProperties::new(order_eq_properties.schema()), ); let provided_sorts = normalized_provided @@ -664,24 +494,23 @@ pub fn get_indices_of_matching_sort_exprs_with_order_eq( } // We did not find all the expressions, consult ordering equivalence properties: - if let Some(oeq_class) = order_eq_properties.oeq_class() { - let head = oeq_class.head(); - for ordering in oeq_class.others().iter().chain(std::iter::once(head)) { - let order_eq_class_exprs = convert_to_expr(ordering); - if let Some(indices_of_equality) = get_lexicographical_match_indices( - &normalized_required_expr, - &order_eq_class_exprs, - ) { - return Some(( - indices_of_equality - .iter() - .map(|index| ordering[*index].options) - .collect(), - indices_of_equality, - )); - } + let oeq_class = order_eq_properties.oeq_group(); + for ordering in oeq_class.iter() { + let order_eq_class_exprs = convert_to_expr(ordering); + if let Some(indices_of_equality) = get_lexicographical_match_indices( + &normalized_required_expr, + &order_eq_class_exprs, + ) { + return Some(( + indices_of_equality + .iter() + .map(|index| ordering[*index].options) + .collect(), + indices_of_equality, + )); } } + // If no match found, return `None`: None } @@ -756,9 +585,9 @@ mod tests { use std::sync::Arc; use super::*; - use crate::equivalence::OrderingEquivalenceProperties; + use crate::equivalence::{OrderingEquivalenceProperties, OrderingEquivalentGroup}; use crate::expressions::{binary, cast, col, in_list, lit, Column, Literal}; - use crate::{OrderingEquivalentClass, PhysicalSortExpr}; + use crate::PhysicalSortExpr; use arrow::compute::SortOptions; use arrow_array::Int32Array; @@ -845,12 +674,12 @@ mod tests { let mut ordering_eq_properties = OrderingEquivalenceProperties::new(test_schema.clone()); ordering_eq_properties.add_equal_conditions((&col_a_expr, &col_c_expr)); - ordering_eq_properties.add_ordering_equal_conditions(( - &vec![PhysicalSortExpr { + ordering_eq_properties.add_new_orderings(&[ + vec![PhysicalSortExpr { expr: Arc::new(col_a.clone()), options: option1, }], - &vec![ + vec![ PhysicalSortExpr { expr: Arc::new(col_d.clone()), options: option1, @@ -860,13 +689,13 @@ mod tests { options: option1, }, ], - )); - ordering_eq_properties.add_ordering_equal_conditions(( - &vec![PhysicalSortExpr { + ]); + ordering_eq_properties.add_new_orderings(&[ + vec![PhysicalSortExpr { expr: Arc::new(col_a.clone()), options: option1, }], - &vec![ + vec![ PhysicalSortExpr { expr: Arc::new(col_e.clone()), options: option2, @@ -880,7 +709,7 @@ mod tests { options: option1, }, ], - )); + ]); Ok((test_schema, ordering_eq_properties)) } @@ -1036,161 +865,6 @@ mod tests { Ok(()) } - #[test] - fn test_ordering_satisfy() -> Result<()> { - let crude = vec![PhysicalSortExpr { - expr: Arc::new(Column::new("a", 0)), - options: SortOptions::default(), - }]; - let crude = Some(&crude[..]); - let finer = vec![ - PhysicalSortExpr { - expr: Arc::new(Column::new("a", 0)), - options: SortOptions::default(), - }, - PhysicalSortExpr { - expr: Arc::new(Column::new("b", 1)), - options: SortOptions::default(), - }, - ]; - let finer = Some(&finer[..]); - let empty_schema = &Arc::new(Schema::empty()); - assert!(ordering_satisfy(finer, crude, || { - OrderingEquivalenceProperties::new(empty_schema.clone()) - },)); - assert!(!ordering_satisfy(crude, finer, || { - OrderingEquivalenceProperties::new(empty_schema.clone()) - },)); - Ok(()) - } - - #[test] - fn test_ordering_satisfy_with_equivalence() -> Result<()> { - let col_a = &Column::new("a", 0); - let col_b = &Column::new("b", 1); - let col_c = &Column::new("c", 2); - let col_d = &Column::new("d", 3); - let col_e = &Column::new("e", 4); - let col_f = &Column::new("f", 5); - let col_g = &Column::new("g", 6); - let option1 = SortOptions { - descending: false, - nulls_first: false, - }; - let option2 = SortOptions { - descending: true, - nulls_first: true, - }; - // The schema is ordered by a ASC NULLS LAST, b ASC NULLS LAST - let provided = vec![ - PhysicalSortExpr { - expr: Arc::new(col_a.clone()), - options: option1, - }, - PhysicalSortExpr { - expr: Arc::new(col_b.clone()), - options: option1, - }, - ]; - let provided = Some(&provided[..]); - let (_test_schema, ordering_eq_properties) = create_test_params()?; - // First element in the tuple stores vector of requirement, second element is the expected return value for ordering_satisfy function - let requirements = vec![ - // `a ASC NULLS LAST`, expects `ordering_satisfy` to be `true`, since existing ordering `a ASC NULLS LAST, b ASC NULLS LAST` satisfies it - (vec![(col_a, option1)], true), - (vec![(col_a, option2)], false), - // Test whether equivalence works as expected - (vec![(col_c, option1)], true), - (vec![(col_c, option2)], false), - // Test whether ordering equivalence works as expected - (vec![(col_d, option1)], true), - (vec![(col_d, option1), (col_b, option1)], true), - (vec![(col_d, option2), (col_b, option1)], false), - ( - vec![(col_e, option2), (col_f, option1), (col_g, option1)], - true, - ), - (vec![(col_e, option2), (col_f, option1)], true), - (vec![(col_e, option1), (col_f, option1)], false), - (vec![(col_e, option2), (col_b, option1)], false), - (vec![(col_e, option1), (col_b, option1)], false), - ( - vec![ - (col_d, option1), - (col_b, option1), - (col_d, option1), - (col_b, option1), - ], - true, - ), - ( - vec![ - (col_d, option1), - (col_b, option1), - (col_e, option2), - (col_f, option1), - ], - true, - ), - ( - vec![ - (col_d, option1), - (col_b, option1), - (col_e, option2), - (col_b, option1), - ], - true, - ), - ( - vec![ - (col_d, option1), - (col_b, option1), - (col_d, option2), - (col_b, option1), - ], - true, - ), - ( - vec![ - (col_d, option1), - (col_b, option1), - (col_e, option1), - (col_f, option1), - ], - false, - ), - ( - vec![ - (col_d, option1), - (col_b, option1), - (col_e, option1), - (col_b, option1), - ], - false, - ), - (vec![(col_d, option1), (col_e, option2)], true), - ]; - - for (cols, expected) in requirements { - let err_msg = format!("Error in test case:{cols:?}"); - let required = cols - .into_iter() - .map(|(col, options)| PhysicalSortExpr { - expr: Arc::new(col.clone()), - options, - }) - .collect::>(); - - let required = Some(&required[..]); - assert_eq!( - ordering_satisfy(provided, required, || ordering_eq_properties.clone(),), - expected, - "{err_msg}" - ); - } - Ok(()) - } - fn convert_to_requirement( in_data: &[(&Column, Option)], ) -> Vec { @@ -1256,7 +930,7 @@ mod tests { let expected_normalized = convert_to_requirement(&expected_normalized); assert_eq!( - normalize_sort_requirements(&req, &ordering_eq_properties,), + ordering_eq_properties.normalize_sort_requirements(&req), expected_normalized ); } @@ -1322,9 +996,10 @@ mod tests { // Cannot normalize column b (&col_b_expr, &col_b_expr), ]; + let eq_groups = ordering_eq_properties.eq_groups(); for (expr, expected_eq) in expressions { assert!( - expected_eq.eq(&ordering_eq_properties.normalize_expr(expr.clone())), + expected_eq.eq(&eq_groups.normalize_expr(expr.clone())), "error in test: expr: {expr:?}" ); } @@ -1409,93 +1084,6 @@ mod tests { Ok(()) } - #[test] - fn test_ordering_satisfy_different_lengths() -> Result<()> { - let test_schema = create_test_schema()?; - let col_a_expr = col("a", &test_schema)?; - let col_b_expr = col("b", &test_schema)?; - let col_c_expr = col("c", &test_schema)?; - let col_d_expr = col("d", &test_schema)?; - let col_e_expr = col("e", &test_schema)?; - let option1 = SortOptions { - descending: false, - nulls_first: false, - }; - // Column a and c are aliases. - let mut ordering_eq_properties = OrderingEquivalenceProperties::new(test_schema); - ordering_eq_properties.add_equal_conditions((&col_a_expr, &col_c_expr)); - - // Column a and e are ordering equivalent (e.g global ordering of the table can be described both as a ASC and e ASC.) - ordering_eq_properties.add_ordering_equal_conditions(( - &vec![PhysicalSortExpr { - expr: col_a_expr.clone(), - options: option1, - }], - &vec![PhysicalSortExpr { - expr: col_e_expr.clone(), - options: option1, - }], - )); - let sort_req_a = PhysicalSortExpr { - expr: col_a_expr.clone(), - options: option1, - }; - let sort_req_b = PhysicalSortExpr { - expr: col_b_expr.clone(), - options: option1, - }; - let sort_req_c = PhysicalSortExpr { - expr: col_c_expr.clone(), - options: option1, - }; - let sort_req_d = PhysicalSortExpr { - expr: col_d_expr.clone(), - options: option1, - }; - let sort_req_e = PhysicalSortExpr { - expr: col_e_expr.clone(), - options: option1, - }; - - assert!(ordering_satisfy_concrete( - // After normalization would be a ASC, b ASC, d ASC - &[sort_req_a.clone(), sort_req_b.clone(), sort_req_d.clone()], - // After normalization would be a ASC, b ASC, d ASC - &[ - sort_req_c.clone(), - sort_req_b.clone(), - sort_req_a.clone(), - sort_req_d.clone(), - sort_req_e.clone(), - ], - || ordering_eq_properties.clone(), - )); - - assert!(!ordering_satisfy_concrete( - // After normalization would be a ASC, b ASC - &[sort_req_a.clone(), sort_req_b.clone()], - // After normalization would be a ASC, b ASC, d ASC - &[ - sort_req_c.clone(), - sort_req_b.clone(), - sort_req_a.clone(), - sort_req_d.clone(), - sort_req_e.clone(), - ], - || ordering_eq_properties.clone(), - )); - - assert!(!ordering_satisfy_concrete( - // After normalization would be a ASC, b ASC, d ASC - &[sort_req_a.clone(), sort_req_b.clone(), sort_req_d.clone()], - // After normalization would be a ASC, d ASC, b ASC - &[sort_req_c, sort_req_d, sort_req_a, sort_req_b, sort_req_e,], - || ordering_eq_properties.clone(), - )); - - Ok(()) - } - #[test] fn test_collect_columns() -> Result<()> { let expr1 = Arc::new(Column::new("col1", 2)) as _; @@ -1628,12 +1216,12 @@ mod tests { OrderingEquivalenceProperties::new(Arc::new(schema.clone())); let mut ordering_equal_properties = OrderingEquivalenceProperties::new(Arc::new(schema)); - ordering_equal_properties.add_ordering_equal_conditions(( - &vec![PhysicalSortExpr { + ordering_equal_properties.add_new_orderings(&[ + vec![PhysicalSortExpr { expr: Arc::new(Column::new("c", 2)), options: sort_options, }], - &vec![ + vec![ PhysicalSortExpr { expr: Arc::new(Column::new("b", 1)), options: sort_options_not, @@ -1643,7 +1231,7 @@ mod tests { options: sort_options, }, ], - )); + ]); assert_eq!( get_indices_of_matching_sort_exprs_with_order_eq( &provided_sorts, @@ -1704,34 +1292,33 @@ mod tests { OrderingEquivalenceProperties::new(Arc::new(schema.clone())); equal_properties.add_equal_conditions((&col_a_expr, &col_c_expr)); - let head = vec![PhysicalSortExpr { - expr: col_b_expr.clone(), - options: sort_options, - }]; - let others = vec![vec![PhysicalSortExpr { - expr: col_c_expr.clone(), - options: sort_options, - }]]; - equal_properties.extend(Some(OrderingEquivalentClass::new(head, others))); + let others = vec![ + vec![PhysicalSortExpr { + expr: col_b_expr.clone(), + options: sort_options, + }], + vec![PhysicalSortExpr { + expr: col_c_expr.clone(), + options: sort_options, + }], + ]; + equal_properties.extend(OrderingEquivalentGroup::new(others)); let mut expected_oeq = OrderingEquivalenceProperties::new(Arc::new(schema)); - expected_oeq.add_ordering_equal_conditions(( - &vec![PhysicalSortExpr { + expected_oeq.add_new_orderings(&[ + vec![PhysicalSortExpr { expr: col_b_expr.clone(), options: sort_options, }], - &vec![PhysicalSortExpr { + vec![PhysicalSortExpr { expr: col_a_expr.clone(), options: sort_options, }], - )); + ]); - let oeq_class = equal_properties.oeq_class().unwrap(); - let expected = expected_oeq.oeq_class().unwrap(); - assert!( - oeq_class.head().eq(expected.head()) - && oeq_class.others().eq(expected.others()) - ); + let oeq_class = equal_properties.oeq_group().clone(); + let expected = expected_oeq.oeq_group(); + assert!(oeq_class.eq(expected)); Ok(()) } diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index a986901f2f31..87eea3cdd4ed 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -57,9 +57,6 @@ pub use datafusion_expr::AggregateFunction; use datafusion_physical_expr::aggregate::is_order_sensitive; pub use datafusion_physical_expr::expressions::create_aggregate_expr; use datafusion_physical_expr::expressions::{Max, Min}; -use datafusion_physical_expr::utils::{ - get_finer_ordering, ordering_satisfy_requirement_concrete, -}; use super::DisplayAs; @@ -425,16 +422,17 @@ fn get_finest_requirement OrderingEquivalenceProperties>( ordering_eq_properties: F2, ) -> Result> { let mut finest_req = get_init_req(aggr_expr, order_by_expr); + let oeq_properties = ordering_eq_properties(); for (aggr_expr, fn_req) in aggr_expr.iter_mut().zip(order_by_expr.iter_mut()) { let fn_req = if let Some(fn_req) = fn_req { fn_req } else { continue; }; + if let Some(finest_req) = &mut finest_req { - if let Some(finer) = - get_finer_ordering(finest_req, fn_req, &ordering_eq_properties) - { + println!("finest_req{:?}, fn_req:{:?}", finest_req, fn_req); + if let Some(finer) = oeq_properties.get_finer_ordering(finest_req, fn_req) { *finest_req = finer.to_vec(); continue; } @@ -442,11 +440,9 @@ fn get_finest_requirement OrderingEquivalenceProperties>( // direction is compatible with existing requirements: if let Some(reverse) = aggr_expr.reverse_expr() { let fn_req_reverse = reverse_order_bys(fn_req); - if let Some(finer) = get_finer_ordering( - finest_req, - &fn_req_reverse, - &ordering_eq_properties, - ) { + if let Some(finer) = + oeq_properties.get_finer_ordering(finest_req, &fn_req_reverse) + { // We need to update `aggr_expr` with its reverse, since only its // reverse requirement is compatible with existing requirements: *aggr_expr = reverse; @@ -550,12 +546,10 @@ fn calc_required_input_ordering( // plan. If neither version satisfies the existing ordering, we use the given ordering // requirement. In short, if running aggregators in reverse order help us to avoid a // sorting step, we do so. Otherwise, we use the aggregators as is. - let existing_ordering = input.output_ordering().unwrap_or(&[]); - if ordering_satisfy_requirement_concrete( - existing_ordering, - &required_input_ordering, - || input.ordering_equivalence_properties(), - ) { + if input + .ordering_equivalence_properties() + .ordering_satisfy_requirement_concrete(&required_input_ordering) + { break; } } @@ -994,11 +988,9 @@ impl ExecutionPlan for AggregateExec { // } fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { - self.input.ordering_equivalence_properties().project( - &self.columns_map, - &self.source_to_target_mapping, - self.schema(), - ) + self.input + .ordering_equivalence_properties() + .project(&self.source_to_target_mapping, self.schema()) } fn children(&self) -> Vec> { @@ -2318,19 +2310,22 @@ mod tests { let col_d = Column::new("d", 3); let col_a_expr = (Arc::new(col_a.clone()) as Arc); let col_b_expr = (Arc::new(col_b.clone()) as Arc); - // eq_properties.add_equal_conditions((&col_a, &col_b)); let mut ordering_eq_properties = OrderingEquivalenceProperties::new(test_schema); + // Columns a and b are equal. ordering_eq_properties.add_equal_conditions((&col_a_expr, &col_b_expr)); - ordering_eq_properties.add_ordering_equal_conditions(( - &vec![PhysicalSortExpr { + // [a ASC], [c DESC] describes ordering of the schema. + ordering_eq_properties.add_new_orderings(&[ + vec![PhysicalSortExpr { expr: Arc::new(col_a.clone()) as _, options: options1, }], - &vec![PhysicalSortExpr { + vec![PhysicalSortExpr { expr: Arc::new(col_c.clone()) as _, options: options2, }], - )); + ]); + // Aggregate requirements are + // [None], [a ASC], [b ASC], [c DESC], [a ASC, d ASC] respectively let mut order_by_exprs = vec![ None, Some(vec![PhysicalSortExpr { diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index eb4e96805807..0c805496068a 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -163,7 +163,9 @@ impl ExecutionPlan for FilterExec { .map(|column| Arc::new(column) as Arc) .collect::>(); let filter_oeq = self.input.ordering_equivalence_properties(); - filter_oeq.with_constants(constants) + let res = filter_oeq.with_constants(constants); + println!("filter res:{:?}", res); + res } else { self.input.ordering_equivalence_properties() } diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index a5fd704c5d5f..a5960e6e6635 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -364,18 +364,6 @@ impl ExecutionPlan for HashJoinExec { Self::maintains_input_order(self.join_type) } - // fn equivalence_properties(&self) -> EquivalenceProperties { - // let left_columns_len = self.left.schema().fields.len(); - // combine_join_equivalence_properties( - // self.join_type, - // self.left.equivalence_properties(), - // self.right.equivalence_properties(), - // left_columns_len, - // self.on(), - // self.schema(), - // ) - // } - fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { combine_join_ordering_equivalence_properties( &self.join_type, @@ -421,7 +409,11 @@ impl ExecutionPlan for HashJoinExec { // for child in self.children(){ // println!("hash join child ordering_equivalence_properties()\n{:?}", child.ordering_equivalence_properties()); // } - // println!("hash join self.ordering_equivalence_properties()\n{:?}", self.ordering_equivalence_properties()); + println!( + "hash join self.ordering_equivalence_properties()\n{:?}", + self.ordering_equivalence_properties() + ); + println!("hash join output ordering: {:?}", self.output_ordering()); if self.mode == PartitionMode::Partitioned && left_partitions != right_partitions { diff --git a/datafusion/physical-plan/src/joins/utils.rs b/datafusion/physical-plan/src/joins/utils.rs index 8b6b87a4b6fb..f1ddacde47b9 100644 --- a/datafusion/physical-plan/src/joins/utils.rs +++ b/datafusion/physical-plan/src/joins/utils.rs @@ -44,11 +44,12 @@ use datafusion_common::{ use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::{ add_offset_to_lex_ordering, LexOrdering, LexOrderingRef, - OrderingEquivalenceProperties, OrderingEquivalentClass, PhysicalExpr, - PhysicalSortExpr, + OrderingEquivalenceProperties, PhysicalExpr, PhysicalSortExpr, }; -use datafusion_physical_expr::equivalence::combine_join_equivalence_properties2; +use datafusion_physical_expr::equivalence::{ + add_offset, combine_join_equivalence_properties2, OrderingEquivalentGroup, +}; use datafusion_physical_expr::utils::merge_vectors; use futures::future::{BoxFuture, Shared}; use futures::{ready, FutureExt}; @@ -228,136 +229,6 @@ pub fn calculate_join_output_ordering( Ok((!output_ordering.is_empty()).then_some(output_ordering)) } -// /// Combine equivalence properties of the given join inputs. -// pub fn combine_join_equivalence_properties( -// join_type: JoinType, -// left_properties: EquivalenceProperties, -// right_properties: EquivalenceProperties, -// left_columns_len: usize, -// on: &[(Column, Column)], -// schema: SchemaRef, -// ) -> EquivalenceProperties { -// let mut new_properties = EquivalenceProperties::new(schema); -// match join_type { -// JoinType::Inner | JoinType::Left | JoinType::Full | JoinType::Right => { -// new_properties.extend(left_properties.classes().to_vec()); -// let new_right_properties = right_properties -// .classes() -// .iter() -// .map(|prop| { -// let new_head = Column::new( -// prop.head().name(), -// left_columns_len + prop.head().index(), -// ); -// let new_others = prop -// .others() -// .iter() -// .map(|col| { -// Column::new(col.name(), left_columns_len + col.index()) -// }) -// .collect::>(); -// EquivalentClass::new(new_head, new_others) -// }) -// .collect::>(); -// -// new_properties.extend(new_right_properties); -// } -// JoinType::LeftSemi | JoinType::LeftAnti => { -// new_properties.extend(left_properties.classes().to_vec()) -// } -// JoinType::RightSemi | JoinType::RightAnti => { -// new_properties.extend(right_properties.classes().to_vec()) -// } -// } -// -// if join_type == JoinType::Inner { -// on.iter().for_each(|(column1, column2)| { -// let new_column2 = -// Column::new(column2.name(), left_columns_len + column2.index()); -// new_properties.add_equal_conditions((column1, &new_column2)) -// }) -// } -// new_properties -// } - -// /// Combine equivalence properties of the given join inputs. -// pub fn combine_join_equivalence_properties2( -// join_type: JoinType, -// left_eq_classes: &[Vec>], -// right_eq_classes: &[Vec>], -// left_columns_len: usize, -// on: &[(Column, Column)], -// ) -> Vec>> { -// let mut res = vec![]; -// match join_type { -// JoinType::Inner | JoinType::Left | JoinType::Full | JoinType::Right => { -// res.extend(left_eq_classes.to_vec()); -// -// let new_right_properties = right_eq_classes -// .iter() -// .map(|prop| { -// let new_head = Column::new( -// prop.head().name(), -// left_columns_len + prop.head().index(), -// ); -// let new_others = prop -// .others() -// .iter() -// .map(|col| { -// Column::new(col.name(), left_columns_len + col.index()) -// }) -// .collect::>(); -// EquivalentClass::new(new_head, new_others) -// }) -// .collect::>(); -// -// new_properties.extend(new_right_properties); -// } -// JoinType::LeftSemi | JoinType::LeftAnti => { -// new_properties.extend(left_properties.classes().to_vec()) -// } -// JoinType::RightSemi | JoinType::RightAnti => { -// new_properties.extend(right_properties.classes().to_vec()) -// } -// } -// -// if join_type == JoinType::Inner { -// on.iter().for_each(|(column1, column2)| { -// let new_column2 = -// Column::new(column2.name(), left_columns_len + column2.index()); -// new_properties.add_equal_conditions((column1, &new_column2)) -// }) -// } -// new_properties -// } - -// /// Calculate equivalence properties for the given cross join operation. -// pub fn cross_join_equivalence_properties( -// left_properties: EquivalenceProperties, -// right_properties: EquivalenceProperties, -// left_columns_len: usize, -// schema: SchemaRef, -// ) -> EquivalenceProperties { -// let mut new_properties = EquivalenceProperties::new(schema); -// new_properties.extend(left_properties.classes().to_vec()); -// let new_right_properties = right_properties -// .classes() -// .iter() -// .map(|prop| { -// let new_head = -// Column::new(prop.head().name(), left_columns_len + prop.head().index()); -// let new_others = prop -// .others() -// .iter() -// .map(|col| Column::new(col.name(), left_columns_len + col.index())) -// .collect::>(); -// EquivalentClass::new(new_head, new_others) -// }) -// .collect::>(); -// new_properties.extend(new_right_properties); -// new_properties -// } - /// Update right table ordering equivalences so that: /// - They point to valid indices at the output of the join schema, and /// - They are normalized with respect to equivalence columns. @@ -374,14 +245,14 @@ pub fn calculate_join_output_ordering( /// it can thereafter safely be used for ordering equivalence normalization. fn get_updated_right_ordering_equivalent_class( join_type: &JoinType, - right_oeq_class: &OrderingEquivalentClass, + right_oeq_class: &OrderingEquivalentGroup, left_columns_len: usize, -) -> Result { +) -> Result { match join_type { // In these modes, indices of the right schema should be offset by // the left table size. JoinType::Inner | JoinType::Left | JoinType::Full | JoinType::Right => { - let right_oeq_class = right_oeq_class.add_offset(left_columns_len)?; + let right_oeq_class = add_offset(&right_oeq_class, left_columns_len)?; return Ok( // right_oeq_class.normalize_with_equivalence_properties(join_eq_properties) right_oeq_class, @@ -405,37 +276,25 @@ pub fn combine_join_ordering_equivalence_properties( on: &[(Column, Column)], ) -> Result { // println!("combine join start"); - let n_schema = schema.fields.len(); let mut new_properties = OrderingEquivalenceProperties::new(schema); let left_columns_len = left.schema().fields.len(); let left_oeq_properties = left.ordering_equivalence_properties(); let right_oeq_properties = right.ordering_equivalence_properties(); - // if n_schema > left.schema().fields.len() && n_schema > right.schema().fields.len() { - // on.iter().for_each(|(lhs, rhs)| { - // let new_lhs = Arc::new(lhs.clone()) as _; - // let new_rhs = - // Arc::new(Column::new(rhs.name(), rhs.index() + left_columns_len)) as _; - // // (new_lhs, new_rhs) - // // println!("new_lhs: {:?}, new_rhs: {:?}", new_lhs, new_rhs); - // new_properties.add_equal_conditions((&new_lhs, &new_rhs)); - // }); - // // println!("on: {:?}", updated_on); - // // new_properties.add_equal_conditions() - // } combine_join_equivalence_properties2( join_type, - &left_oeq_properties.eq_classes(), - &right_oeq_properties.eq_classes(), + left_oeq_properties.eq_groups(), + right_oeq_properties.eq_groups(), left_columns_len, on, &mut new_properties, )?; - // println!("new_eq_classes join before: {:?}", new_properties.eq_classes()); // All joins have 2 children assert_eq!(maintains_input_order.len(), 2); let left_maintains = maintains_input_order[0]; let right_maintains = maintains_input_order[1]; + let left_oeq_class = left_oeq_properties.oeq_group(); + let right_oeq_class = right_oeq_properties.oeq_group(); match (left_maintains, right_maintains) { (true, true) => { return Err(DataFusionError::Plan( @@ -443,25 +302,11 @@ pub fn combine_join_ordering_equivalence_properties( )) } (true, false) => { - new_properties.extend(left_oeq_properties.oeq_class().cloned()); // In this special case, right side ordering can be prefixed with left side ordering. - if let ( - Some(JoinSide::Left), - // right side have an ordering - Some(_), - JoinType::Inner, - Some(oeq_class), - ) = ( - probe_side, - right.output_ordering(), - join_type, - right_oeq_properties.oeq_class(), - ) { - let left_output_ordering = left.output_ordering().unwrap_or(&[]); - + if let (Some(JoinSide::Left), JoinType::Inner) = (probe_side, join_type) { let updated_right_oeq = get_updated_right_ordering_equivalent_class( join_type, - oeq_class, + right_oeq_class, left_columns_len, )?; @@ -473,42 +318,20 @@ pub fn combine_join_ordering_equivalence_properties( // ordering of the left table is `a ASC`, then the ordering equivalence `b ASC` // for the right table should be converted to `a ASC, b ASC` before it is added // to the ordering equivalences of the join. - let updated_right_oeq_class = updated_right_oeq - .prefix_ordering_equivalent_class_with_existing_ordering( - left_output_ordering, - ); - new_properties.extend(Some(updated_right_oeq_class)); + let out_oeq_class = left_oeq_class.join_postfix(&updated_right_oeq); + new_properties.extend(out_oeq_class); + } else { + new_properties.extend(left_oeq_class.clone()); } } (false, true) => { - let updated_right_oeq = right_oeq_properties - .oeq_class() - .map(|right_oeq_class| { - get_updated_right_ordering_equivalent_class( - join_type, - right_oeq_class, - left_columns_len, - ) - }) - .transpose()?; - new_properties.extend(updated_right_oeq); - // In this special case, left side ordering can be prefixed with right side ordering. - if let ( - Some(JoinSide::Right), - // left side have an ordering - Some(_), - JoinType::Inner, - Some(left_oeq_class), - ) = ( - probe_side, - left.output_ordering(), + let updated_right_oeq = get_updated_right_ordering_equivalent_class( join_type, - left_oeq_properties.oeq_class(), - ) { - let right_output_ordering = right.output_ordering().unwrap_or(&[]); - let right_output_ordering = - add_offset_to_lex_ordering(right_output_ordering, left_columns_len)?; - + right_oeq_properties.oeq_group(), + left_columns_len, + )?; + // In this special case, left side ordering can be prefixed with right side ordering. + if let (Some(JoinSide::Right), JoinType::Inner) = (probe_side, join_type) { // Left side ordering equivalence properties should be prepended with // those of the right side while constructing output ordering equivalence // properties since stream side is the right side. @@ -517,11 +340,10 @@ pub fn combine_join_ordering_equivalence_properties( // ordering of the left table is `a ASC`, then the ordering equivalence `b ASC` // for the right table should be converted to `a ASC, b ASC` before it is added // to the ordering equivalences of the join. - let updated_left_oeq_class = left_oeq_class - .prefix_ordering_equivalent_class_with_existing_ordering( - &right_output_ordering, - ); - new_properties.extend(Some(updated_left_oeq_class)); + let out_oeq_class = updated_right_oeq.join_postfix(left_oeq_class); + new_properties.extend(out_oeq_class); + } else { + new_properties.extend(updated_right_oeq); } } (false, false) => {} @@ -1913,7 +1735,7 @@ mod tests { let join_type = JoinType::Inner; let options = SortOptions::default(); - let right_oeq_class = OrderingEquivalentClass::new( + let right_oeq_class = OrderingEquivalentGroup::new(vec![ vec![ PhysicalSortExpr { expr: Arc::new(Column::new("x", 0)), @@ -1924,7 +1746,7 @@ mod tests { options, }, ], - vec![vec![ + vec![ PhysicalSortExpr { expr: Arc::new(Column::new("z", 2)), options, @@ -1933,8 +1755,8 @@ mod tests { expr: Arc::new(Column::new("w", 3)), options, }, - ]], - ); + ], + ]); let left_columns_len = 4; @@ -1961,10 +1783,10 @@ mod tests { &right_oeq_class, left_columns_len, )?; - join_eq_properties.extend(Some(result)); - let result = join_eq_properties.oeq_class().unwrap(); + join_eq_properties.extend(result); + let result = join_eq_properties.oeq_group().clone(); - let expected = OrderingEquivalentClass::new( + let expected = OrderingEquivalentGroup::new(vec![ vec![ PhysicalSortExpr { expr: col_a_expr, @@ -1975,7 +1797,7 @@ mod tests { options, }, ], - vec![vec![ + vec![ PhysicalSortExpr { expr: col_z_expr, options, @@ -1984,11 +1806,10 @@ mod tests { expr: col_d_expr, options, }, - ]], - ); + ], + ]); - assert_eq!(result.head(), expected.head()); - assert_eq!(result.others(), expected.others()); + assert_eq!(result, expected); Ok(()) } diff --git a/datafusion/physical-plan/src/memory.rs b/datafusion/physical-plan/src/memory.rs index b29c8e9c7bd9..6ca06809614c 100644 --- a/datafusion/physical-plan/src/memory.rs +++ b/datafusion/physical-plan/src/memory.rs @@ -33,6 +33,7 @@ use std::task::{Context, Poll}; use crate::ordering_equivalence_properties_helper; use datafusion_common::DataFusionError; use datafusion_execution::TaskContext; +use datafusion_physical_expr::equivalence::OrderingEquivalentGroup; use datafusion_physical_expr::{LexOrdering, OrderingEquivalenceProperties}; use futures::Stream; @@ -295,10 +296,7 @@ mod tests { assert_eq!(mem_exec.output_ordering().unwrap(), expected_output_order); let order_eq = mem_exec.ordering_equivalence_properties(); - assert!(order_eq - .oeq_class() - .map(|class| class.contains(&expected_order_eq)) - .unwrap_or(false)); + assert!(order_eq.oeq_group().contains(&expected_order_eq)); Ok(()) } } diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index 914948be56ef..5fc7d26bf985 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -140,28 +140,6 @@ impl ProjectionExec { }; } - // Output Ordering need to respect the alias - let child_output_ordering = input.output_ordering(); - let output_ordering = match child_output_ordering { - Some(sort_exprs) => { - let normalized_exprs = sort_exprs - .iter() - .map(|sort_expr| { - let expr = normalize_out_expr_with_columns_map( - sort_expr.expr.clone(), - &columns_map, - ); - PhysicalSortExpr { - expr, - options: sort_expr.options, - } - }) - .collect::>(); - Some(normalized_exprs) - } - None => None, - }; - let orderings = find_orderings_of_exprs( &expr, input.output_ordering(), @@ -177,15 +155,13 @@ impl ProjectionExec { // let output_ordering = // validate_output_ordering(output_ordering, &orderings, &expr); - let output_ordering = if let Some(oeq_class) = input - .ordering_equivalence_properties() - .project(&columns_map, &source_to_target_mapping, schema.clone()) - .oeq_class() - { - Some(oeq_class.head().to_vec()) - } else { - None - }; + let input_oeq = input.ordering_equivalence_properties(); + // println!("input.ordering_equivalence_properties(): {:?}", input_oeq); + let project_oeq = input_oeq.project(&source_to_target_mapping, schema.clone()); + let project_orderings = project_oeq.oeq_group(); + // println!("project_oeq: {:?}", project_oeq); + let output_ordering = project_orderings.output_ordering(); + // println!("output ordering:{:?}", output_ordering); Ok(Self { expr, @@ -279,13 +255,6 @@ impl ExecutionPlan for ProjectionExec { } fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - // let oeq = self.ordering_equivalence_properties(); - // if let Some(oeq_class) = oeq.oeq_class(){ - // Some(oeq_class.head()) - // } else{ - // None - // } - // oeq.oeq_class().map(|oeq_class| &(oeq_class.head().to_vec())) self.output_ordering.as_deref() } @@ -294,57 +263,10 @@ impl ExecutionPlan for ProjectionExec { vec![true] } - // fn equivalence_properties(&self) -> EquivalenceProperties { - // let mut new_properties = EquivalenceProperties::new(self.schema()); - // project_equivalence_properties( - // self.input.equivalence_properties(), - // &self.columns_map, - // &mut new_properties, - // ); - // new_properties - // } - - // fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { - // let mut new_properties = OrderingEquivalenceProperties::new(self.schema()); - // if self.output_ordering.is_none() { - // // If there is no output ordering, return an "empty" equivalence set: - // return new_properties; - // } - // - // let input_oeq = self.input().ordering_equivalence_properties(); - // - // project_ordering_equivalence_properties( - // input_oeq, - // &self.columns_map, - // &mut new_properties, - // ); - // - // if let Some(leading_ordering) = self - // .output_ordering - // .as_ref() - // .map(|output_ordering| &output_ordering[0]) - // { - // for order in self.orderings.iter().flatten() { - // if !order.eq(leading_ordering) - // && !new_properties.satisfies_leading_ordering(order) - // { - // new_properties.add_ordering_equal_conditions(( - // &vec![leading_ordering.clone()], - // &vec![order.clone()], - // )); - // } - // } - // } - // - // new_properties - // } - fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { - self.input.ordering_equivalence_properties().project( - &self.columns_map, - &self.source_to_target_mapping, - self.schema(), - ) + self.input + .ordering_equivalence_properties() + .project(&self.source_to_target_mapping, self.schema()) } fn with_new_children( diff --git a/datafusion/physical-plan/src/union.rs b/datafusion/physical-plan/src/union.rs index af765e257db2..19c2f81b7a03 100644 --- a/datafusion/physical-plan/src/union.rs +++ b/datafusion/physical-plan/src/union.rs @@ -46,6 +46,7 @@ use crate::stream::ObservedStream; use crate::{expressions, metrics::BaselineMetrics}; use datafusion_common::Result; use datafusion_execution::TaskContext; +use datafusion_physical_expr::OrderingEquivalenceProperties; use tokio::macros::support::thread_rng_n; /// `UnionExec`: `UNION ALL` execution plan. @@ -224,6 +225,26 @@ impl ExecutionPlan for UnionExec { } } + fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { + let child_oeqs = self + .inputs + .iter() + .map(|child| child.ordering_equivalence_properties().oeq_group().clone()) + .collect::>(); + let first_oeq = child_oeqs[0].clone(); + let mut union_oeq = OrderingEquivalenceProperties::new(self.schema()); + for elem in first_oeq.iter() { + if child_oeqs.iter().all(|child_oeq| child_oeq.contains(elem)) { + // res.push(elem); + // Search meet instead of exact + union_oeq.add_new_orderings(&[elem.clone()]) + } + } + // let mut union_oeq = OrderingEquivalenceProperties::new(self.schema()); + // union_oeq.add_ordering_equal_conditions() + union_oeq + } + fn with_new_children( self: Arc, children: Vec>, diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 3d9f7511be26..527300824954 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -1957,7 +1957,7 @@ Sort: aggregate_test_100.c1 ASC NULLS LAST ----WindowAggr: windowExpr=[[ROW_NUMBER() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING]] ------TableScan: aggregate_test_100 projection=[c1] physical_plan -SortPreservingMergeExec: [c1@0 ASC NULLS LAST] +SortPreservingMergeExec: [c1@0 ASC NULLS LAST,rn1@1 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() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "ROW_NUMBER() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", 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)) }], mode=[Sorted] ------SortExec: expr=[c1@0 ASC NULLS LAST] From f2a46000169ff52c9cad66951400ccef74b5152d Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Mon, 2 Oct 2023 10:49:55 +0300 Subject: [PATCH 006/122] Simplifications --- .../combine_partial_final_agg.rs | 2 +- .../src/physical_optimizer/enforce_sorting.rs | 3 +- datafusion/core/src/physical_planner.rs | 2 +- datafusion/physical-expr/src/equivalence.rs | 166 ++++-------------- datafusion/physical-expr/src/physical_expr.rs | 9 +- datafusion/physical-expr/src/utils.rs | 6 +- .../physical-plan/src/aggregates/mod.rs | 7 +- datafusion/physical-plan/src/filter.rs | 19 +- datafusion/physical-plan/src/memory.rs | 1 - datafusion/physical-plan/src/projection.rs | 49 +----- datafusion/physical-plan/src/sorts/sort.rs | 10 +- 11 files changed, 66 insertions(+), 208 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs b/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs index 0fbabef323f7..13d4cd161e4c 100644 --- a/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs +++ b/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs @@ -43,7 +43,7 @@ impl CombinePartialFinalAggregate { } } -fn print_plan(plan: &Arc) -> () { +fn print_plan(plan: &Arc) { let formatted = crate::physical_plan::displayable(plan.as_ref()) .indent(true) .to_string(); diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index 686fb60851db..e94faf84c275 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -310,7 +310,7 @@ impl TreeNode for PlanWithCorrespondingCoalescePartitions { } } -fn print_plan(plan: &Arc) -> () { +fn print_plan(plan: &Arc) { let formatted = crate::physical_plan::displayable(plan.as_ref()) .indent(true) .to_string(); @@ -3692,5 +3692,4 @@ mod tmp_tests { print_batches(&actual)?; Ok(()) } - } diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 793a2730a750..86595966f97f 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -2791,7 +2791,7 @@ digraph { } } -fn print_plan(plan: &Arc) -> () { +fn print_plan(plan: &Arc) { let formatted = crate::physical_plan::displayable(plan.as_ref()) .indent(true) .to_string(); diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 0266b79fc766..9b0bc1595058 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -16,14 +16,14 @@ // under the License. use crate::expressions::{CastExpr, Column}; -use crate::utils::{collect_columns, get_indices_of_matching_exprs}; +use crate::utils::get_indices_of_matching_exprs; use crate::{ physical_exprs_contains, LexOrdering, LexOrderingRef, LexOrderingReq, PhysicalExpr, PhysicalSortExpr, PhysicalSortRequirement, }; use arrow::datatypes::SchemaRef; -use arrow_schema::{Fields, SortOptions}; +use arrow_schema::SortOptions; use crate::physical_expr::{deduplicate_physical_exprs, have_common_entries}; use crate::sort_properties::{ExprOrdering, SortProperties}; @@ -31,7 +31,6 @@ use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::utils::longest_consecutive_prefix; use datafusion_common::{JoinType, Result}; use itertools::izip; -use petgraph::visit::Walker; use std::hash::Hash; use std::ops::Range; use std::sync::Arc; @@ -63,6 +62,7 @@ impl EquivalentGroups { self.inner.iter() } + #[allow(dead_code)] fn into_iter(self) -> impl Iterator>> { self.inner.into_iter() } @@ -133,7 +133,7 @@ impl EquivalentGroups { // Delete groups in the `out_groups` that have common entry with `group`. // Append deleted groups to the `bridged_group` out_groups.retain(|distinct_group| { - let have_common = have_common_entries(distinct_group, &group); + let have_common = have_common_entries(distinct_group, group); if have_common { bridged_group.extend(distinct_group.clone()); } @@ -147,8 +147,7 @@ impl EquivalentGroups { self.inner = out_groups; } - // TODO: Add a method for bridging equalities - + #[allow(dead_code)] fn len(&self) -> usize { self.inner.len() } @@ -351,7 +350,7 @@ impl OrderingEquivalentGroup { /// Adds new ordering into the ordering equivalent group. pub fn add_new_orderings(&mut self, orderings: &[LexOrdering]) { for ordering in orderings.iter() { - if !self.contains(&ordering) { + if !self.contains(ordering) { self.push(ordering.clone()); } } @@ -364,7 +363,7 @@ impl OrderingEquivalentGroup { for ordering in self.iter() { let mut is_inside = false; for item in &mut res { - if let Some(finer) = get_finer(item, &ordering) { + if let Some(finer) = get_finer(item, ordering) { *item = finer; is_inside = true; } @@ -388,6 +387,7 @@ impl OrderingEquivalentGroup { self.inner.into_iter() } + #[allow(dead_code)] fn len(&self) -> usize { self.inner.len() } @@ -402,11 +402,7 @@ impl OrderingEquivalentGroup { } pub fn output_ordering(&self) -> Option> { - if let Some(first) = self.inner.first() { - Some(first.clone()) - } else { - None - } + self.inner.first().cloned() } // Append other as postfix to existing ordering equivalences @@ -508,7 +504,7 @@ impl OrderingEquivalenceProperties { .inner .iter() .map(|ordering| { - let ordering = self.eq_groups.normalize_sort_exprs(&ordering); + let ordering = self.eq_groups.normalize_sort_exprs(ordering); let req = prune_sort_reqs_with_constants( &PhysicalSortRequirement::from_sort_exprs(&ordering), &self.constants, @@ -627,9 +623,10 @@ impl OrderingEquivalenceProperties { } normalized_sort_reqs = simplify_lex_req(normalized_sort_reqs, &self.oeq_group); - let res = collapse_lex_req(normalized_sort_reqs); - // println!("normalzied sort_reqs:{:?}", res); - res + // let res = collapse_lex_req(normalized_sort_reqs); + // // println!("normalzied sort_reqs:{:?}", res); + // res + collapse_lex_req(normalized_sort_reqs) } /// Checks whether `leading_ordering` is contained in any of the ordering @@ -756,7 +753,7 @@ impl OrderingEquivalenceProperties { for (source, target) in source_to_target_mapping { let initial_expr = ExprOrdering::new(source.clone()); let transformed = initial_expr - .transform_up(&|expr| update_ordering(expr, &self)) + .transform_up(&|expr| update_ordering(expr, self)) .unwrap(); if let Some(SortProperties::Ordered(sort_options)) = transformed.state { let sort_expr = PhysicalSortExpr { @@ -876,12 +873,18 @@ impl OrderingEquivalenceProperties { if required_normalized.len() > provided_normalized.len() { return false; } - let res = required_normalized + + // let res = required_normalized + // .into_iter() + // .zip(provided_normalized) + // .all(|(req, given)| given == req); + // // println!("res:{:?}", res); + // res + + required_normalized .into_iter() .zip(provided_normalized) - .all(|(req, given)| given == req); - // println!("res:{:?}", res); - res + .all(|(req, given)| given == req) } /// Find the finer requirement among `req1` and `req2` @@ -1013,53 +1016,6 @@ impl OrderingEquivalenceProperties { type ProjectionMapping = Vec<(Arc, Arc)>; -/// Update each expression in `ordering` with alias expressions. Assume -/// `ordering` is `a ASC, b ASC` and `c` is alias of `b`. Then, the result -/// will be `a ASC, c ASC`. -fn update_with_alias( - mut ordering: LexOrdering, - oeq_alias_map: &[(Column, Column)], -) -> LexOrdering { - for (source_col, target_col) in oeq_alias_map { - let source_col: Arc = Arc::new(source_col.clone()); - // Replace invalidated columns with its alias in the ordering expression. - let target_col: Arc = Arc::new(target_col.clone()); - for item in ordering.iter_mut() { - if item.expr.eq(&source_col) { - // Change the corresponding entry with alias expression - item.expr = target_col.clone(); - } - } - } - ordering -} - -fn update_with_aliases( - in_data: &OrderingEquivalentGroup, - oeq_alias_map: &[(Column, Column)], - fields: &Fields, -) -> OrderingEquivalentGroup { - let new_data = in_data - .iter() - .filter_map(|ordering| { - let new_ordering = update_with_alias(ordering.clone(), oeq_alias_map); - let is_invalid = new_ordering.iter().any(|sort_expr| { - // If any one of the columns, used in Expression is invalid after projection, - // remove expression from ordering equivalences - collect_columns(&sort_expr.expr) - .iter() - .any(|col| is_column_invalid_in_new_schema(col, fields)) - }); - if is_invalid { - None - } else { - Some(new_ordering) - } - }) - .collect(); - OrderingEquivalentGroup::new(new_data) -} - /// Adds `offset` value to the index of each expression inside `self.head` and `self.others`. pub fn add_offset( in_data: &OrderingEquivalentGroup, @@ -1279,12 +1235,6 @@ impl OrderingEquivalenceBuilder { } } -/// Checks whether column is still valid after projection. -fn is_column_invalid_in_new_schema(column: &Column, fields: &Fields) -> bool { - let idx = column.index(); - idx >= fields.len() || fields[idx].name() != column.name() -} - // /// This function applies the given projection to the given ordering // /// equivalence properties to compute the resulting (projected) ordering // /// equivalence properties; e.g. @@ -1372,10 +1322,10 @@ pub fn ordering_equivalence_properties_helper( let mut oep = OrderingEquivalenceProperties::new(schema); if eq_orderings.is_empty() { // Return an empty OrderingEquivalenceProperties: - return oep; + oep } else { oep.extend(OrderingEquivalentGroup::new(eq_orderings.to_vec())); - return oep; + oep } // oep.extend(Some(OrderingEquivalentClass::new())) // let first_ordering = if let Some(first) = eq_orderings.first() { @@ -1393,35 +1343,6 @@ pub fn ordering_equivalence_properties_helper( // oep } -/// This function constructs a duplicate-free vector by filtering out duplicate -/// entries inside the given vector `input`. -fn collapse_vec(input: Vec) -> Vec { - let mut output = vec![]; - for item in input { - if !output.contains(&item) { - output.push(item); - } - } - output -} - -/// This function constructs a duplicate-free `LexOrderingReq` by filtering out duplicate -/// entries that have same physical expression inside the given vector `input`. -/// `vec![a Some(Asc), a Some(Desc)]` is collapsed to the `vec![a Some(Asc)]`. Since -/// when same expression is already seen before, following expressions are redundant. -fn collapse_lex_sort_exprs(input: LexOrdering) -> LexOrdering { - let mut output = vec![]; - for item in input { - if output - .iter() - .all(|elem: &PhysicalSortExpr| !elem.expr.eq(&item.expr)) - { - output.push(item); - } - } - output -} - /// This function constructs a duplicate-free `LexOrderingReq` by filtering out duplicate /// entries that have same physical expression inside the given vector `input`. /// `vec![a Some(Asc), a Some(Desc)]` is collapsed to the `vec![a Some(Asc)]`. Since @@ -1700,10 +1621,12 @@ mod tests { Ok(schema) } + /// Construct a schema with following properties + /// Schema satisfied following orderings: + /// [a ASC], [d ASC, b ASC], [e DESC, f ASC, g ASC] + /// and + /// Column [a=c] (e.g they are aliases). fn create_test_params() -> Result<(SchemaRef, OrderingEquivalenceProperties)> { - // Assume schema satisfies ordering a ASC NULLS LAST - // and d ASC NULLS LAST, b ASC NULLS LAST and e DESC NULLS FIRST, f ASC NULLS LAST, g ASC NULLS LAST - // Assume that column a and c are aliases. let col_a = &Column::new("a", 0); let col_b = &Column::new("b", 1); let col_c = &Column::new("c", 2); @@ -1740,12 +1663,6 @@ mod tests { options: option1, }, ], - ]); - ordering_eq_properties.add_new_orderings(&[ - vec![PhysicalSortExpr { - expr: Arc::new(col_a.clone()), - options: option1, - }], vec![ PhysicalSortExpr { expr: Arc::new(col_e.clone()), @@ -1884,14 +1801,6 @@ mod tests { Ok(()) } - #[test] - fn test_collapse_vec() -> Result<()> { - assert_eq!(collapse_vec(vec![1, 2, 3]), vec![1, 2, 3]); - assert_eq!(collapse_vec(vec![1, 2, 3, 2, 3]), vec![1, 2, 3]); - assert_eq!(collapse_vec(vec![3, 1, 2, 3, 2, 3]), vec![3, 1, 2]); - Ok(()) - } - #[test] fn test_get_compatible_ranges() -> Result<()> { let col_a = &Column::new("a", 0); @@ -1980,17 +1889,6 @@ mod tests { nulls_first: true, }; // The schema is ordered by a ASC NULLS LAST, b ASC NULLS LAST - let provided = vec![ - PhysicalSortExpr { - expr: Arc::new(col_a.clone()), - options: option1, - }, - PhysicalSortExpr { - expr: Arc::new(col_b.clone()), - options: option1, - }, - ]; - let provided = Some(&provided[..]); let (_test_schema, ordering_eq_properties) = create_test_params()?; // First element in the tuple stores vector of requirement, second element is the expected return value for ordering_satisfy function let requirements = vec![ diff --git a/datafusion/physical-expr/src/physical_expr.rs b/datafusion/physical-expr/src/physical_expr.rs index fad6487e9878..6ffe5ed5019a 100644 --- a/datafusion/physical-expr/src/physical_expr.rs +++ b/datafusion/physical-expr/src/physical_expr.rs @@ -27,7 +27,6 @@ use datafusion_common::utils::DataPtr; use datafusion_common::{internal_err, not_impl_err, DataFusionError, Result}; use datafusion_expr::ColumnarValue; -use hashbrown::HashSet; use itertools::izip; use std::any::Any; use std::fmt::{Debug, Display}; @@ -218,6 +217,7 @@ pub fn have_common_entries( } /// Check whether physical exprs vectors are equal. +#[allow(dead_code)] pub fn physical_exprs_equal( lhs: &[Arc], rhs: &[Arc], @@ -233,7 +233,6 @@ mod tests { }; use crate::{physical_exprs_contains, PhysicalExpr}; use datafusion_common::{Result, ScalarValue}; - use itertools::izip; use std::sync::Arc; #[test] @@ -282,11 +281,8 @@ mod tests { Arc::new(Literal::new(ScalarValue::Int32(Some(4)))) as Arc; let lit2 = Arc::new(Literal::new(ScalarValue::Int32(Some(2)))) as Arc; - let lit1 = - Arc::new(Literal::new(ScalarValue::Int32(Some(1)))) as Arc; let col_a_expr = Arc::new(Column::new("a", 0)) as Arc; let col_b_expr = Arc::new(Column::new("b", 1)) as Arc; - let col_c_expr = Arc::new(Column::new("c", 2)) as Arc; // lit(true), lit(false), lit(4), lit(2), Col(a), Col(b) let physical_exprs: Vec> = vec![ @@ -321,13 +317,10 @@ mod tests { as Arc; let lit_false = Arc::new(Literal::new(ScalarValue::Boolean(Some(false)))) as Arc; - let lit4 = - Arc::new(Literal::new(ScalarValue::Int32(Some(4)))) as Arc; let lit2 = Arc::new(Literal::new(ScalarValue::Int32(Some(2)))) as Arc; let lit1 = Arc::new(Literal::new(ScalarValue::Int32(Some(1)))) as Arc; - let col_a_expr = Arc::new(Column::new("a", 0)) as Arc; let col_b_expr = Arc::new(Column::new("b", 1)) as Arc; let col_c_expr = Arc::new(Column::new("c", 2)) as Arc; diff --git a/datafusion/physical-expr/src/utils.rs b/datafusion/physical-expr/src/utils.rs index 9382012fea86..ef28b4bc3d30 100644 --- a/datafusion/physical-expr/src/utils.rs +++ b/datafusion/physical-expr/src/utils.rs @@ -178,11 +178,7 @@ pub fn get_indices_of_matching_exprs( targets: &[Arc], items: &[Arc], ) -> Vec { - // let eq_properties = equal_properties(); - // let normalized_items = eq_properties.normalize_exprs(items); - // let normalized_targets = eq_properties.normalize_exprs(targets); - // get_indices_of_exprs_strict(normalized_targets, &normalized_items) - get_indices_of_exprs_strict(targets, &items) + get_indices_of_exprs_strict(targets, items) } /// This function finds the indices of `targets` within `items` using strict diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 87eea3cdd4ed..a26ef800f259 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -464,7 +464,8 @@ fn get_finest_requirement OrderingEquivalenceProperties>( Ok(finest_req) } -fn print_plan(plan: &Arc) -> () { +#[allow(dead_code)] +fn print_plan(plan: &Arc) { let formatted = displayable(plan.as_ref()).indent(true).to_string(); let actual: Vec<&str> = formatted.trim().lines().collect(); println!("{:#?}", actual); @@ -2308,8 +2309,8 @@ mod tests { let col_b = Column::new("b", 1); let col_c = Column::new("c", 2); let col_d = Column::new("d", 3); - let col_a_expr = (Arc::new(col_a.clone()) as Arc); - let col_b_expr = (Arc::new(col_b.clone()) as Arc); + let col_a_expr = Arc::new(col_a.clone()) as Arc; + let col_b_expr = Arc::new(col_b.clone()) as Arc; let mut ordering_eq_properties = OrderingEquivalenceProperties::new(test_schema); // Columns a and b are equal. ordering_eq_properties.add_equal_conditions((&col_a_expr, &col_b_expr)); diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index 0c805496068a..83c948c00ca6 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -155,6 +155,14 @@ impl ExecutionPlan for FilterExec { fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { let stats = self.statistics(); + // Combine the equal predicates with the input equivalence properties + let mut filter_oeq = self.input.ordering_equivalence_properties(); + let (equal_pairs, _ne_pairs) = collect_columns_from_predicate(&self.predicate); + for (lhs, rhs) in equal_pairs { + let lhs_expr = Arc::new(lhs.clone()) as _; + let rhs_expr = Arc::new(rhs.clone()) as _; + filter_oeq.add_equal_conditions((&lhs_expr, &rhs_expr)) + } // Add the columns that have only one value (singleton) after filtering to constants. if let Some(col_stats) = stats.column_statistics { let constants = collect_columns(self.predicate()) @@ -162,12 +170,13 @@ impl ExecutionPlan for FilterExec { .filter(|column| col_stats[column.index()].is_singleton()) .map(|column| Arc::new(column) as Arc) .collect::>(); - let filter_oeq = self.input.ordering_equivalence_properties(); - let res = filter_oeq.with_constants(constants); - println!("filter res:{:?}", res); - res + // // let filter_oeq = self.input.ordering_equivalence_properties(); + // let res = filter_oeq.with_constants(constants); + // println!("filter res:{:?}", res); + // res + filter_oeq.with_constants(constants) } else { - self.input.ordering_equivalence_properties() + filter_oeq } } diff --git a/datafusion/physical-plan/src/memory.rs b/datafusion/physical-plan/src/memory.rs index 6ca06809614c..d5fc6904c839 100644 --- a/datafusion/physical-plan/src/memory.rs +++ b/datafusion/physical-plan/src/memory.rs @@ -33,7 +33,6 @@ use std::task::{Context, Poll}; use crate::ordering_equivalence_properties_helper; use datafusion_common::DataFusionError; use datafusion_execution::TaskContext; -use datafusion_physical_expr::equivalence::OrderingEquivalentGroup; use datafusion_physical_expr::{LexOrdering, OrderingEquivalenceProperties}; use futures::Stream; diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index 5fc7d26bf985..22c861fbb081 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -64,10 +64,6 @@ pub struct ProjectionExec { source_to_target_mapping: Vec<(Arc, Arc)>, /// Execution metrics metrics: ExecutionPlanMetricsSet, - /// Expressions' normalized orderings (as given by the output ordering API - /// and normalized with respect to equivalence classes of input plan). The - /// projected expressions are mapped by their indices to this vector. - orderings: Vec>, } impl ProjectionExec { @@ -140,11 +136,11 @@ impl ProjectionExec { }; } - let orderings = find_orderings_of_exprs( - &expr, - input.output_ordering(), - input.ordering_equivalence_properties(), - )?; + // let orderings = find_orderings_of_exprs( + // &expr, + // input.output_ordering(), + // input.ordering_equivalence_properties(), + // )?; // println!("source_to_target_mapping:{:?}", source_to_target_mapping); // println!("input.ordering_equivalence_properties():{:?}", input.ordering_equivalence_properties()); @@ -171,7 +167,6 @@ impl ProjectionExec { columns_map, source_to_target_mapping, metrics: ExecutionPlanMetricsSet::new(), - orderings, }) } @@ -326,40 +321,6 @@ impl ExecutionPlan for ProjectionExec { } } -/// This function takes the current `output_ordering`, the `orderings` based on projected expressions, -/// and the `expr` representing the projected expressions themselves. It aims to ensure that the output -/// ordering is valid and correctly corresponds to the projected columns. -/// -/// If the leading expression in the `output_ordering` is an [`UnKnownColumn`], it indicates that the column -/// referenced in the ordering is not found among the projected expressions. In such cases, this function -/// attempts to create a new output ordering by referring to valid columns from the leftmost side of the -/// expressions that have an ordering specified. -fn validate_output_ordering( - output_ordering: Option>, - orderings: &[Option], - expr: &[(Arc, String)], -) -> Option> { - output_ordering.and_then(|ordering| { - // If the leading expression is invalid column, change output - // ordering of the projection so that it refers to valid columns if - // possible. - if ordering[0].expr.as_any().is::() { - for (idx, order) in orderings.iter().enumerate() { - if let Some(sort_expr) = order { - let (_, col_name) = &expr[idx]; - return Some(vec![PhysicalSortExpr { - expr: Arc::new(Column::new(col_name, idx)), - options: sort_expr.options, - }]); - } - } - None - } else { - Some(ordering) - } - }) -} - /// If e is a direct column reference, returns the field level /// metadata for that field, if any. Otherwise returns None fn get_field_metadata( diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index 277c4798817a..ab929649db1f 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -830,13 +830,15 @@ impl ExecutionPlan for SortExec { // } fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { - let mut input_oeq = self.input.ordering_equivalence_properties(); + let input_oeq = self.input.ordering_equivalence_properties(); // println!("sort input: {:?}", input_oeq); // let mut res = OrderingEquivalenceProperties::new(self.schema()); - let res = input_oeq.with_reorder(self.expr.to_vec()); - // println!("sort output: {:?}", res); - res + // let res = input_oeq.with_reorder(self.expr.to_vec()); + // // println!("sort output: {:?}", res); + // res + + input_oeq.with_reorder(self.expr.to_vec()) } fn with_new_children( From 346e64e8089281ccb5f44c44d36279d3fa4881a1 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Mon, 2 Oct 2023 11:05:37 +0300 Subject: [PATCH 007/122] Resolve linter errors --- .../combine_partial_final_agg.rs | 1 + .../src/physical_optimizer/enforce_sorting.rs | 18 +++++++++-------- .../src/physical_optimizer/sort_pushdown.rs | 20 +++++++++---------- datafusion/core/src/physical_planner.rs | 1 + datafusion/physical-expr/src/physical_expr.rs | 5 ----- datafusion/physical-expr/src/utils.rs | 12 ++--------- .../physical-plan/src/aggregates/mod.rs | 16 +++++++-------- datafusion/physical-plan/src/joins/utils.rs | 8 ++------ datafusion/physical-plan/src/projection.rs | 3 +-- 9 files changed, 34 insertions(+), 50 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs b/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs index 13d4cd161e4c..ba5947989cf6 100644 --- a/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs +++ b/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs @@ -43,6 +43,7 @@ impl CombinePartialFinalAggregate { } } +#[allow(dead_code)] fn print_plan(plan: &Arc) { let formatted = crate::physical_plan::displayable(plan.as_ref()) .indent(true) diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index e94faf84c275..4b625fc5d787 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -310,6 +310,7 @@ impl TreeNode for PlanWithCorrespondingCoalescePartitions { } } +#[allow(dead_code)] fn print_plan(plan: &Arc) { let formatted = crate::physical_plan::displayable(plan.as_ref()) .indent(true) @@ -2797,6 +2798,7 @@ mod tests { } } +#[cfg(test)] mod tmp_tests { use crate::physical_optimizer::utils::get_plan_string; use crate::physical_plan::{collect, displayable, ExecutionPlan}; @@ -3048,7 +3050,7 @@ mod tmp_tests { let dataframe = ctx.sql(sql).await.expect(&msg); let physical_plan = dataframe.create_physical_plan().await?; print_plan(&physical_plan)?; - let actual = collect(physical_plan.clone(), ctx.task_ctx()).await?; + let _actual = collect(physical_plan.clone(), ctx.task_ctx()).await?; // print_batches(&actual)?; let expected_optimized_lines: Vec<&str> = vec![ @@ -3111,7 +3113,7 @@ mod tmp_tests { let dataframe = ctx.sql(sql).await.expect(&msg); let physical_plan = dataframe.create_physical_plan().await?; print_plan(&physical_plan)?; - let actual = collect(physical_plan.clone(), ctx.task_ctx()).await?; + let _actual = collect(physical_plan.clone(), ctx.task_ctx()).await?; // print_batches(&actual)?; let expected_optimized_lines: Vec<&str> = vec![ @@ -3173,7 +3175,7 @@ mod tmp_tests { let dataframe = ctx.sql(sql).await.expect(&msg); let physical_plan = dataframe.create_physical_plan().await?; print_plan(&physical_plan)?; - let actual = collect(physical_plan.clone(), ctx.task_ctx()).await?; + let _actual = collect(physical_plan.clone(), ctx.task_ctx()).await?; // print_batches(&actual)?; let expected_optimized_lines: Vec<&str> = vec![ @@ -3227,7 +3229,7 @@ mod tmp_tests { let dataframe = ctx.sql(sql).await.expect(&msg); let physical_plan = dataframe.create_physical_plan().await?; print_plan(&physical_plan)?; - let actual = collect(physical_plan.clone(), ctx.task_ctx()).await?; + let _actual = collect(physical_plan.clone(), ctx.task_ctx()).await?; // print_batches(&actual)?; let expected_optimized_lines: Vec<&str> = vec![ @@ -3293,7 +3295,7 @@ mod tmp_tests { let dataframe = ctx.sql(sql).await.expect(&msg); let physical_plan = dataframe.create_physical_plan().await?; print_plan(&physical_plan)?; - let actual = collect(physical_plan.clone(), ctx.task_ctx()).await?; + let _actual = collect(physical_plan.clone(), ctx.task_ctx()).await?; // print_batches(&actual)?; let expected_optimized_lines: Vec<&str> = vec![ @@ -3362,7 +3364,7 @@ mod tmp_tests { let dataframe = ctx.sql(sql).await.expect(&msg); let physical_plan = dataframe.create_physical_plan().await?; print_plan(&physical_plan)?; - let actual = collect(physical_plan.clone(), ctx.task_ctx()).await?; + let _actual = collect(physical_plan.clone(), ctx.task_ctx()).await?; // print_batches(&actual)?; let expected_optimized_lines: Vec<&str> = vec![ @@ -3417,7 +3419,7 @@ mod tmp_tests { let dataframe = ctx.sql(sql).await.expect(&msg); let physical_plan = dataframe.create_physical_plan().await?; print_plan(&physical_plan)?; - let actual = collect(physical_plan.clone(), ctx.task_ctx()).await?; + let _actual = collect(physical_plan.clone(), ctx.task_ctx()).await?; // print_batches(&actual)?; let expected_optimized_lines: Vec<&str> = vec![ @@ -3472,7 +3474,7 @@ mod tmp_tests { let dataframe = ctx.sql(sql).await.expect(&msg); let physical_plan = dataframe.create_physical_plan().await?; print_plan(&physical_plan)?; - let actual = collect(physical_plan.clone(), ctx.task_ctx()).await?; + let _actual = collect(physical_plan.clone(), ctx.task_ctx()).await?; // print_batches(&actual)?; let expected_optimized_lines: Vec<&str> = vec![ diff --git a/datafusion/core/src/physical_optimizer/sort_pushdown.rs b/datafusion/core/src/physical_optimizer/sort_pushdown.rs index b5e7df6a2159..fe1236472dd6 100644 --- a/datafusion/core/src/physical_optimizer/sort_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/sort_pushdown.rs @@ -257,16 +257,16 @@ fn pushdown_requirement_to_children( || plan.as_any().is::() // Do not push-down through SortPreservingMergeExec when // ordering requirement invalidates requirement of sort preserving merge exec. - || (is_sort_preserving_merge(plan) && true - // !ordering_satisfy( - // parent_required - // .map(|req| PhysicalSortRequirement::to_sort_exprs(req.to_vec())) - // .as_deref(), - // plan.output_ordering(), - // || plan.ordering_equivalence_properties(), - // ) - // && plan.ordering_equivalence_properties().get_finer_ordering() - ) + || is_sort_preserving_merge(plan) + // !ordering_satisfy( + // parent_required + // .map(|req| PhysicalSortRequirement::to_sort_exprs(req.to_vec())) + // .as_deref(), + // plan.output_ordering(), + // || plan.ordering_equivalence_properties(), + // ) + // && plan.ordering_equivalence_properties().get_finer_ordering() + // ) { println!("parent_required if:{:?}", parent_required); // If the current plan is a leaf node or can not maintain any of the input ordering, can not pushed down requirements. diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 86595966f97f..f7cd031a6f69 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -2791,6 +2791,7 @@ digraph { } } +#[allow(dead_code)] fn print_plan(plan: &Arc) { let formatted = crate::physical_plan::displayable(plan.as_ref()) .indent(true) diff --git a/datafusion/physical-expr/src/physical_expr.rs b/datafusion/physical-expr/src/physical_expr.rs index 6ffe5ed5019a..567b6f3cb5ca 100644 --- a/datafusion/physical-expr/src/physical_expr.rs +++ b/datafusion/physical-expr/src/physical_expr.rs @@ -322,7 +322,6 @@ mod tests { let lit1 = Arc::new(Literal::new(ScalarValue::Int32(Some(1)))) as Arc; let col_b_expr = Arc::new(Column::new("b", 1)) as Arc; - let col_c_expr = Arc::new(Column::new("c", 2)) as Arc; let vec1: Vec> = vec![lit_true.clone(), lit_false.clone()]; @@ -345,15 +344,11 @@ mod tests { as Arc; let lit_false = Arc::new(Literal::new(ScalarValue::Boolean(Some(false)))) as Arc; - let lit4 = - Arc::new(Literal::new(ScalarValue::Int32(Some(4)))) as Arc; let lit2 = Arc::new(Literal::new(ScalarValue::Int32(Some(2)))) as Arc; let lit1 = Arc::new(Literal::new(ScalarValue::Int32(Some(1)))) as Arc; - let col_a_expr = Arc::new(Column::new("a", 0)) as Arc; let col_b_expr = Arc::new(Column::new("b", 1)) as Arc; - let col_c_expr = Arc::new(Column::new("c", 2)) as Arc; let vec1: Vec> = vec![lit_true.clone(), lit_false.clone()]; diff --git a/datafusion/physical-expr/src/utils.rs b/datafusion/physical-expr/src/utils.rs index ef28b4bc3d30..b04a0c68a95b 100644 --- a/datafusion/physical-expr/src/utils.rs +++ b/datafusion/physical-expr/src/utils.rs @@ -788,7 +788,6 @@ mod tests { #[test] fn test_get_indices_of_matching_exprs() { - let empty_schema = &Arc::new(Schema::empty()); let list1: Vec> = vec![ Arc::new(Column::new("a", 0)), Arc::new(Column::new("b", 1)), @@ -1005,11 +1004,6 @@ mod tests { #[test] fn test_normalize_sort_requirement_with_equivalence() -> Result<()> { - let col_a = &Column::new("a", 0); - let _col_b = &Column::new("b", 1); - let col_c = &Column::new("c", 2); - let col_d = &Column::new("d", 3); - let _col_e = &Column::new("e", 4); let option1 = SortOptions { descending: false, nulls_first: false, @@ -1017,10 +1011,10 @@ mod tests { // Assume that column a and c are aliases. let (test_schema, ordering_eq_properties) = create_test_params()?; let col_a_expr = col("a", &test_schema)?; - let col_b_expr = col("b", &test_schema)?; + let _col_b_expr = col("b", &test_schema)?; let col_c_expr = col("c", &test_schema)?; let col_d_expr = col("d", &test_schema)?; - let col_e_expr = col("e", &test_schema)?; + let _col_e_expr = col("e", &test_schema)?; // Test cases for equivalence normalization // First entry in the tuple is PhysicalExpr, second entry is its ordering, third entry is result after normalization. @@ -1208,8 +1202,6 @@ mod tests { Field::new("b", DataType::Int32, true), Field::new("c", DataType::Int32, true), ]); - let equal_properties = - OrderingEquivalenceProperties::new(Arc::new(schema.clone())); let mut ordering_equal_properties = OrderingEquivalenceProperties::new(Arc::new(schema)); ordering_equal_properties.add_new_orderings(&[ diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index a26ef800f259..37ae2850aba4 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -339,18 +339,16 @@ fn get_working_mode( // let ordered_group_by_indices = // get_indices_of_matching_exprs(&ordered_exprs, &groupby_exprs); - if let Some(ordered_group_by_info) = input + input .ordering_equivalence_properties() .set_satisfy(&groupby_exprs) - { - Some(if ordered_group_by_info.len() == group_by.expr.len() { - (GroupByOrderMode::FullyOrdered, ordered_group_by_info) - } else { - (GroupByOrderMode::PartiallyOrdered, ordered_group_by_info) + .map(|ordered_group_by_info| { + if ordered_group_by_info.len() == group_by.expr.len() { + (GroupByOrderMode::FullyOrdered, ordered_group_by_info) + } else { + (GroupByOrderMode::PartiallyOrdered, ordered_group_by_info) + } }) - } else { - None - } } /// This function gathers the ordering information for the GROUP BY columns. diff --git a/datafusion/physical-plan/src/joins/utils.rs b/datafusion/physical-plan/src/joins/utils.rs index f1ddacde47b9..bbf3d91344de 100644 --- a/datafusion/physical-plan/src/joins/utils.rs +++ b/datafusion/physical-plan/src/joins/utils.rs @@ -252,15 +252,11 @@ fn get_updated_right_ordering_equivalent_class( // In these modes, indices of the right schema should be offset by // the left table size. JoinType::Inner | JoinType::Left | JoinType::Full | JoinType::Right => { - let right_oeq_class = add_offset(&right_oeq_class, left_columns_len)?; - return Ok( - // right_oeq_class.normalize_with_equivalence_properties(join_eq_properties) - right_oeq_class, - ); + let right_oeq_class = add_offset(right_oeq_class, left_columns_len)?; + return Ok(right_oeq_class); } _ => {} }; - // Ok(right_oeq_class.normalize_with_equivalence_properties(join_eq_properties)) Ok(right_oeq_class.clone()) } diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index 22c861fbb081..04bfbe2daffa 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -37,13 +37,12 @@ use arrow::datatypes::{Field, Schema, SchemaRef}; use arrow::record_batch::{RecordBatch, RecordBatchOptions}; use datafusion_common::Result; use datafusion_execution::TaskContext; -use datafusion_physical_expr::expressions::{Literal, UnKnownColumn}; +use datafusion_physical_expr::expressions::Literal; use datafusion_physical_expr::{ normalize_out_expr_with_columns_map, OrderingEquivalenceProperties, }; use datafusion_common::tree_node::{Transformed, TreeNode}; -use datafusion_physical_expr::utils::find_orderings_of_exprs; use futures::stream::{Stream, StreamExt}; use log::trace; From 9ca970b7a638f72269aa598c4b00640b799bdd71 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Mon, 2 Oct 2023 11:10:36 +0300 Subject: [PATCH 008/122] Minor changes --- .../physical_optimizer/combine_partial_final_agg.rs | 10 ---------- .../src/physical_optimizer/enforce_distribution.rs | 3 --- datafusion/physical-expr/src/equivalence.rs | 13 +++++++------ 3 files changed, 7 insertions(+), 19 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs b/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs index ba5947989cf6..40b2bcc3e140 100644 --- a/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs +++ b/datafusion/core/src/physical_optimizer/combine_partial_final_agg.rs @@ -43,22 +43,12 @@ impl CombinePartialFinalAggregate { } } -#[allow(dead_code)] -fn print_plan(plan: &Arc) { - let formatted = crate::physical_plan::displayable(plan.as_ref()) - .indent(true) - .to_string(); - let actual: Vec<&str> = formatted.trim().lines().collect(); - println!("{:#?}", actual); -} - impl PhysicalOptimizerRule for CombinePartialFinalAggregate { fn optimize( &self, plan: Arc, _config: &ConfigOptions, ) -> Result> { - // print_plan(&plan); plan.transform_down(&|plan| { let transformed = plan.as_any() diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index d76c5939c36a..3d7566e56172 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -791,9 +791,6 @@ fn try_reorder( expected: &[Arc], equivalence_properties: &OrderingEquivalenceProperties, ) -> Option<(JoinKeyPairs, Vec)> { - // println!("join_keys:{:?}", join_keys); - // println!("expected:{:?}", expected); - // println!("equivalence_properties:{:?}", equivalence_properties); let eq_groups = equivalence_properties.eq_groups(); let mut normalized_expected = vec![]; let mut normalized_left_keys = vec![]; diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 9b0bc1595058..dc5ce8a8ba54 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -54,8 +54,14 @@ impl EquivalentGroups { res } + /// Get how many equivalent groups there are. + fn len(&self) -> usize { + self.inner.len() + } + + /// Check whether equivalent groups is empty pub fn is_empty(&self) -> bool { - self.inner.is_empty() + self.len() == 0 } fn iter(&self) -> impl Iterator>> { @@ -147,11 +153,6 @@ impl EquivalentGroups { self.inner = out_groups; } - #[allow(dead_code)] - fn len(&self) -> usize { - self.inner.len() - } - fn extend(&mut self, other: EquivalentGroups) { // TODO: Add check for redundancy self.inner.extend(other.inner); From 722c2c2db7b806291a767d71206e422760248d18 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Mon, 2 Oct 2023 11:13:12 +0300 Subject: [PATCH 009/122] Minor changes --- datafusion/core/src/physical_planner.rs | 10 ---------- .../physical-expr/src/expressions/get_indexed_field.rs | 1 - 2 files changed, 11 deletions(-) diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index f7cd031a6f69..2328ffce235d 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -2433,7 +2433,6 @@ mod tests { .build()?; let execution_plan = plan(&logical_plan).await?; - // print_plan(&execution_plan); let final_hash_agg = execution_plan .as_any() .downcast_ref::() @@ -2790,12 +2789,3 @@ digraph { assert_contains!(generated_graph, expected_tooltip); } } - -#[allow(dead_code)] -fn print_plan(plan: &Arc) { - let formatted = crate::physical_plan::displayable(plan.as_ref()) - .indent(true) - .to_string(); - let actual: Vec<&str> = formatted.trim().lines().collect(); - println!("{:#?}", actual); -} diff --git a/datafusion/physical-expr/src/expressions/get_indexed_field.rs b/datafusion/physical-expr/src/expressions/get_indexed_field.rs index b7a19f4638db..71613cb47446 100644 --- a/datafusion/physical-expr/src/expressions/get_indexed_field.rs +++ b/datafusion/physical-expr/src/expressions/get_indexed_field.rs @@ -255,7 +255,6 @@ impl PhysicalExpr for GetIndexedFieldExpr { impl PartialEq for GetIndexedFieldExpr { fn eq(&self, other: &dyn Any) -> bool { - // println!("self:{:?}, other:{:?}", self, other); down_cast_any_ref(other) .downcast_ref::() .map(|x| self.arg.eq(&x.arg) && self.field.eq(&x.field)) From f3f76d63795bd06ca169c233299407e9ee618ab5 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Mon, 2 Oct 2023 14:46:23 +0300 Subject: [PATCH 010/122] Add new tests --- .../src/physical_optimizer/enforce_sorting.rs | 109 ++++++++++++++++++ .../sqllogictest/test_files/groupby.slt | 39 +++++++ datafusion/sqllogictest/test_files/window.slt | 34 ++++++ 3 files changed, 182 insertions(+) diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index 7efbbcbcc1c6..d15252f98b7b 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -3157,4 +3157,113 @@ mod tmp_tests { print_batches(&actual)?; Ok(()) } + + #[tokio::test] + async fn test_query16() -> Result<()> { + let config = SessionConfig::new().with_target_partitions(1); + let ctx = SessionContext::with_config(config); + + ctx.sql( + "CREATE EXTERNAL TABLE multiple_ordered_table ( + a0 INTEGER, + a INTEGER, + b INTEGER, + c INTEGER, + d INTEGER + ) + STORED AS CSV + WITH HEADER ROW + WITH ORDER (a ASC, b ASC) + WITH ORDER (c ASC) + LOCATION '../core/tests/data/window_2.csv'", + ) + .await?; + + let sql = "SELECT LAST_VALUE(l.d ORDER BY l.a) AS amount_usd + FROM multiple_ordered_table AS l + INNER JOIN ( + SELECT *, ROW_NUMBER() OVER (ORDER BY r.a) as row_n FROM multiple_ordered_table AS r + ) + ON l.d = r.d AND + l.a >= r.a - 10 + GROUP BY row_n + ORDER BY row_n"; + + let msg = format!("Creating logical plan for '{sql}'"); + let dataframe = ctx.sql(sql).await.expect(&msg); + let physical_plan = dataframe.create_physical_plan().await?; + print_plan(&physical_plan)?; + + let expected = vec![ + "ProjectionExec: expr=[amount_usd@0 as amount_usd]", + " ProjectionExec: expr=[LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST]@1 as amount_usd, row_n@0 as row_n]", + " AggregateExec: mode=Single, gby=[row_n@2 as row_n], aggr=[LAST_VALUE(l.d)], ordering_mode=FullyOrdered", + " ProjectionExec: expr=[a@0 as a, d@1 as d, row_n@4 as row_n]", + " CoalesceBatchesExec: target_batch_size=8192", + " HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d@1, d@1)], filter=CAST(a@0 AS Int64) >= CAST(a@1 AS Int64) - 10", + " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + " ProjectionExec: expr=[a@0 as a, d@1 as d, ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as row_n]", + " BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: \"ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW\", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow }], mode=[Sorted]", + " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", + ]; + // Get string representation of the plan + let actual = get_plan_string(&physical_plan); + assert_eq!( + expected, actual, + "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" + ); + + let actual = collect(physical_plan, ctx.task_ctx()).await?; + print_batches(&actual)?; + Ok(()) + } + + #[tokio::test] + async fn test_query17() -> Result<()> { + let config = SessionConfig::new().with_target_partitions(1); + let ctx = SessionContext::with_config(config); + + ctx.sql( + "CREATE EXTERNAL TABLE multiple_ordered_table ( + a0 INTEGER, + a INTEGER, + b INTEGER, + c INTEGER, + d INTEGER + ) + STORED AS CSV + WITH HEADER ROW + WITH ORDER (a ASC, b ASC) + WITH ORDER (c ASC) + LOCATION '../core/tests/data/window_2.csv'", + ) + .await?; + + let sql = "EXPLAIN VERBOSE SELECT MIN(d) OVER(ORDER BY c ASC) as min1, + MAX(d) OVER(PARTITION BY b, a ORDER BY c ASC) as max1 + FROM multiple_ordered_table"; + + let msg = format!("Creating logical plan for '{sql}'"); + let dataframe = ctx.sql(sql).await.expect(&msg); + let physical_plan = dataframe.create_physical_plan().await?; + print_plan(&physical_plan)?; + + let expected = vec![ + "ProjectionExec: expr=[MIN(multiple_ordered_table.d) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as min1, MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as max1]", + " BoundedWindowAggExec: wdw=[MIN(multiple_ordered_table.d) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: \"MIN(multiple_ordered_table.d) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow }], mode=[Sorted]", + " ProjectionExec: expr=[c@2 as c, d@3 as d, MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]", + " BoundedWindowAggExec: wdw=[MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: \"MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow }], mode=[Sorted]", + " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], has_header=true", + ]; + // Get string representation of the plan + // let actual = get_plan_string(&physical_plan); + // assert_eq!( + // expected, actual, + // "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" + // ); + + let actual = collect(physical_plan, ctx.task_ctx()).await?; + print_batches(&actual)?; + Ok(()) + } } diff --git a/datafusion/sqllogictest/test_files/groupby.slt b/datafusion/sqllogictest/test_files/groupby.slt index 5bb0f31ed542..da695273c7e3 100644 --- a/datafusion/sqllogictest/test_files/groupby.slt +++ b/datafusion/sqllogictest/test_files/groupby.slt @@ -3563,3 +3563,42 @@ logical_plan Sort: multiple_ordered_table.c ASC NULLS LAST --TableScan: multiple_ordered_table projection=[c] physical_plan CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c], output_ordering=[c@0 ASC NULLS LAST], has_header=true + +statement ok +set datafusion.execution.target_partitions = 1; + +query TT +EXPLAIN SELECT LAST_VALUE(l.d ORDER BY l.a) AS amount_usd +FROM multiple_ordered_table AS l +INNER JOIN ( + SELECT *, ROW_NUMBER() OVER (ORDER BY r.a) as row_n FROM multiple_ordered_table AS r +) +ON l.d = r.d AND + l.a >= r.a - 10 +GROUP BY row_n +ORDER BY row_n +---- +logical_plan +Projection: amount_usd +--Sort: row_n ASC NULLS LAST +----Projection: LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST] AS amount_usd, row_n +------Aggregate: groupBy=[[row_n]], aggr=[[LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST]]] +--------Projection: l.a, l.d, row_n +----------Inner Join: l.d = r.d Filter: CAST(l.a AS Int64) >= CAST(r.a AS Int64) - Int64(10) +------------SubqueryAlias: l +--------------TableScan: multiple_ordered_table projection=[a, d] +------------Projection: r.a, r.d, ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS row_n +--------------WindowAggr: windowExpr=[[ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +----------------SubqueryAlias: r +------------------TableScan: multiple_ordered_table projection=[a, d] +physical_plan +ProjectionExec: expr=[amount_usd@0 as amount_usd] +--ProjectionExec: expr=[LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST]@1 as amount_usd, row_n@0 as row_n] +----AggregateExec: mode=Single, gby=[row_n@2 as row_n], aggr=[LAST_VALUE(l.d)], ordering_mode=FullyOrdered +------ProjectionExec: expr=[a@0 as a, d@1 as d, row_n@4 as row_n] +--------CoalesceBatchesExec: target_batch_size=8192 +----------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d@1, d@1)], filter=CAST(a@0 AS Int64) >= CAST(a@1 AS Int64) - 10 +------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true +------------ProjectionExec: expr=[a@0 as a, d@1 as d, ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as row_n] +--------------BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow }], mode=[Sorted] +----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 299ce807d370..3cb679c53359 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -3381,3 +3381,37 @@ SELECT window1 AS (ORDER BY C3) ORDER BY C3 LIMIT 5 + +# Create a source where there is multiple orderings. +statement ok +CREATE EXTERNAL TABLE multiple_ordered_table ( + a0 INTEGER, + a INTEGER, + b INTEGER, + c INTEGER, + d INTEGER +) +STORED AS CSV +WITH HEADER ROW +WITH ORDER (a ASC, b ASC) +WITH ORDER (c ASC) +LOCATION '../core/tests/data/window_2.csv'; + +query TT +EXPLAIN SELECT MIN(d) OVER(ORDER BY c ASC) as min1, + MAX(d) OVER(PARTITION BY b, a ORDER BY c ASC) as max1 +FROM multiple_ordered_table +---- +logical_plan +Projection: MIN(multiple_ordered_table.d) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS min1, MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS max1 +--WindowAggr: windowExpr=[[MIN(multiple_ordered_table.d) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +----Projection: multiple_ordered_table.c, multiple_ordered_table.d, MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW +------WindowAggr: windowExpr=[[MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +--------TableScan: multiple_ordered_table projection=[a, b, c, d] +physical_plan +ProjectionExec: expr=[MIN(multiple_ordered_table.d) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as min1, MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as max1] +--BoundedWindowAggExec: wdw=[MIN(multiple_ordered_table.d) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MIN(multiple_ordered_table.d) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow }], mode=[Sorted] +----ProjectionExec: expr=[c@2 as c, d@3 as d, MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] +------BoundedWindowAggExec: wdw=[MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow }], mode=[Sorted] +--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], has_header=true + From 9707b352b66957aeb511eb49a48112dd1d031a4a Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Mon, 2 Oct 2023 17:19:03 +0300 Subject: [PATCH 011/122] Simplifications window mode selection --- .../src/physical_optimizer/enforce_sorting.rs | 19 +- .../core/src/physical_optimizer/test_utils.rs | 3 +- datafusion/core/src/physical_planner.rs | 5 +- .../core/tests/fuzz_cases/window_fuzz.rs | 5 +- datafusion/physical-expr/src/equivalence.rs | 111 ++++++--- datafusion/physical-expr/src/sort_expr.rs | 25 +++ .../physical-plan/src/aggregates/mod.rs | 40 ---- datafusion/physical-plan/src/streaming.rs | 12 +- .../src/windows/bounded_window_agg_exec.rs | 21 +- datafusion/physical-plan/src/windows/mod.rs | 211 +++--------------- datafusion/proto/src/physical_plan/mod.rs | 5 +- 11 files changed, 186 insertions(+), 271 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index d15252f98b7b..a9372c42f46b 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -51,7 +51,7 @@ use crate::physical_plan::coalesce_partitions::CoalescePartitionsExec; use crate::physical_plan::sorts::sort::SortExec; use crate::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; use crate::physical_plan::windows::{ - get_best_fitting_window, BoundedWindowAggExec, PartitionSearchMode, WindowAggExec, + get_best_fitting_window, BoundedWindowAggExec, WindowAggExec, }; use crate::physical_plan::{with_new_children_if_necessary, Distribution, ExecutionPlan}; @@ -59,6 +59,7 @@ use datafusion_common::tree_node::{Transformed, TreeNode, VisitRecursion}; use datafusion_common::{plan_err, DataFusionError}; use datafusion_physical_expr::{PhysicalSortExpr, PhysicalSortRequirement}; +use datafusion_physical_expr::equivalence::PartitionSearchMode; use itertools::izip; /// This rule inspects [`SortExec`]'s in the given physical plan and removes the @@ -3177,7 +3178,7 @@ mod tmp_tests { WITH ORDER (c ASC) LOCATION '../core/tests/data/window_2.csv'", ) - .await?; + .await?; let sql = "SELECT LAST_VALUE(l.d ORDER BY l.a) AS amount_usd FROM multiple_ordered_table AS l @@ -3237,9 +3238,9 @@ mod tmp_tests { WITH ORDER (c ASC) LOCATION '../core/tests/data/window_2.csv'", ) - .await?; + .await?; - let sql = "EXPLAIN VERBOSE SELECT MIN(d) OVER(ORDER BY c ASC) as min1, + let sql = "SELECT MIN(d) OVER(ORDER BY c ASC) as min1, MAX(d) OVER(PARTITION BY b, a ORDER BY c ASC) as max1 FROM multiple_ordered_table"; @@ -3256,11 +3257,11 @@ mod tmp_tests { " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], has_header=true", ]; // Get string representation of the plan - // let actual = get_plan_string(&physical_plan); - // assert_eq!( - // expected, actual, - // "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" - // ); + let actual = get_plan_string(&physical_plan); + assert_eq!( + expected, actual, + "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" + ); let actual = collect(physical_plan, ctx.task_ctx()).await?; print_batches(&actual)?; diff --git a/datafusion/core/src/physical_optimizer/test_utils.rs b/datafusion/core/src/physical_optimizer/test_utils.rs index e021cda2c868..56b640266549 100644 --- a/datafusion/core/src/physical_optimizer/test_utils.rs +++ b/datafusion/core/src/physical_optimizer/test_utils.rs @@ -46,6 +46,7 @@ use datafusion_physical_expr::expressions::col; use datafusion_physical_expr::{PhysicalExpr, PhysicalSortExpr}; use async_trait::async_trait; +use datafusion_physical_expr::equivalence::PartitionSearchMode; async fn register_current_csv( ctx: &SessionContext, @@ -241,7 +242,7 @@ pub fn bounded_window_exec( input.clone(), input.schema(), vec![], - crate::physical_plan::windows::PartitionSearchMode::Sorted, + PartitionSearchMode::Sorted, ) .unwrap(), ) diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 2328ffce235d..0e9f4e4f87bc 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -61,9 +61,7 @@ use crate::physical_plan::projection::ProjectionExec; use crate::physical_plan::repartition::RepartitionExec; use crate::physical_plan::sorts::sort::SortExec; use crate::physical_plan::unnest::UnnestExec; -use crate::physical_plan::windows::{ - BoundedWindowAggExec, PartitionSearchMode, WindowAggExec, -}; +use crate::physical_plan::windows::{BoundedWindowAggExec, WindowAggExec}; use crate::physical_plan::{ aggregates, empty::EmptyExec, joins::PartitionMode, udaf, union::UnionExec, values::ValuesExec, windows, @@ -89,6 +87,7 @@ use datafusion_expr::expr_rewriter::{unalias, unnormalize_cols}; use datafusion_expr::logical_plan::builder::wrap_projection_for_join_if_necessary; use datafusion_expr::{DescribeTable, DmlStatement, StringifiedPlan, WriteOp}; use datafusion_expr::{WindowFrame, WindowFrameBound}; +use datafusion_physical_expr::equivalence::PartitionSearchMode; use datafusion_physical_expr::expressions::Literal; use datafusion_sql::utils::window_expr_common_partition_keys; use futures::future::BoxFuture; diff --git a/datafusion/core/tests/fuzz_cases/window_fuzz.rs b/datafusion/core/tests/fuzz_cases/window_fuzz.rs index 3d103ee70ee8..278ea37acb36 100644 --- a/datafusion/core/tests/fuzz_cases/window_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/window_fuzz.rs @@ -29,7 +29,7 @@ use rand::{Rng, SeedableRng}; use datafusion::physical_plan::memory::MemoryExec; use datafusion::physical_plan::sorts::sort::SortExec; use datafusion::physical_plan::windows::{ - create_window_expr, BoundedWindowAggExec, PartitionSearchMode, WindowAggExec, + create_window_expr, BoundedWindowAggExec, WindowAggExec, }; use datafusion::physical_plan::{collect, ExecutionPlan}; use datafusion_expr::{ @@ -40,6 +40,7 @@ use datafusion_expr::{ use datafusion::prelude::{SessionConfig, SessionContext}; use datafusion_common::{Result, ScalarValue}; use datafusion_expr::type_coercion::aggregates::coerce_types; +use datafusion_physical_expr::equivalence::PartitionSearchMode; use datafusion_physical_expr::expressions::{cast, col, lit}; use datafusion_physical_expr::{PhysicalExpr, PhysicalSortExpr}; use test_utils::add_empty_batches; @@ -47,7 +48,7 @@ use test_utils::add_empty_batches; #[cfg(test)] mod tests { use super::*; - use datafusion::physical_plan::windows::PartitionSearchMode::{ + use datafusion_physical_expr::equivalence::PartitionSearchMode::{ Linear, PartiallySorted, Sorted, }; diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index dc5ce8a8ba54..04f34ec54655 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -18,8 +18,8 @@ use crate::expressions::{CastExpr, Column}; use crate::utils::get_indices_of_matching_exprs; use crate::{ - physical_exprs_contains, LexOrdering, LexOrderingRef, LexOrderingReq, PhysicalExpr, - PhysicalSortExpr, PhysicalSortRequirement, + physical_exprs_contains, reverse_order_bys, LexOrdering, LexOrderingRef, + LexOrderingReq, PhysicalExpr, PhysicalSortExpr, PhysicalSortRequirement, }; use arrow::datatypes::SchemaRef; @@ -800,10 +800,6 @@ impl OrderingEquivalenceProperties { exprs: &[Arc], ) -> Option> { let exprs_normalized = self.eq_groups.normalize_exprs(exprs); - // println!("exprs: {:?}", exprs); - // println!("exprs_normalized: {:?}", exprs_normalized); - // println!("self.eq_classes: {:?}", self.eq_classes); - // println!("self.oeq_class: {:?}", self.oeq_class); let mut best = vec![]; for ordering in self.oeq_group.iter() { @@ -812,11 +808,8 @@ impl OrderingEquivalenceProperties { .iter() .map(|sort_expr| sort_expr.expr.clone()) .collect::>(); - // let ordering_exprs = self.normalize_exprs(&ordering); - // println!("exprs_normalized: {:?}, normalized_ordering_exprs:{:?}", exprs_normalized, ordering_exprs); let mut ordered_indices = get_indices_of_matching_exprs(&exprs_normalized, &ordering_exprs); - // println!("ordered_indices: {:?}", ordered_indices); ordered_indices.sort(); // Find out how many expressions of the existing ordering define ordering // for expressions in the GROUP BY clause. For example, if the input is @@ -831,7 +824,6 @@ impl OrderingEquivalenceProperties { // GROUP BY expressions b, a, d match input ordering. let indices = get_indices_of_matching_exprs(&ordered_exprs, &exprs_normalized); - // println!("indices:{:?}, ordered_exprs: {:?}, exprs_normalized:{:?}", indices, ordered_exprs, exprs_normalized); best = indices .iter() .enumerate() @@ -867,21 +859,10 @@ impl OrderingEquivalenceProperties { pub fn ordering_satisfy_concrete(&self, required: &[PhysicalSortExpr]) -> bool { let required_normalized = self.normalize_sort_exprs(required); let provided_normalized = self.oeq_group().output_ordering().unwrap_or(vec![]); - // println!("required: {:?}", required); - // println!("self.oeq_class():{:?}", self.oeq_class()); - // println!("required_normalized:{:?}", required_normalized); - // println!("provided_normalized:{:?}", provided_normalized); if required_normalized.len() > provided_normalized.len() { return false; } - // let res = required_normalized - // .into_iter() - // .zip(provided_normalized) - // .all(|(req, given)| given == req); - // // println!("res:{:?}", res); - // res - required_normalized .into_iter() .zip(provided_normalized) @@ -964,20 +945,17 @@ impl OrderingEquivalenceProperties { /// provided [`PhysicalSortExpr`]s. pub fn ordering_satisfy_requirement_concrete( &self, - // provided: &[PhysicalSortExpr], required: &[PhysicalSortRequirement], ) -> bool { - // let eq_properties = equal_properties(); let provided_normalized = self.oeq_group().output_ordering().unwrap_or(vec![]); let required_normalized = self.normalize_sort_requirements(required); - // let provided_normalized = self.normalize_sort_exprs2(provided); if required_normalized.len() > provided_normalized.len() { return false; } required_normalized .into_iter() .zip(provided_normalized) - .all(|(req, given)| given.satisfy(&req)) + .all(|(req, given)| given.satisfy_with_schema(&req, &self.schema)) } /// Checks whether the given [`PhysicalSortRequirement`]s are equal or more @@ -1013,6 +991,89 @@ impl OrderingEquivalenceProperties { .zip(provided_normalized) .all(|(req, given)| given.compatible(&req)) } + + /// Compares physical ordering (output ordering of the `input` operator) with + /// `partitionby_exprs` and `orderby_keys` to decide whether existing ordering + /// is sufficient to run the current window operator. + /// - A `None` return value indicates that we can not remove the sort in question + /// (input ordering is not sufficient to run current window executor). + /// - A `Some((bool, PartitionSearchMode))` value indicates that the window operator + /// can run with existing input ordering, so we can remove `SortExec` before it. + /// The `bool` field in the return value represents whether we should reverse window + /// operator to remove `SortExec` before it. The `PartitionSearchMode` field represents + /// the mode this window operator should work in to accomodate the existing ordering. + pub fn get_window_mode( + &self, + partitionby_exprs: &[Arc], + orderby_keys: &[PhysicalSortExpr], + ) -> Result> { + let partitionby_exprs = self.eq_groups.normalize_exprs(partitionby_exprs); + let partition_by_oeq = self.clone().with_constants(partitionby_exprs.clone()); + let mut orderby_keys = self.eq_groups.normalize_sort_exprs(orderby_keys); + // Keep the order by expressions that are not inside partition by expressions. + orderby_keys.retain(|sort_expr| { + !physical_exprs_contains(&partitionby_exprs, &sort_expr.expr) + }); + let mut partition_search_mode = PartitionSearchMode::Linear; + let mut partition_by_reqs: Vec = vec![]; + if partitionby_exprs.is_empty() { + partition_search_mode = PartitionSearchMode::Sorted; + } else if let Some(indices_and_ordering) = self.set_satisfy(&partitionby_exprs) { + let indices = indices_and_ordering + .iter() + .map(|(idx, _options)| *idx) + .collect::>(); + let elem = indices + .iter() + .map(|&idx| PhysicalSortRequirement { + expr: partitionby_exprs[idx].clone(), + options: None, + }) + .collect::>(); + partition_by_reqs.extend(elem); + if indices.len() == partitionby_exprs.len() { + partition_search_mode = PartitionSearchMode::Sorted; + } else if !indices.is_empty() { + partition_search_mode = PartitionSearchMode::PartiallySorted(indices); + } + } + + let order_by_reqs = PhysicalSortRequirement::from_sort_exprs(&orderby_keys); + let req = [partition_by_reqs.clone(), order_by_reqs].concat(); + let req = collapse_lex_req(req); + if req.is_empty() { + // When requirement is empty, + // prefer None. Instead of Linear. + return Ok(None); + } + println!("req1:{:?}", req); + println!("partition_by_oeq: {:?}", partition_by_oeq); + if partition_by_oeq.ordering_satisfy_requirement_concrete(&req) { + // Window can be run with existing ordering + return Ok(Some((false, partition_search_mode))); + } + let reverse_order_by_reqs = + PhysicalSortRequirement::from_sort_exprs(&reverse_order_bys(&orderby_keys)); + let req = [partition_by_reqs, reverse_order_by_reqs].concat(); + let req = collapse_lex_req(req); + println!("req2:{:?}", req); + if partition_by_oeq.ordering_satisfy_requirement_concrete(&req) { + // Window can be run with existing ordering, if the ordering requirements would be reversed + return Ok(Some((true, partition_search_mode))); + } + Ok(None) + } +} + +#[derive(Debug, Clone, PartialEq)] +/// Specifies partition column properties in terms of input ordering +pub enum PartitionSearchMode { + /// None of the columns among the partition columns is ordered. + Linear, + /// Some columns of the partition columns are ordered but not all + PartiallySorted(Vec), + /// All Partition columns are ordered (Also empty case) + Sorted, } type ProjectionMapping = Vec<(Arc, Arc)>; diff --git a/datafusion/physical-expr/src/sort_expr.rs b/datafusion/physical-expr/src/sort_expr.rs index 83d32dfeec17..f20a5b3d64a3 100644 --- a/datafusion/physical-expr/src/sort_expr.rs +++ b/datafusion/physical-expr/src/sort_expr.rs @@ -24,6 +24,7 @@ use crate::PhysicalExpr; use arrow::compute::kernels::sort::{SortColumn, SortOptions}; use arrow::record_batch::RecordBatch; +use arrow_schema::Schema; use datafusion_common::plan_err; use datafusion_common::{DataFusionError, Result}; use datafusion_expr::ColumnarValue; @@ -89,6 +90,30 @@ impl PhysicalSortExpr { .options .map_or(true, |opts| self.options == opts) } + + /// Check whether sort expression satisfies [`PhysicalSortRequirement`]. + /// + /// If sort options is Some in `PhysicalSortRequirement`, `expr` + /// and `options` field are compared for equality. + /// + /// If sort options is None in `PhysicalSortRequirement`, only + /// `expr` is compared for equality. + pub fn satisfy_with_schema( + &self, + requirement: &PhysicalSortRequirement, + schema: &Schema, + ) -> bool { + let nullable = self.expr.nullable(schema).unwrap_or(true); + if nullable { + self.satisfy(requirement) + } else { + self.expr.eq(&requirement.expr) + && requirement + .options + // If the column is not nullable, NULLS FIRST/LAST is not important. + .map_or(true, |opts| self.options.descending == opts.descending) + } + } } /// Represents sort requirement associated with a plan diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index a138dfe94f10..a4d93e434457 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -317,28 +317,6 @@ fn get_working_mode( .iter() .map(|(item, _)| item.clone()) .collect::>(); - // Find where each expression of the GROUP BY clause occurs in the existing - // ordering (if it occurs): - - // let mut ordered_indices = - // get_indices_of_matching_exprs(&groupby_exprs, &ordering_exprs); - // ordered_indices.sort(); - // // Find out how many expressions of the existing ordering define ordering - // // for expressions in the GROUP BY clause. For example, if the input is - // // ordered by a, b, c, d and we group by b, a, d; the result below would be. - // // 2, meaning 2 elements (a, b) among the GROUP BY columns define ordering. - // let first_n = longest_consecutive_prefix(ordered_indices); - // if first_n == 0 { - // // No GROUP by columns are ordered, we can not do streaming execution. - // return None; - // } - // let ordered_exprs = ordering_exprs[0..first_n].to_vec(); - // // Find indices for the GROUP BY expressions such that when we iterate with - // // these indices, we would match existing ordering. For the example above, - // // this would produce 1, 0; meaning 1st and 0th entries (a, b) among the - // // GROUP BY expressions b, a, d match input ordering. - // let ordered_group_by_indices = - // get_indices_of_matching_exprs(&ordered_exprs, &groupby_exprs); input .ordering_equivalence_properties() @@ -1019,24 +997,6 @@ impl ExecutionPlan for AggregateExec { partition: usize, context: Arc, ) -> Result { - // println!("self.columns_map: {:?}", self.columns_map); - - // if self.mode.is_first_stage() { - // println!("aggregate self.input().ordering_equivalence_properties():\n {:?}", self.input().ordering_equivalence_properties()); - // println!("self.source_to_target_mapping: {:?}", self.source_to_target_mapping); - // println!("aggregate self.ordering_equivalence_properties():\n {:?}", self.ordering_equivalence_properties()); - // } - - // println!("self.aggregation_ordering:{:?}", self.aggregation_ordering); - // println!("aggregate self.required_input_ordering: {:?}", self.required_input_ordering); - - // let groupby_exprs = self.group_by - // .expr - // .iter() - // .map(|(item, _)| item.clone()) - // .collect::>(); - // let res = self.input.ordering_equivalence_properties().set_satisfy(&groupby_exprs); - // println!("res:{:?}", res); self.execute_typed(partition, context) .map(|stream| stream.into()) } diff --git a/datafusion/physical-plan/src/streaming.rs b/datafusion/physical-plan/src/streaming.rs index cb972fa41e3e..ad813cf54c8d 100644 --- a/datafusion/physical-plan/src/streaming.rs +++ b/datafusion/physical-plan/src/streaming.rs @@ -25,7 +25,9 @@ use async_trait::async_trait; use futures::stream::StreamExt; use datafusion_common::{internal_err, plan_err, DataFusionError, Result, Statistics}; -use datafusion_physical_expr::{LexOrdering, PhysicalSortExpr}; +use datafusion_physical_expr::{ + LexOrdering, OrderingEquivalenceProperties, PhysicalSortExpr, +}; use log::debug; use crate::display::{OutputOrderingDisplay, ProjectSchemaDisplay}; @@ -157,6 +159,14 @@ impl ExecutionPlan for StreamingTableExec { self.projected_output_ordering.as_deref() } + fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { + let mut oeq = OrderingEquivalenceProperties::new(self.schema()); + if let Some(ordering) = &self.projected_output_ordering { + oeq.add_new_orderings(&[ordering.clone()]) + } + oeq + } + fn children(&self) -> Vec> { vec![] } diff --git a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs index dc97fbe39de8..6d3d4f3a6454 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -61,22 +61,23 @@ use datafusion_physical_expr::{ }; use ahash::RandomState; +use datafusion_physical_expr::equivalence::PartitionSearchMode; use futures::stream::Stream; use futures::{ready, StreamExt}; use hashbrown::raw::RawTable; use indexmap::IndexMap; use log::debug; -#[derive(Debug, Clone, PartialEq)] -/// Specifies partition column properties in terms of input ordering -pub enum PartitionSearchMode { - /// None of the columns among the partition columns is ordered. - Linear, - /// Some columns of the partition columns are ordered but not all - PartiallySorted(Vec), - /// All Partition columns are ordered (Also empty case) - Sorted, -} +// #[derive(Debug, Clone, PartialEq)] +// /// Specifies partition column properties in terms of input ordering +// pub enum PartitionSearchMode { +// /// None of the columns among the partition columns is ordered. +// Linear, +// /// Some columns of the partition columns are ordered but not all +// PartiallySorted(Vec), +// /// All Partition columns are ordered (Also empty case) +// Sorted, +// } /// Window execution plan #[derive(Debug)] diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index f6196eb0b7ae..36a146d2043a 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -32,10 +32,7 @@ use crate::{ use arrow::datatypes::Schema; use arrow_schema::{DataType, Field, SchemaRef}; -use datafusion_common::utils::{ - find_indices, get_at_indices, is_sorted, longest_consecutive_prefix, - merge_and_order_indices, set_difference, -}; +use datafusion_common::utils::longest_consecutive_prefix; use datafusion_common::{DataFusionError, Result, ScalarValue}; use datafusion_expr::{ window_function::{BuiltInWindowFunction, WindowFunction}, @@ -48,13 +45,11 @@ use datafusion_physical_expr::{ AggregateExpr, OrderingEquivalenceProperties, PhysicalSortRequirement, }; -use itertools::{izip, Itertools}; - mod bounded_window_agg_exec; mod window_agg_exec; pub use bounded_window_agg_exec::BoundedWindowAggExec; -pub use bounded_window_agg_exec::PartitionSearchMode; +use datafusion_physical_expr::equivalence::PartitionSearchMode; pub use window_agg_exec::WindowAggExec; pub use datafusion_physical_expr::window::{ @@ -378,8 +373,9 @@ pub fn get_best_fitting_window( let partitionby_exprs = window_exprs[0].partition_by(); let orderby_keys = window_exprs[0].order_by(); let (should_reverse, partition_search_mode) = - if let Some((should_reverse, partition_search_mode)) = - can_skip_sort(partitionby_exprs, orderby_keys, input)? + if let Some((should_reverse, partition_search_mode)) = input + .ordering_equivalence_properties() + .get_window_mode(partitionby_exprs, orderby_keys)? { (should_reverse, partition_search_mode) } else { @@ -434,152 +430,6 @@ pub fn get_best_fitting_window( } } -/// Compares physical ordering (output ordering of the `input` operator) with -/// `partitionby_exprs` and `orderby_keys` to decide whether existing ordering -/// is sufficient to run the current window operator. -/// - A `None` return value indicates that we can not remove the sort in question -/// (input ordering is not sufficient to run current window executor). -/// - A `Some((bool, PartitionSearchMode))` value indicates that the window operator -/// can run with existing input ordering, so we can remove `SortExec` before it. -/// The `bool` field in the return value represents whether we should reverse window -/// operator to remove `SortExec` before it. The `PartitionSearchMode` field represents -/// the mode this window operator should work in to accomodate the existing ordering. -fn can_skip_sort( - partitionby_exprs: &[Arc], - orderby_keys: &[PhysicalSortExpr], - input: &Arc, -) -> Result> { - let physical_ordering = if let Some(physical_ordering) = input.output_ordering() { - physical_ordering - } else { - // If there is no physical ordering, there is no way to remove a - // sort, so immediately return. - return Ok(None); - }; - let orderby_exprs = convert_to_expr(orderby_keys); - let physical_ordering_exprs = convert_to_expr(physical_ordering); - // Get the indices of the ORDER BY expressions among input ordering expressions: - // Do this check considering alternative orderings. - let ob_indices = - get_indices_of_matching_exprs(&orderby_exprs, &physical_ordering_exprs); - if ob_indices.len() != orderby_exprs.len() { - // If all order by expressions are not in the input ordering, - // there is no way to remove a sort -- immediately return: - return Ok(None); - } - // Get the indices of the PARTITION BY expressions among input ordering expressions: - let pb_indices = - get_indices_of_matching_exprs(partitionby_exprs, &physical_ordering_exprs); - let ordered_merged_indices = merge_and_order_indices(&pb_indices, &ob_indices); - // Get the indices of the ORDER BY columns that don't appear in the - // PARTITION BY clause; i.e. calculate (ORDER BY columns) ∖ (PARTITION - // BY columns) where `∖` represents set difference. - let unique_ob_indices = set_difference(&ob_indices, &pb_indices); - if !is_sorted(&unique_ob_indices) { - // ORDER BY indices should be ascending ordered - return Ok(None); - } - let first_n = longest_consecutive_prefix(ordered_merged_indices); - let furthest_ob_index = *unique_ob_indices.last().unwrap_or(&0); - // Cannot skip sort if last order by index is not within consecutive prefix. - // For instance, if input is ordered by a, b, c, d for the expression - // `PARTITION BY a, ORDER BY b, d`, then `first_n` would be 2 (meaning a, b defines a - // prefix for input ordering). However, `furthest_ob_index` would be 3 as column d - // occurs at the 3rd index of the existing ordering. Hence, existing ordering would - // not be sufficient to run the current operator. - // However, for expression `PARTITION BY a, ORDER BY b, c, d`, `first_n` would be 4 (meaning - // a, b, c, d defines a prefix for input ordering). Similarly, `furthest_ob_index` would be - // 3 as column d occurs at the 3rd index of the existing ordering. Therefore, the existing - // ordering would be sufficient to run the current operator. - if first_n <= furthest_ob_index { - return Ok(None); - } - let input_orderby_columns = get_at_indices(physical_ordering, &unique_ob_indices)?; - let expected_orderby_columns = - get_at_indices(orderby_keys, find_indices(&ob_indices, &unique_ob_indices)?)?; - let should_reverse = if let Some(should_reverse) = check_alignments( - &input.schema(), - &input_orderby_columns, - &expected_orderby_columns, - )? { - should_reverse - } else { - // If ordering directions are not aligned, we cannot calculate the - // result without changing existing ordering. - return Ok(None); - }; - - let ordered_pb_indices = pb_indices.iter().copied().sorted().collect::>(); - // Determine how many elements in the PARTITION BY columns defines a consecutive range from zero. - let first_n = longest_consecutive_prefix(&ordered_pb_indices); - let mode = if first_n == partitionby_exprs.len() { - // All of the PARTITION BY columns defines a consecutive range from zero. - PartitionSearchMode::Sorted - } else if first_n > 0 { - // All of the PARTITION BY columns defines a consecutive range from zero. - let ordered_range = &ordered_pb_indices[0..first_n]; - let input_pb_exprs = get_at_indices(&physical_ordering_exprs, ordered_range)?; - let partially_ordered_indices = - get_indices_of_matching_exprs(&input_pb_exprs, partitionby_exprs); - PartitionSearchMode::PartiallySorted(partially_ordered_indices) - } else { - // None of the PARTITION BY columns defines a consecutive range from zero. - PartitionSearchMode::Linear - }; - - Ok(Some((should_reverse, mode))) -} - -/// Compares all the orderings in `physical_ordering` and `required`, decides -/// whether alignments match. A `None` return value indicates that current -/// column is not aligned. A `Some(bool)` value indicates otherwise, and signals -/// whether we should reverse the window expression in order to avoid sorting. -fn check_alignments( - schema: &SchemaRef, - physical_ordering: &[PhysicalSortExpr], - required: &[PhysicalSortExpr], -) -> Result> { - let result = izip!(physical_ordering, required) - .map(|(lhs, rhs)| check_alignment(schema, lhs, rhs)) - .collect::>>>()?; - Ok(if let Some(res) = result { - if !res.is_empty() { - let first = res[0]; - let all_same = res.into_iter().all(|elem| elem == first); - all_same.then_some(first) - } else { - Some(false) - } - } else { - // Cannot skip some of the requirements in the input. - None - }) -} - -/// Compares `physical_ordering` and `required` ordering, decides whether -/// alignments match. A `None` return value indicates that current column is -/// not aligned. A `Some(bool)` value indicates otherwise, and signals whether -/// we should reverse the window expression in order to avoid sorting. -fn check_alignment( - input_schema: &SchemaRef, - physical_ordering: &PhysicalSortExpr, - required: &PhysicalSortExpr, -) -> Result> { - Ok(if required.expr.eq(&physical_ordering.expr) { - let physical_opts = physical_ordering.options; - let required_opts = required.options; - if required.expr.nullable(input_schema)? { - let reverse = physical_opts == !required_opts; - (reverse || physical_opts == required_opts).then_some(reverse) - } else { - // If the column is not nullable, NULLS FIRST/LAST is not important. - Some(physical_opts.descending != required_opts.descending) - } - } else { - None - }) -} - #[cfg(test)] mod tests { use super::*; @@ -589,12 +439,14 @@ mod tests { use crate::streaming::StreamingTableExec; use crate::test::assert_is_pending; use crate::test::exec::{assert_strong_count_converges_to_zero, BlockingExec}; - use crate::windows::PartitionSearchMode::{Linear, PartiallySorted, Sorted}; use arrow::compute::SortOptions; use arrow::datatypes::{DataType, Field, SchemaRef}; use datafusion_execution::TaskContext; + use datafusion_physical_expr::equivalence::PartitionSearchMode::{ + Linear, PartiallySorted, Sorted, + }; use futures::FutureExt; fn create_test_schema() -> Result { @@ -758,15 +610,16 @@ mod tests { } #[tokio::test] - async fn test_is_column_aligned_nullable() -> Result<()> { + async fn test_satisfiy_nullable() -> Result<()> { let schema = create_test_schema()?; let params = vec![ - ((true, true), (false, false), Some(true)), - ((true, true), (false, true), None), - ((true, true), (true, false), None), - ((true, false), (false, true), Some(true)), - ((true, false), (false, false), None), - ((true, false), (true, true), None), + ((true, true), (false, false), false), + ((true, true), (false, true), false), + ((true, true), (true, false), false), + ((true, false), (false, true), false), + ((true, false), (false, false), false), + ((true, false), (true, true), false), + ((true, false), (true, false), true), ]; for ( (physical_desc, physical_nulls_first), @@ -788,7 +641,8 @@ mod tests { nulls_first: req_nulls_first, }, }; - let res = check_alignment(&schema, &physical_ordering, &required_ordering)?; + let res = + physical_ordering.satisfy_with_schema(&required_ordering.into(), &schema); assert_eq!(res, expected); } @@ -796,16 +650,17 @@ mod tests { } #[tokio::test] - async fn test_is_column_aligned_non_nullable() -> Result<()> { + async fn test_satisfy_non_nullable() -> Result<()> { let schema = create_test_schema()?; let params = vec![ - ((true, true), (false, false), Some(true)), - ((true, true), (false, true), Some(true)), - ((true, true), (true, false), Some(false)), - ((true, false), (false, true), Some(true)), - ((true, false), (false, false), Some(true)), - ((true, false), (true, true), Some(false)), + ((true, true), (false, false), false), + ((true, true), (false, true), false), + ((true, true), (true, false), true), + ((true, false), (false, true), false), + ((true, false), (false, false), false), + ((true, false), (true, true), true), + ((true, false), (true, false), true), ]; for ( (physical_desc, physical_nulls_first), @@ -827,7 +682,8 @@ mod tests { nulls_first: req_nulls_first, }, }; - let res = check_alignment(&schema, &physical_ordering, &required_ordering)?; + let res = + physical_ordering.satisfy_with_schema(&required_ordering.into(), &schema); assert_eq!(res, expected); } @@ -835,7 +691,7 @@ mod tests { } #[tokio::test] - async fn test_can_skip_ordering_exhaustive() -> Result<()> { + async fn test_get_window_mode_exhaustive() -> Result<()> { let test_schema = create_test_schema3()?; // Columns a,c are nullable whereas b,d are not nullable. // Source is sorted by a ASC NULLS FIRST, b ASC NULLS FIRST, c ASC NULLS FIRST, d ASC NULLS FIRST @@ -941,8 +797,9 @@ mod tests { let options = SortOptions::default(); order_by_exprs.push(PhysicalSortExpr { expr, options }); } - let res = - can_skip_sort(&partition_by_exprs, &order_by_exprs, &exec_unbounded)?; + let res = exec_unbounded + .ordering_equivalence_properties() + .get_window_mode(&partition_by_exprs, &order_by_exprs)?; // Since reversibility is not important in this test. Convert Option<(bool, PartitionSearchMode)> to Option let res = res.map(|(_, mode)| mode); assert_eq!( @@ -955,7 +812,7 @@ mod tests { } #[tokio::test] - async fn test_can_skip_ordering() -> Result<()> { + async fn test_get_window_mode() -> Result<()> { let test_schema = create_test_schema3()?; // Columns a,c are nullable whereas b,d are not nullable. // Source is sorted by a ASC NULLS FIRST, b ASC NULLS FIRST, c ASC NULLS FIRST, d ASC NULLS FIRST @@ -1106,7 +963,7 @@ mod tests { } assert_eq!( - can_skip_sort(&partition_by_exprs, &order_by_exprs, &exec_unbounded)?, + exec_unbounded.ordering_equivalence_properties().get_window_mode(&partition_by_exprs, &order_by_exprs)?, *expected, "Unexpected result for in unbounded test case#: {case_idx:?}, case: {test_case:?}" ); diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index 8257f9aa3458..708f8872fda2 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -25,6 +25,7 @@ use datafusion::datasource::file_format::file_compression_type::FileCompressionT use datafusion::datasource::physical_plan::{AvroExec, CsvExec, ParquetExec}; use datafusion::execution::runtime_env::RuntimeEnv; use datafusion::execution::FunctionRegistry; +use datafusion::physical_expr::equivalence::PartitionSearchMode; use datafusion::physical_plan::aggregates::{create_aggregate_expr, AggregateMode}; use datafusion::physical_plan::aggregates::{AggregateExec, PhysicalGroupBy}; use datafusion::physical_plan::analyze::AnalyzeExec; @@ -43,9 +44,7 @@ use datafusion::physical_plan::repartition::RepartitionExec; use datafusion::physical_plan::sorts::sort::SortExec; use datafusion::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; use datafusion::physical_plan::union::UnionExec; -use datafusion::physical_plan::windows::{ - BoundedWindowAggExec, PartitionSearchMode, WindowAggExec, -}; +use datafusion::physical_plan::windows::{BoundedWindowAggExec, WindowAggExec}; use datafusion::physical_plan::{ udaf, AggregateExpr, ExecutionPlan, Partitioning, PhysicalExpr, WindowExpr, }; From c4d7c992ef372745379037fb90c7867cfb781558 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Mon, 2 Oct 2023 17:26:37 +0300 Subject: [PATCH 012/122] Simplifications --- datafusion/physical-expr/src/equivalence.rs | 6 +++--- datafusion/physical-expr/src/utils.rs | 15 +++------------ datafusion/physical-expr/src/window/built_in.rs | 4 ++-- datafusion/physical-plan/src/windows/mod.rs | 7 ++++--- 4 files changed, 12 insertions(+), 20 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 04f34ec54655..eea4ba1c342c 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -16,7 +16,7 @@ // under the License. use crate::expressions::{CastExpr, Column}; -use crate::utils::get_indices_of_matching_exprs; +use crate::utils::get_indices_of_exprs_strict; use crate::{ physical_exprs_contains, reverse_order_bys, LexOrdering, LexOrderingRef, LexOrderingReq, PhysicalExpr, PhysicalSortExpr, PhysicalSortRequirement, @@ -809,7 +809,7 @@ impl OrderingEquivalenceProperties { .map(|sort_expr| sort_expr.expr.clone()) .collect::>(); let mut ordered_indices = - get_indices_of_matching_exprs(&exprs_normalized, &ordering_exprs); + get_indices_of_exprs_strict(&exprs_normalized, &ordering_exprs); ordered_indices.sort(); // Find out how many expressions of the existing ordering define ordering // for expressions in the GROUP BY clause. For example, if the input is @@ -823,7 +823,7 @@ impl OrderingEquivalenceProperties { // this would produce 1, 0; meaning 1st and 0th entries (a, b) among the // GROUP BY expressions b, a, d match input ordering. let indices = - get_indices_of_matching_exprs(&ordered_exprs, &exprs_normalized); + get_indices_of_exprs_strict(&ordered_exprs, &exprs_normalized); best = indices .iter() .enumerate() diff --git a/datafusion/physical-expr/src/utils.rs b/datafusion/physical-expr/src/utils.rs index b04a0c68a95b..d73064827355 100644 --- a/datafusion/physical-expr/src/utils.rs +++ b/datafusion/physical-expr/src/utils.rs @@ -172,15 +172,6 @@ pub fn convert_to_expr>( .collect() } -/// This function finds the indices of `targets` within `items`, taking into -/// account equivalences according to `equal_properties`. -pub fn get_indices_of_matching_exprs( - targets: &[Arc], - items: &[Arc], -) -> Vec { - get_indices_of_exprs_strict(targets, items) -} - /// This function finds the indices of `targets` within `items` using strict /// equality. pub fn get_indices_of_exprs_strict>>( @@ -787,7 +778,7 @@ mod tests { } #[test] - fn test_get_indices_of_matching_exprs() { + fn test_get_indices_of_exprs_strict() { let list1: Vec> = vec![ Arc::new(Column::new("a", 0)), Arc::new(Column::new("b", 1)), @@ -799,8 +790,8 @@ mod tests { Arc::new(Column::new("c", 2)), Arc::new(Column::new("a", 0)), ]; - assert_eq!(get_indices_of_matching_exprs(&list1, &list2), vec![2, 0, 1]); - assert_eq!(get_indices_of_matching_exprs(&list2, &list1), vec![1, 2, 0]); + assert_eq!(get_indices_of_exprs_strict(&list1, &list2), vec![2, 0, 1]); + assert_eq!(get_indices_of_exprs_strict(&list2, &list1), vec![1, 2, 0]); } #[test] diff --git a/datafusion/physical-expr/src/window/built_in.rs b/datafusion/physical-expr/src/window/built_in.rs index 9be13d1c20a9..9bb360ff3699 100644 --- a/datafusion/physical-expr/src/window/built_in.rs +++ b/datafusion/physical-expr/src/window/built_in.rs @@ -25,7 +25,7 @@ use super::BuiltInWindowFunctionExpr; use super::WindowExpr; use crate::equivalence::OrderingEquivalenceBuilder; use crate::expressions::PhysicalSortExpr; -use crate::utils::{convert_to_expr, get_indices_of_matching_exprs}; +use crate::utils::{convert_to_expr, get_indices_of_exprs_strict}; use crate::window::window_expr::{get_orderby_values, WindowFn}; use crate::window::{PartitionBatches, PartitionWindowAggStates, WindowState}; use crate::{reverse_order_bys, PhysicalExpr}; @@ -91,7 +91,7 @@ impl BuiltInWindowExpr { let existing_ordering = builder.existing_ordering(); let existing_ordering_exprs = convert_to_expr(existing_ordering); // Get indices of the PARTITION BY expressions among input ordering expressions: - let pb_indices = get_indices_of_matching_exprs( + let pb_indices = get_indices_of_exprs_strict( &self.partition_by, &existing_ordering_exprs, ); diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index 36a146d2043a..1950cd2893b4 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -40,7 +40,7 @@ use datafusion_expr::{ }; use datafusion_physical_expr::{ equivalence::OrderingEquivalenceBuilder, - utils::{convert_to_expr, get_indices_of_matching_exprs}, + utils::convert_to_expr, window::{BuiltInWindowFunctionExpr, SlidingAggregateWindowExpr}, AggregateExpr, OrderingEquivalenceProperties, PhysicalSortRequirement, }; @@ -50,6 +50,7 @@ mod window_agg_exec; pub use bounded_window_agg_exec::BoundedWindowAggExec; use datafusion_physical_expr::equivalence::PartitionSearchMode; +use datafusion_physical_expr::utils::get_indices_of_exprs_strict; pub use window_agg_exec::WindowAggExec; pub use datafusion_physical_expr::window::{ @@ -320,9 +321,9 @@ pub(crate) fn get_ordered_partition_by_indices( let input_ordering_exprs = convert_to_expr(input_ordering); // let equal_properties = || input.equivalence_properties(); let input_places = - get_indices_of_matching_exprs(&input_ordering_exprs, partition_by_exprs); + get_indices_of_exprs_strict(&input_ordering_exprs, partition_by_exprs); let mut partition_places = - get_indices_of_matching_exprs(partition_by_exprs, &input_ordering_exprs); + get_indices_of_exprs_strict(partition_by_exprs, &input_ordering_exprs); partition_places.sort(); let first_n = longest_consecutive_prefix(partition_places); input_places[0..first_n].to_vec() From aa7de00523851b32e870bdcf83ff613fbf1ece7c Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Mon, 2 Oct 2023 17:48:54 +0300 Subject: [PATCH 013/122] Use set_satisfy api --- datafusion/physical-plan/src/windows/mod.rs | 24 ++++++++++----------- 1 file changed, 11 insertions(+), 13 deletions(-) diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index 1950cd2893b4..0ad0b3ccbbdf 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -32,7 +32,6 @@ use crate::{ use arrow::datatypes::Schema; use arrow_schema::{DataType, Field, SchemaRef}; -use datafusion_common::utils::longest_consecutive_prefix; use datafusion_common::{DataFusionError, Result, ScalarValue}; use datafusion_expr::{ window_function::{BuiltInWindowFunction, WindowFunction}, @@ -40,7 +39,6 @@ use datafusion_expr::{ }; use datafusion_physical_expr::{ equivalence::OrderingEquivalenceBuilder, - utils::convert_to_expr, window::{BuiltInWindowFunctionExpr, SlidingAggregateWindowExpr}, AggregateExpr, OrderingEquivalenceProperties, PhysicalSortRequirement, }; @@ -50,7 +48,6 @@ mod window_agg_exec; pub use bounded_window_agg_exec::BoundedWindowAggExec; use datafusion_physical_expr::equivalence::PartitionSearchMode; -use datafusion_physical_expr::utils::get_indices_of_exprs_strict; pub use window_agg_exec::WindowAggExec; pub use datafusion_physical_expr::window::{ @@ -317,16 +314,17 @@ pub(crate) fn get_ordered_partition_by_indices( partition_by_exprs: &[Arc], input: &Arc, ) -> Vec { - let input_ordering = input.output_ordering().unwrap_or(&[]); - let input_ordering_exprs = convert_to_expr(input_ordering); - // let equal_properties = || input.equivalence_properties(); - let input_places = - get_indices_of_exprs_strict(&input_ordering_exprs, partition_by_exprs); - let mut partition_places = - get_indices_of_exprs_strict(partition_by_exprs, &input_ordering_exprs); - partition_places.sort(); - let first_n = longest_consecutive_prefix(partition_places); - input_places[0..first_n].to_vec() + if let Some(indices_and_orderings) = input + .ordering_equivalence_properties() + .set_satisfy(partition_by_exprs) + { + indices_and_orderings + .into_iter() + .map(|(idx, _options)| idx) + .collect() + } else { + vec![] + } } pub(crate) fn window_ordering_equivalence( From 6dbc85b1504e9b134ed354a22b3bd5d2b8360aab Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 3 Oct 2023 09:45:24 +0300 Subject: [PATCH 014/122] Use utils for aggregate --- .../src/physical_optimizer/enforce_sorting.rs | 36 +++--- datafusion/physical-expr/src/equivalence.rs | 2 +- .../physical-plan/src/aggregates/mod.rs | 110 ++++++++++++++---- 3 files changed, 103 insertions(+), 45 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index a9372c42f46b..1d6a4942d221 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -2999,12 +2999,11 @@ mod tmp_tests { let expected_optimized_lines: Vec<&str> = vec![ "ProjectionExec: expr=[a@0 as a, LAST_VALUE(r.b) ORDER BY [r.a ASC NULLS LAST]@3 as last_col1]", - " AggregateExec: mode=Final, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[FIRST_VALUE(r.b)], ordering_mode=PartiallyOrdered", - " AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[FIRST_VALUE(r.b)], ordering_mode=PartiallyOrdered", - " CoalesceBatchesExec: target_batch_size=8192", - " HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@0, a@0)]", - " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true", - " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], has_header=true", + " AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[LAST_VALUE(r.b)], ordering_mode=PartiallyOrdered", + " CoalesceBatchesExec: target_batch_size=8192", + " HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@0, a@0)]", + " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true", + " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], has_header=true", ]; // Get string representation of the plan @@ -3058,21 +3057,20 @@ mod tmp_tests { "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", " SortExec: expr=[a@0 ASC NULLS LAST]", " ProjectionExec: expr=[a@0 as a, LAST_VALUE(r.b) ORDER BY [r.a ASC NULLS LAST]@3 as last_col1]", - " AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[FIRST_VALUE(r.b)]", + " AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[LAST_VALUE(r.b)]", " CoalesceBatchesExec: target_batch_size=8192", " RepartitionExec: partitioning=Hash([a@0, b@1, c@2], 4), input_partitions=4", - " AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[FIRST_VALUE(r.b)], ordering_mode=PartiallyOrdered", - " SortExec: expr=[a@3 DESC]", - " CoalesceBatchesExec: target_batch_size=8192", - " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a@0)]", - " CoalesceBatchesExec: target_batch_size=8192", - " RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4", - " RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1", - " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true", - " CoalesceBatchesExec: target_batch_size=8192", - " RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4", - " RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1", - " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], has_header=true", + " AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[LAST_VALUE(r.b)], ordering_mode=PartiallyOrdered", + " CoalesceBatchesExec: target_batch_size=8192", + " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a@0)]", + " CoalesceBatchesExec: target_batch_size=8192", + " RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4", + " RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1", + " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true", + " CoalesceBatchesExec: target_batch_size=8192", + " SortPreservingRepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4", + " RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1", + " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], has_header=true", ]; // Get string representation of the plan diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index eea4ba1c342c..94d845fd7dca 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -1409,7 +1409,7 @@ pub fn ordering_equivalence_properties_helper( /// entries that have same physical expression inside the given vector `input`. /// `vec![a Some(Asc), a Some(Desc)]` is collapsed to the `vec![a Some(Asc)]`. Since /// when same expression is already seen before, following expressions are redundant. -fn collapse_lex_req(input: LexOrderingReq) -> LexOrderingReq { +pub fn collapse_lex_req(input: LexOrderingReq) -> LexOrderingReq { let mut output = vec![]; for item in input { if output diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index a4d93e434457..1815afaee69d 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -41,7 +41,7 @@ use datafusion_physical_expr::{ use arrow_schema::SortOptions; use datafusion_common::tree_node::{Transformed, TreeNode}; -use itertools::Itertools; +use itertools::{izip, Itertools}; use std::any::Any; use std::collections::HashMap; use std::sync::Arc; @@ -54,8 +54,10 @@ mod topk; mod topk_stream; use crate::aggregates::topk_stream::GroupedTopKAggregateStream; +use crate::windows::get_ordered_partition_by_indices; pub use datafusion_expr::AggregateFunction; use datafusion_physical_expr::aggregate::is_order_sensitive; +use datafusion_physical_expr::equivalence::{collapse_lex_req, PartitionSearchMode}; pub use datafusion_physical_expr::expressions::create_aggregate_expr; use datafusion_physical_expr::expressions::{Max, Min}; @@ -292,6 +294,7 @@ pub struct AggregateExec { /// Stores mode and output ordering information for the `AggregateExec`. aggregation_ordering: Option, required_input_ordering: Option, + partition_search_mode: PartitionSearchMode, } /// Calculates the working mode for `GROUP BY` queries. @@ -621,7 +624,7 @@ impl AggregateExec { }) }) .collect::>(); - let mut aggregator_reverse_reqs = None; + // let mut aggregator_reverse_reqs = None; // Currently we support order-sensitive aggregation only in `Single` mode. // For `Final` and `FinalPartitioned` modes, we cannot guarantee they will receive // data according to ordering requirements. As long as we cannot produce correct result @@ -633,21 +636,71 @@ impl AggregateExec { get_finest_requirement(&mut aggr_expr, &mut order_by_expr, || { input.ordering_equivalence_properties() })?; - let aggregator_requirement = requirement - .as_ref() - .map(|exprs| PhysicalSortRequirement::from_sort_exprs(exprs.iter())); - let aggregator_reqs = aggregator_requirement.unwrap_or(vec![]); - // If all aggregate expressions are reversible, also consider reverse - // requirement(s). The reason is that existing ordering may satisfy the - // given requirement or its reverse. By considering both, we can generate better plans. - if aggr_expr + let req = requirement.unwrap_or(vec![]); + let groupby_exprs = group_by + .expr .iter() - .all(|expr| !is_order_sensitive(expr) || expr.reverse_expr().is_some()) + .map(|(item, _)| item.clone()) + .collect::>(); + let mut partition_search_mode = PartitionSearchMode::Linear; + let mut new_requirement: Vec = vec![]; + let indices = get_ordered_partition_by_indices(&groupby_exprs, &input); + let elem = indices + .into_iter() + .map(|idx| PhysicalSortRequirement { + expr: groupby_exprs[idx].clone(), + options: None, + }) + .collect::>(); + new_requirement.extend(elem); + let input_oeq = input.ordering_equivalence_properties(); + if let Some((should_reverse, mode)) = + input_oeq.get_window_mode(&groupby_exprs, &req)? { - aggregator_reverse_reqs = requirement.map(|reqs| { - PhysicalSortRequirement::from_sort_exprs(reverse_order_bys(&reqs).iter()) - }); + let all_reversible = aggr_expr + .iter() + .all(|expr| !is_order_sensitive(expr) || expr.reverse_expr().is_some()); + if should_reverse && all_reversible { + izip!(aggr_expr.iter_mut(), order_by_expr.iter_mut()).for_each( + |(aggr, order_by)| { + if let Some(reverse) = aggr.reverse_expr() { + *aggr = reverse; + } else { + unreachable!(); + } + *order_by = order_by.as_ref().map(|ob| reverse_order_bys(ob)); + }, + ); + let reverse_req = PhysicalSortRequirement::from_sort_exprs( + reverse_order_bys(&req).iter(), + ); + new_requirement.extend(reverse_req); + } else { + let req = PhysicalSortRequirement::from_sort_exprs(&req); + new_requirement.extend(req); + } + partition_search_mode = mode; + } else { + let req = PhysicalSortRequirement::from_sort_exprs(&req); + new_requirement.extend(req); } + new_requirement = collapse_lex_req(new_requirement); + + // let aggregator_requirement = requirement + // .as_ref() + // .map(|exprs| PhysicalSortRequirement::from_sort_exprs(exprs.iter())); + // let aggregator_reqs = aggregator_requirement.unwrap_or(vec![]); + // // If all aggregate expressions are reversible, also consider reverse + // // requirement(s). The reason is that existing ordering may satisfy the + // // given requirement or its reverse. By considering both, we can generate better plans. + // if aggr_expr + // .iter() + // .all(|expr| !is_order_sensitive(expr) || expr.reverse_expr().is_some()) + // { + // aggregator_reverse_reqs = requirement.map(|reqs| { + // PhysicalSortRequirement::from_sort_exprs(reverse_order_bys(&reqs).iter()) + // }); + // } // construct a map from the input columns to the output columns of the Aggregation let mut columns_map: HashMap> = HashMap::new(); @@ -680,19 +733,25 @@ impl AggregateExec { })?; source_to_target_mapping.push((source_expr, target_expr)); } + // println!("source_to_target_mapping: {:?}", source_to_target_mapping); let mut aggregation_ordering = calc_aggregation_ordering(&input, &group_by); - // println!("start aggregation_ordering: {:?}", aggregation_ordering); - let required_input_ordering = calc_required_input_ordering( - &input, - &mut aggr_expr, - &mut order_by_expr, - aggregator_reqs, - aggregator_reverse_reqs, - &mut aggregation_ordering, - &mode, - )?; - // println!("end aggregation_ordering: {:?}", aggregation_ordering); + let required_input_ordering = if new_requirement.is_empty() { + None + } else { + Some(new_requirement) + }; + // // println!("start aggregation_ordering: {:?}", aggregation_ordering); + // let required_input_ordering = calc_required_input_ordering( + // &input, + // &mut aggr_expr, + // &mut order_by_expr, + // aggregator_reqs, + // aggregator_reverse_reqs, + // &mut aggregation_ordering, + // &mode, + // )?; + // // println!("end aggregation_ordering: {:?}", aggregation_ordering); Ok(AggregateExec { mode, @@ -709,6 +768,7 @@ impl AggregateExec { aggregation_ordering, required_input_ordering, limit: None, + partition_search_mode, }) } From c1656c7431f265ae07cc6a76a37e7733b7f52ee0 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 3 Oct 2023 10:47:54 +0300 Subject: [PATCH 015/122] Minor changes --- .../src/physical_optimizer/enforce_sorting.rs | 14 +- datafusion/physical-expr/src/equivalence.rs | 6 +- .../physical-plan/src/aggregates/mod.rs | 226 ++++-------------- .../sqllogictest/test_files/groupby.slt | 32 +-- datafusion/sqllogictest/test_files/window.slt | 2 +- 5 files changed, 76 insertions(+), 204 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index 1d6a4942d221..fbd8e99e56fd 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -2304,7 +2304,7 @@ mod tmp_tests { let expected_optimized_lines: Vec<&str> = vec![ "ProjectionExec: expr=[col0@0 as col0, LAST_VALUE(r.col1) ORDER BY [r.col0 ASC NULLS LAST]@3 as last_col1]", - " AggregateExec: mode=Single, gby=[col0@0 as col0, col1@1 as col1, col2@2 as col2], aggr=[LAST_VALUE(r.col1)], ordering_mode=PartiallyOrdered", + " AggregateExec: mode=Single, gby=[col0@0 as col0, col1@1 as col1, col2@2 as col2], aggr=[LAST_VALUE(r.col1)], ordering_mode=PartiallySorted([0])", " SortExec: expr=[col0@3 ASC NULLS LAST]", " CoalesceBatchesExec: target_batch_size=8192", " HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(col0@0, col0@0)]", @@ -2351,7 +2351,7 @@ mod tmp_tests { " AggregateExec: mode=FinalPartitioned, gby=[col0@0 as col0, col1@1 as col1, col2@2 as col2], aggr=[LAST_VALUE(r.col1)]", " CoalesceBatchesExec: target_batch_size=8192", " RepartitionExec: partitioning=Hash([col0@0, col1@1, col2@2], 2), input_partitions=2", - " AggregateExec: mode=Partial, gby=[col0@0 as col0, col1@1 as col1, col2@2 as col2], aggr=[LAST_VALUE(r.col1)], ordering_mode=PartiallyOrdered", + " AggregateExec: mode=Partial, gby=[col0@0 as col0, col1@1 as col1, col2@2 as col2], aggr=[LAST_VALUE(r.col1)], ordering_mode=PartiallySorted([0])", " SortExec: expr=[col0@3 ASC NULLS LAST]", " CoalesceBatchesExec: target_batch_size=8192", " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(col0@0, col0@0)]", @@ -2698,7 +2698,7 @@ mod tmp_tests { let expected_optimized_lines: Vec<&str> = vec![ "ProjectionExec: expr=[a@1 as a, d@0 as d, SUM(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]@2 as summation1]", - " AggregateExec: mode=Single, gby=[d@2 as d, a@0 as a], aggr=[SUM(annotated_data_infinite2.c)], ordering_mode=PartiallyOrdered", + " AggregateExec: mode=Single, gby=[d@2 as d, a@0 as a], aggr=[SUM(annotated_data_infinite2.c)], ordering_mode=PartiallySorted([1])", " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", ]; @@ -2999,7 +2999,7 @@ mod tmp_tests { let expected_optimized_lines: Vec<&str> = vec![ "ProjectionExec: expr=[a@0 as a, LAST_VALUE(r.b) ORDER BY [r.a ASC NULLS LAST]@3 as last_col1]", - " AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[LAST_VALUE(r.b)], ordering_mode=PartiallyOrdered", + " AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[LAST_VALUE(r.b)], ordering_mode=PartiallySorted([0])", " CoalesceBatchesExec: target_batch_size=8192", " HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@0, a@0)]", " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true", @@ -3060,7 +3060,7 @@ mod tmp_tests { " AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[LAST_VALUE(r.b)]", " CoalesceBatchesExec: target_batch_size=8192", " RepartitionExec: partitioning=Hash([a@0, b@1, c@2], 4), input_partitions=4", - " AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[LAST_VALUE(r.b)], ordering_mode=PartiallyOrdered", + " AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[LAST_VALUE(r.b)], ordering_mode=PartiallySorted([0])", " CoalesceBatchesExec: target_batch_size=8192", " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a@0)]", " CoalesceBatchesExec: target_batch_size=8192", @@ -3136,7 +3136,7 @@ mod tmp_tests { let expected = vec![ "ProjectionExec: expr=[amount_usd@0 as amount_usd]", " ProjectionExec: expr=[LAST_VALUE(lineitem.l_d) ORDER BY [lineitem.l_a ASC NULLS LAST]@1 as amount_usd, row_n@0 as row_n]", - " AggregateExec: mode=Single, gby=[row_n@2 as row_n], aggr=[LAST_VALUE(lineitem.l_d)], ordering_mode=FullyOrdered", + " AggregateExec: mode=Single, gby=[row_n@2 as row_n], aggr=[LAST_VALUE(lineitem.l_d)], ordering_mode=Sorted", " ProjectionExec: expr=[l_a@0 as l_a, l_d@1 as l_d, row_n@4 as row_n]", " CoalesceBatchesExec: target_batch_size=8192", " HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(l_d@1, o_d@1)], filter=CAST(l_a@0 AS Int64) >= CAST(o_a@1 AS Int64) - 10", @@ -3196,7 +3196,7 @@ mod tmp_tests { let expected = vec![ "ProjectionExec: expr=[amount_usd@0 as amount_usd]", " ProjectionExec: expr=[LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST]@1 as amount_usd, row_n@0 as row_n]", - " AggregateExec: mode=Single, gby=[row_n@2 as row_n], aggr=[LAST_VALUE(l.d)], ordering_mode=FullyOrdered", + " AggregateExec: mode=Single, gby=[row_n@2 as row_n], aggr=[LAST_VALUE(l.d)], ordering_mode=Sorted", " ProjectionExec: expr=[a@0 as a, d@1 as d, row_n@4 as row_n]", " CoalesceBatchesExec: target_batch_size=8192", " HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d@1, d@1)], filter=CAST(a@0 AS Int64) >= CAST(a@1 AS Int64) - 10", diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 94d845fd7dca..19ed32204c7b 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -662,9 +662,9 @@ impl OrderingEquivalenceProperties { if children.is_empty() { for (source, target) in source_to_target_mapping.iter() { if source.eq(expr) - || old_eq_class - .iter() - .any(|eq_class| eq_class.iter().any(|item| item.eq(source))) + // || old_eq_class + // .iter() + // .any(|eq_class| eq_class.iter().any(|item| item.eq(expr))) { return Some(target.clone()); } else if let Some(group) = Self::get_eq_class_group(old_eq_class, source) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 1815afaee69d..c46aa051197a 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -60,6 +60,7 @@ use datafusion_physical_expr::aggregate::is_order_sensitive; use datafusion_physical_expr::equivalence::{collapse_lex_req, PartitionSearchMode}; pub use datafusion_physical_expr::expressions::create_aggregate_expr; use datafusion_physical_expr::expressions::{Max, Min}; +use crate::joins::StreamJoinPartitionMode::Partitioned; use super::DisplayAs; @@ -295,6 +296,7 @@ pub struct AggregateExec { aggregation_ordering: Option, required_input_ordering: Option, partition_search_mode: PartitionSearchMode, + output_ordering: Option, } /// Calculates the working mode for `GROUP BY` queries. @@ -451,114 +453,6 @@ fn print_plan(plan: &Arc) { println!("{:#?}", actual); } -/// Calculate the required input ordering for the [`AggregateExec`] by considering -/// ordering requirements of order-sensitive aggregation functions. -fn calc_required_input_ordering( - input: &Arc, - aggr_exprs: &mut [Arc], - order_by_exprs: &mut [Option], - aggregator_reqs: LexOrderingReq, - aggregator_reverse_reqs: Option, - aggregation_ordering: &mut Option, - mode: &AggregateMode, -) -> Result> { - let mut required_input_ordering = vec![]; - // Boolean shows that whether `required_input_ordering` stored comes from - // `aggregator_reqs` or `aggregator_reverse_reqs` - let mut reverse_req = false; - // If reverse aggregator is None, there is no way to run aggregators in reverse mode. Hence ignore it during analysis - let aggregator_requirements = - if let Some(aggregator_reverse_reqs) = aggregator_reverse_reqs { - // If existing ordering doesn't satisfy requirement, we should do calculations - // on naive requirement (by convention, otherwise the final plan will be unintuitive), - // even if reverse ordering is possible. - // Hence, while iterating consider naive requirement last, by this way - // we prioritize naive requirement over reverse requirement, when - // reverse requirement is not helpful with removing SortExec from the plan. - vec![(true, aggregator_reverse_reqs), (false, aggregator_reqs)] - } else { - vec![(false, aggregator_reqs)] - }; - for (is_reverse, aggregator_requirement) in aggregator_requirements.into_iter() { - if let Some(AggregationOrdering { - // If the mode is FullyOrdered or PartiallyOrdered (i.e. we are - // running with bounded memory, without breaking the pipeline), - // then we append the aggregator ordering requirement to the existing - // ordering. This way, we can still run with bounded memory. - mode: GroupByOrderMode::FullyOrdered | GroupByOrderMode::PartiallyOrdered, - order_indices, - .. - }) = aggregation_ordering - { - // Get the section of the input ordering that enables us to run in - // FullyOrdered or PartiallyOrdered modes: - let requirement_prefix = - if let Some(existing_ordering) = input.output_ordering() { - // print_plan(&input); - // println!("existing ordering:{:?}", existing_ordering); - &existing_ordering[0..order_indices.len()] - } else { - &[] - }; - let mut requirement = - PhysicalSortRequirement::from_sort_exprs(requirement_prefix.iter()); - for req in aggregator_requirement { - // Final and FinalPartitioned modes don't enforce ordering - // requirements since order-sensitive aggregators handle such - // requirements during merging. - if mode.is_first_stage() - && requirement.iter().all(|item| req.expr.ne(&item.expr)) - { - requirement.push(req); - } - } - required_input_ordering = requirement; - } else if mode.is_first_stage() { - required_input_ordering = aggregator_requirement; - } - // Keep track of the direction from which required_input_ordering is constructed: - reverse_req = is_reverse; - // If all the order-sensitive aggregate functions are reversible (e.g. all the - // order-sensitive aggregators are either FIRST_VALUE or LAST_VALUE), then we can - // run aggregate expressions either in the given required ordering, (i.e. finest - // requirement that satisfies every aggregate function requirement) or its reverse - // (opposite) direction. We analyze these two possibilities, and use the version that - // satisfies existing ordering. This enables us to avoid an extra sort step in the final - // plan. If neither version satisfies the existing ordering, we use the given ordering - // requirement. In short, if running aggregators in reverse order help us to avoid a - // sorting step, we do so. Otherwise, we use the aggregators as is. - if input - .ordering_equivalence_properties() - .ordering_satisfy_requirement_concrete(&required_input_ordering) - { - break; - } - } - // If `required_input_ordering` is constructed using the reverse requirement, we - // should reverse each `aggr_expr` in order to correctly calculate their results - // in reverse order. - if reverse_req { - aggr_exprs - .iter_mut() - .zip(order_by_exprs.iter_mut()) - .map(|(aggr_expr, ob_expr)| { - if is_order_sensitive(aggr_expr) { - if let Some(reverse) = aggr_expr.reverse_expr() { - *aggr_expr = reverse; - *ob_expr = ob_expr.as_ref().map(|obs| reverse_order_bys(obs)); - } else { - return plan_err!( - "Aggregate expression should have a reverse expression" - ); - } - } - Ok(()) - }) - .collect::>>()?; - } - Ok((!required_input_ordering.is_empty()).then_some(required_input_ordering)) -} - /// Check whether group by expression contains all of the expression inside `requirement` // As an example Group By (c,b,a) contains all of the expressions in the `requirement`: (a ASC, b DESC) fn group_by_contains_all_requirements( @@ -643,71 +537,60 @@ impl AggregateExec { .map(|(item, _)| item.clone()) .collect::>(); let mut partition_search_mode = PartitionSearchMode::Linear; - let mut new_requirement: Vec = vec![]; - let indices = get_ordered_partition_by_indices(&groupby_exprs, &input); - let elem = indices - .into_iter() - .map(|idx| PhysicalSortRequirement { - expr: groupby_exprs[idx].clone(), - options: None, - }) - .collect::>(); - new_requirement.extend(elem); let input_oeq = input.ordering_equivalence_properties(); - if let Some((should_reverse, mode)) = - input_oeq.get_window_mode(&groupby_exprs, &req)? - { - let all_reversible = aggr_expr - .iter() - .all(|expr| !is_order_sensitive(expr) || expr.reverse_expr().is_some()); - if should_reverse && all_reversible { - izip!(aggr_expr.iter_mut(), order_by_expr.iter_mut()).for_each( - |(aggr, order_by)| { - if let Some(reverse) = aggr.reverse_expr() { - *aggr = reverse; - } else { - unreachable!(); - } - *order_by = order_by.as_ref().map(|ob| reverse_order_bys(ob)); - }, - ); - let reverse_req = PhysicalSortRequirement::from_sort_exprs( - reverse_order_bys(&req).iter(), - ); - new_requirement.extend(reverse_req); + let mut new_requirement: Vec = vec![]; + if group_by.is_single() && !groupby_exprs.is_empty(){ + println!("groupby_exprs: {:?}, req:{:?}", groupby_exprs, req); + println!("input_oeq: {:?}", input_oeq); + let indices = get_ordered_partition_by_indices(&groupby_exprs, &input); + let elem = indices + .into_iter() + .map(|idx| PhysicalSortRequirement { + expr: groupby_exprs[idx].clone(), + options: None, + }) + .collect::>(); + new_requirement.extend(elem); + if let Some((should_reverse, mode)) = + input_oeq.get_window_mode(&groupby_exprs, &req)? + { + let all_reversible = aggr_expr + .iter() + .all(|expr| !is_order_sensitive(expr) || expr.reverse_expr().is_some()); + if should_reverse && all_reversible { + izip!(aggr_expr.iter_mut(), order_by_expr.iter_mut()).for_each( + |(aggr, order_by)| { + if let Some(reverse) = aggr.reverse_expr() { + *aggr = reverse; + } else { + unreachable!(); + } + *order_by = order_by.as_ref().map(|ob| reverse_order_bys(ob)); + }, + ); + let reverse_req = PhysicalSortRequirement::from_sort_exprs( + reverse_order_bys(&req).iter(), + ); + new_requirement.extend(reverse_req); + } else { + let req = PhysicalSortRequirement::from_sort_exprs(&req); + new_requirement.extend(req); + } + partition_search_mode = mode; } else { let req = PhysicalSortRequirement::from_sort_exprs(&req); new_requirement.extend(req); } - partition_search_mode = mode; } else { let req = PhysicalSortRequirement::from_sort_exprs(&req); new_requirement.extend(req); } - new_requirement = collapse_lex_req(new_requirement); - // let aggregator_requirement = requirement - // .as_ref() - // .map(|exprs| PhysicalSortRequirement::from_sort_exprs(exprs.iter())); - // let aggregator_reqs = aggregator_requirement.unwrap_or(vec![]); - // // If all aggregate expressions are reversible, also consider reverse - // // requirement(s). The reason is that existing ordering may satisfy the - // // given requirement or its reverse. By considering both, we can generate better plans. - // if aggr_expr - // .iter() - // .all(|expr| !is_order_sensitive(expr) || expr.reverse_expr().is_some()) - // { - // aggregator_reverse_reqs = requirement.map(|reqs| { - // PhysicalSortRequirement::from_sort_exprs(reverse_order_bys(&reqs).iter()) - // }); - // } + new_requirement = collapse_lex_req(new_requirement); // construct a map from the input columns to the output columns of the Aggregation let mut columns_map: HashMap> = HashMap::new(); let mut source_to_target_mapping = vec![]; - // println!(" group_by.expr: {:?}", group_by.expr); - // println!("input schema: {:?}", input.schema()); - // println!("input_schema: {:?}", _input_schema); let schema_of_input = input.schema(); for (expr_idx, (expression, name)) in group_by.expr.iter().enumerate() { if let Some(column) = expression.as_any().downcast_ref::() { @@ -734,24 +617,14 @@ impl AggregateExec { source_to_target_mapping.push((source_expr, target_expr)); } - // println!("source_to_target_mapping: {:?}", source_to_target_mapping); let mut aggregation_ordering = calc_aggregation_ordering(&input, &group_by); let required_input_ordering = if new_requirement.is_empty() { None } else { Some(new_requirement) }; - // // println!("start aggregation_ordering: {:?}", aggregation_ordering); - // let required_input_ordering = calc_required_input_ordering( - // &input, - // &mut aggr_expr, - // &mut order_by_expr, - // aggregator_reqs, - // aggregator_reverse_reqs, - // &mut aggregation_ordering, - // &mode, - // )?; - // // println!("end aggregation_ordering: {:?}", aggregation_ordering); + let aggregate_oeq = input_oeq.project(&source_to_target_mapping, schema.clone()); + let output_ordering = aggregate_oeq.oeq_group().output_ordering(); Ok(AggregateExec { mode, @@ -769,6 +642,7 @@ impl AggregateExec { required_input_ordering, limit: None, partition_search_mode, + output_ordering, }) } @@ -927,8 +801,8 @@ impl DisplayAs for AggregateExec { write!(f, ", lim=[{limit}]")?; } - if let Some(aggregation_ordering) = &self.aggregation_ordering { - write!(f, ", ordering_mode={:?}", aggregation_ordering.mode)?; + if self.partition_search_mode != PartitionSearchMode::Linear { + write!(f, ", ordering_mode={:?}", self.partition_search_mode)?; } } } @@ -978,7 +852,7 @@ impl ExecutionPlan for AggregateExec { /// infinite, returns an error to indicate this. fn unbounded_output(&self, children: &[bool]) -> Result { if children[0] { - if self.aggregation_ordering.is_none() { + if self.partition_search_mode == PartitionSearchMode::Linear { // Cannot run without breaking pipeline. plan_err!( "Aggregate Error: `GROUP BY` clauses with columns without ordering and GROUPING SETS are not supported for unbounded inputs." @@ -992,9 +866,7 @@ impl ExecutionPlan for AggregateExec { } fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - self.aggregation_ordering - .as_ref() - .map(|item: &AggregationOrdering| item.ordering.as_slice()) + self.output_ordering.as_deref() } fn required_input_distribution(&self) -> Vec { diff --git a/datafusion/sqllogictest/test_files/groupby.slt b/datafusion/sqllogictest/test_files/groupby.slt index da695273c7e3..34108d5ce8cf 100644 --- a/datafusion/sqllogictest/test_files/groupby.slt +++ b/datafusion/sqllogictest/test_files/groupby.slt @@ -2019,7 +2019,7 @@ SortPreservingMergeExec: [col0@0 ASC NULLS LAST] ------AggregateExec: mode=FinalPartitioned, gby=[col0@0 as col0, col1@1 as col1, col2@2 as col2], aggr=[LAST_VALUE(r.col1)] --------CoalesceBatchesExec: target_batch_size=8192 ----------RepartitionExec: partitioning=Hash([col0@0, col1@1, col2@2], 4), input_partitions=4 -------------AggregateExec: mode=Partial, gby=[col0@0 as col0, col1@1 as col1, col2@2 as col2], aggr=[LAST_VALUE(r.col1)], ordering_mode=PartiallyOrdered +------------AggregateExec: mode=Partial, gby=[col0@0 as col0, col1@1 as col1, col2@2 as col2], aggr=[LAST_VALUE(r.col1)], ordering_mode=PartiallySorted([0]) --------------SortExec: expr=[col0@3 ASC NULLS LAST] ----------------CoalesceBatchesExec: target_batch_size=8192 ------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(col0@0, col0@0)] @@ -2120,7 +2120,7 @@ Projection: annotated_data_infinite2.a, annotated_data_infinite2.b, SUM(annotate ----TableScan: annotated_data_infinite2 projection=[a, b, c] physical_plan ProjectionExec: expr=[a@1 as a, b@0 as b, SUM(annotated_data_infinite2.c)@2 as summation1] ---AggregateExec: mode=Single, gby=[b@1 as b, a@0 as a], aggr=[SUM(annotated_data_infinite2.c)], ordering_mode=FullyOrdered +--AggregateExec: mode=Single, gby=[b@1 as b, a@0 as a], aggr=[SUM(annotated_data_infinite2.c)], ordering_mode=Sorted ----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true @@ -2151,7 +2151,7 @@ Projection: annotated_data_infinite2.a, annotated_data_infinite2.d, SUM(annotate ----TableScan: annotated_data_infinite2 projection=[a, c, d] physical_plan ProjectionExec: expr=[a@1 as a, d@0 as d, SUM(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]@2 as summation1] ---AggregateExec: mode=Single, gby=[d@2 as d, a@0 as a], aggr=[SUM(annotated_data_infinite2.c)], ordering_mode=PartiallyOrdered +--AggregateExec: mode=Single, gby=[d@2 as d, a@0 as a], aggr=[SUM(annotated_data_infinite2.c)], ordering_mode=PartiallySorted([1]) ----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true query III @@ -2184,7 +2184,7 @@ Projection: annotated_data_infinite2.a, annotated_data_infinite2.b, FIRST_VALUE( ----TableScan: annotated_data_infinite2 projection=[a, b, c] physical_plan ProjectionExec: expr=[a@0 as a, b@1 as b, FIRST_VALUE(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]@2 as first_c] ---AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[FIRST_VALUE(annotated_data_infinite2.c)], ordering_mode=FullyOrdered +--AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[FIRST_VALUE(annotated_data_infinite2.c)], ordering_mode=Sorted ----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true query III @@ -2210,7 +2210,7 @@ Projection: annotated_data_infinite2.a, annotated_data_infinite2.b, LAST_VALUE(a ----TableScan: annotated_data_infinite2 projection=[a, b, c] physical_plan ProjectionExec: expr=[a@0 as a, b@1 as b, LAST_VALUE(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]@2 as last_c] ---AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[LAST_VALUE(annotated_data_infinite2.c)], ordering_mode=FullyOrdered +--AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[LAST_VALUE(annotated_data_infinite2.c)], ordering_mode=Sorted ----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true query III @@ -2237,7 +2237,7 @@ Projection: annotated_data_infinite2.a, annotated_data_infinite2.b, LAST_VALUE(a ----TableScan: annotated_data_infinite2 projection=[a, b, c] physical_plan ProjectionExec: expr=[a@0 as a, b@1 as b, LAST_VALUE(annotated_data_infinite2.c)@2 as last_c] ---AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[LAST_VALUE(annotated_data_infinite2.c)], ordering_mode=FullyOrdered +--AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b], aggr=[LAST_VALUE(annotated_data_infinite2.c)], ordering_mode=Sorted ----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true query III @@ -2356,9 +2356,9 @@ SELECT ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, GROUP BY s.country # test_ordering_sensitive_aggregation4 -# If aggregators can work with bounded memory (FullyOrdered or PartiallyOrdered mode), we should append requirement to +# If aggregators can work with bounded memory (Sorted or PartiallySorted mode), we should append requirement to # the existing ordering. This enables us to still work with bounded memory, and also satisfy aggregation requirement. -# This test checks for whether we can satisfy aggregation requirement in FullyOrdered mode. +# This test checks for whether we can satisfy aggregation requirement in Sorted mode. query TT EXPLAIN SELECT s.country, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, SUM(s.amount) AS sum1 @@ -2375,7 +2375,7 @@ Projection: s.country, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST] --------TableScan: sales_global projection=[country, amount] physical_plan ProjectionExec: expr=[country@0 as country, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST]@1 as amounts, SUM(s.amount)@2 as sum1] ---AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(s.amount), SUM(s.amount)], ordering_mode=FullyOrdered +--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(s.amount), SUM(s.amount)], ordering_mode=Sorted ----SortExec: expr=[country@0 ASC NULLS LAST,amount@1 DESC] ------MemoryExec: partitions=1, partition_sizes=[1] @@ -2392,9 +2392,9 @@ GRC [80.0, 30.0] 110 TUR [100.0, 75.0] 175 # test_ordering_sensitive_aggregation5 -# If aggregators can work with bounded memory (FullyOrdered or PartiallyOrdered mode), we should be append requirement to +# If aggregators can work with bounded memory (Sorted or PartiallySorted mode), we should be append requirement to # the existing ordering. This enables us to still work with bounded memory, and also satisfy aggregation requirement. -# This test checks for whether we can satisfy aggregation requirement in PartiallyOrdered mode. +# This test checks for whether we can satisfy aggregation requirement in PartiallySorted mode. query TT EXPLAIN SELECT s.country, s.zip_code, ARRAY_AGG(s.amount ORDER BY s.amount DESC) AS amounts, SUM(s.amount) AS sum1 @@ -2411,7 +2411,7 @@ Projection: s.country, s.zip_code, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC N --------TableScan: sales_global projection=[zip_code, country, amount] physical_plan ProjectionExec: expr=[country@0 as country, zip_code@1 as zip_code, ARRAY_AGG(s.amount) ORDER BY [s.amount DESC NULLS FIRST]@2 as amounts, SUM(s.amount)@3 as sum1] ---AggregateExec: mode=Single, gby=[country@1 as country, zip_code@0 as zip_code], aggr=[ARRAY_AGG(s.amount), SUM(s.amount)], ordering_mode=PartiallyOrdered +--AggregateExec: mode=Single, gby=[country@1 as country, zip_code@0 as zip_code], aggr=[ARRAY_AGG(s.amount), SUM(s.amount)], ordering_mode=PartiallySorted([0]) ----SortExec: expr=[country@1 ASC NULLS LAST,amount@2 DESC] ------MemoryExec: partitions=1, partition_sizes=[1] @@ -2428,7 +2428,7 @@ GRC 0 [80.0, 30.0] 110 TUR 1 [100.0, 75.0] 175 # test_ordering_sensitive_aggregation6 -# If aggregators can work with bounded memory (FullyOrdered or PartiallyOrdered mode), we should be append requirement to +# If aggregators can work with bounded memory (FullySorted or PartiallySorted mode), we should be append requirement to # the existing ordering. When group by expressions contain aggregation requirement, we shouldn't append redundant expression. # Hence in the final plan SortExec should be SortExec: expr=[country@0 DESC] not SortExec: expr=[country@0 ASC NULLS LAST,country@0 DESC] query TT @@ -2447,7 +2447,7 @@ Projection: s.country, ARRAY_AGG(s.amount) ORDER BY [s.country DESC NULLS FIRST] --------TableScan: sales_global projection=[country, amount] physical_plan ProjectionExec: expr=[country@0 as country, ARRAY_AGG(s.amount) ORDER BY [s.country DESC NULLS FIRST]@1 as amounts, SUM(s.amount)@2 as sum1] ---AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(s.amount), SUM(s.amount)], ordering_mode=FullyOrdered +--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(s.amount), SUM(s.amount)], ordering_mode=Sorted ----SortExec: expr=[country@0 ASC NULLS LAST] ------MemoryExec: partitions=1, partition_sizes=[1] @@ -2482,7 +2482,7 @@ Projection: s.country, ARRAY_AGG(s.amount) ORDER BY [s.country DESC NULLS FIRST, --------TableScan: sales_global projection=[country, amount] physical_plan ProjectionExec: expr=[country@0 as country, ARRAY_AGG(s.amount) ORDER BY [s.country DESC NULLS FIRST, s.amount DESC NULLS FIRST]@1 as amounts, SUM(s.amount)@2 as sum1] ---AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(s.amount), SUM(s.amount)], ordering_mode=FullyOrdered +--AggregateExec: mode=Single, gby=[country@0 as country], aggr=[ARRAY_AGG(s.amount), SUM(s.amount)], ordering_mode=Sorted ----SortExec: expr=[country@0 ASC NULLS LAST,amount@1 DESC] ------MemoryExec: partitions=1, partition_sizes=[1] @@ -3594,7 +3594,7 @@ Projection: amount_usd physical_plan ProjectionExec: expr=[amount_usd@0 as amount_usd] --ProjectionExec: expr=[LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST]@1 as amount_usd, row_n@0 as row_n] -----AggregateExec: mode=Single, gby=[row_n@2 as row_n], aggr=[LAST_VALUE(l.d)], ordering_mode=FullyOrdered +----AggregateExec: mode=Single, gby=[row_n@2 as row_n], aggr=[LAST_VALUE(l.d)], ordering_mode=Sorted ------ProjectionExec: expr=[a@0 as a, d@1 as d, row_n@4 as row_n] --------CoalesceBatchesExec: target_batch_size=8192 ----------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d@1, d@1)], filter=CAST(a@0 AS Int64) >= CAST(a@1 AS Int64) - 10 diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 3cb679c53359..4e8bae754764 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -357,7 +357,7 @@ Sort: d.b ASC NULLS LAST physical_plan SortPreservingMergeExec: [b@0 ASC NULLS LAST] --ProjectionExec: expr=[b@0 as b, MAX(d.a)@1 as max_a, MAX(d.seq)@2 as MAX(d.seq)] -----AggregateExec: mode=SinglePartitioned, gby=[b@2 as b], aggr=[MAX(d.a), MAX(d.seq)], ordering_mode=FullyOrdered +----AggregateExec: mode=SinglePartitioned, gby=[b@2 as b], aggr=[MAX(d.a), MAX(d.seq)], ordering_mode=Sorted ------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() PARTITION BY [s.b] ORDER BY [s.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() PARTITION BY [s.b] ORDER BY [s.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", 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 }], mode=[Sorted] ----------SortExec: expr=[b@1 ASC NULLS LAST,a@0 ASC NULLS LAST] From d77a6e08eadfb2d2e0fda521b8fa3d86e9733fe3 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 3 Oct 2023 10:59:51 +0300 Subject: [PATCH 016/122] Minor changes --- datafusion/physical-expr/src/equivalence.rs | 6 +++--- datafusion/physical-plan/src/aggregates/mod.rs | 13 +++++++------ 2 files changed, 10 insertions(+), 9 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 19ed32204c7b..4632293ae474 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -662,9 +662,9 @@ impl OrderingEquivalenceProperties { if children.is_empty() { for (source, target) in source_to_target_mapping.iter() { if source.eq(expr) - // || old_eq_class - // .iter() - // .any(|eq_class| eq_class.iter().any(|item| item.eq(expr))) + // || old_eq_class + // .iter() + // .any(|eq_class| eq_class.iter().any(|item| item.eq(expr))) { return Some(target.clone()); } else if let Some(group) = Self::get_eq_class_group(old_eq_class, source) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index c46aa051197a..3d190f7cbb94 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -54,13 +54,13 @@ mod topk; mod topk_stream; use crate::aggregates::topk_stream::GroupedTopKAggregateStream; +use crate::joins::StreamJoinPartitionMode::Partitioned; use crate::windows::get_ordered_partition_by_indices; pub use datafusion_expr::AggregateFunction; use datafusion_physical_expr::aggregate::is_order_sensitive; use datafusion_physical_expr::equivalence::{collapse_lex_req, PartitionSearchMode}; pub use datafusion_physical_expr::expressions::create_aggregate_expr; use datafusion_physical_expr::expressions::{Max, Min}; -use crate::joins::StreamJoinPartitionMode::Partitioned; use super::DisplayAs; @@ -539,7 +539,7 @@ impl AggregateExec { let mut partition_search_mode = PartitionSearchMode::Linear; let input_oeq = input.ordering_equivalence_properties(); let mut new_requirement: Vec = vec![]; - if group_by.is_single() && !groupby_exprs.is_empty(){ + if group_by.is_single() && !groupby_exprs.is_empty() { println!("groupby_exprs: {:?}, req:{:?}", groupby_exprs, req); println!("input_oeq: {:?}", input_oeq); let indices = get_ordered_partition_by_indices(&groupby_exprs, &input); @@ -554,9 +554,9 @@ impl AggregateExec { if let Some((should_reverse, mode)) = input_oeq.get_window_mode(&groupby_exprs, &req)? { - let all_reversible = aggr_expr - .iter() - .all(|expr| !is_order_sensitive(expr) || expr.reverse_expr().is_some()); + let all_reversible = aggr_expr.iter().all(|expr| { + !is_order_sensitive(expr) || expr.reverse_expr().is_some() + }); if should_reverse && all_reversible { izip!(aggr_expr.iter_mut(), order_by_expr.iter_mut()).for_each( |(aggr, order_by)| { @@ -623,7 +623,8 @@ impl AggregateExec { } else { Some(new_requirement) }; - let aggregate_oeq = input_oeq.project(&source_to_target_mapping, schema.clone()); + + let aggregate_oeq = input_oeq.project(&source_to_target_mapping, schema.clone()); let output_ordering = aggregate_oeq.oeq_group().output_ordering(); Ok(AggregateExec { From fb08ccec57115f2eead11c122ffac0f5e0c9202b Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 3 Oct 2023 11:01:24 +0300 Subject: [PATCH 017/122] Minor changes --- datafusion/physical-plan/src/aggregates/mod.rs | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 3d190f7cbb94..ab25957e50cc 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -54,7 +54,6 @@ mod topk; mod topk_stream; use crate::aggregates::topk_stream::GroupedTopKAggregateStream; -use crate::joins::StreamJoinPartitionMode::Partitioned; use crate::windows::get_ordered_partition_by_indices; pub use datafusion_expr::AggregateFunction; use datafusion_physical_expr::aggregate::is_order_sensitive; @@ -617,7 +616,7 @@ impl AggregateExec { source_to_target_mapping.push((source_expr, target_expr)); } - let mut aggregation_ordering = calc_aggregation_ordering(&input, &group_by); + let aggregation_ordering = calc_aggregation_ordering(&input, &group_by); let required_input_ordering = if new_requirement.is_empty() { None } else { From 8ce33bed7e84b3b379343b9d97ec3c04e5f18486 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 3 Oct 2023 11:19:10 +0300 Subject: [PATCH 018/122] All tests pass --- .../physical-plan/src/aggregates/mod.rs | 174 +----------------- .../physical-plan/src/aggregates/order/mod.rs | 21 +-- .../physical-plan/src/aggregates/row_hash.rs | 15 +- 3 files changed, 15 insertions(+), 195 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index ab25957e50cc..1cedf65e6f45 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -39,7 +39,6 @@ use datafusion_physical_expr::{ PhysicalSortRequirement, }; -use arrow_schema::SortOptions; use datafusion_common::tree_node::{Transformed, TreeNode}; use itertools::{izip, Itertools}; use std::any::Any; @@ -250,18 +249,6 @@ impl From for SendableRecordBatchStream { } } -/// This object encapsulates ordering-related information on GROUP BY columns. -#[derive(Debug, Clone)] -pub(crate) struct AggregationOrdering { - /// Specifies whether the GROUP BY columns are partially or fully ordered. - mode: GroupByOrderMode, - /// Stores indices such that when we iterate with these indices, GROUP BY - /// expressions match input ordering. - order_indices: Vec, - /// Actual ordering information of the GROUP BY columns. - ordering: LexOrdering, -} - /// Hash aggregate execution plan #[derive(Debug)] pub struct AggregateExec { @@ -291,74 +278,11 @@ pub struct AggregateExec { source_to_target_mapping: Vec<(Arc, Arc)>, /// Execution Metrics metrics: ExecutionPlanMetricsSet, - /// Stores mode and output ordering information for the `AggregateExec`. - aggregation_ordering: Option, required_input_ordering: Option, partition_search_mode: PartitionSearchMode, output_ordering: Option, } -/// Calculates the working mode for `GROUP BY` queries. -/// - If no GROUP BY expression has an ordering, returns `None`. -/// - If some GROUP BY expressions have an ordering, returns `Some(GroupByOrderMode::PartiallyOrdered)`. -/// - If all GROUP BY expressions have orderings, returns `Some(GroupByOrderMode::Ordered)`. -fn get_working_mode( - input: &Arc, - group_by: &PhysicalGroupBy, -) -> Option<(GroupByOrderMode, Vec<(usize, SortOptions)>)> { - if !group_by.is_single() { - // We do not currently support streaming execution if we have more - // than one group (e.g. we have grouping sets). - return None; - }; - - // let output_ordering = input.output_ordering().unwrap_or(&[]); - // Since direction of the ordering is not important for GROUP BY columns, - // we convert PhysicalSortExpr to PhysicalExpr in the existing ordering. - // let ordering_exprs = convert_to_expr(output_ordering); - let groupby_exprs = group_by - .expr - .iter() - .map(|(item, _)| item.clone()) - .collect::>(); - - input - .ordering_equivalence_properties() - .set_satisfy(&groupby_exprs) - .map(|ordered_group_by_info| { - if ordered_group_by_info.len() == group_by.expr.len() { - (GroupByOrderMode::FullyOrdered, ordered_group_by_info) - } else { - (GroupByOrderMode::PartiallyOrdered, ordered_group_by_info) - } - }) -} - -/// This function gathers the ordering information for the GROUP BY columns. -fn calc_aggregation_ordering( - input: &Arc, - group_by: &PhysicalGroupBy, -) -> Option { - get_working_mode(input, group_by).map(|(mode, order_info)| { - // let existing_ordering = input.output_ordering().unwrap_or(&[]); - let out_group_expr = output_group_expr_helper(group_by); - // Calculate output ordering information for the operator: - let out_ordering = order_info - .iter() - .map(|(idx, sort_options)| PhysicalSortExpr { - expr: out_group_expr[*idx].clone(), - options: *sort_options, - }) - .collect::>(); - let order_indices = order_info.iter().map(|(idx, _)| *idx).collect(); - AggregationOrdering { - mode, - order_indices, - ordering: out_ordering, - } - }) -} - /// This function returns grouping expressions as they occur in the output schema. fn output_group_expr_helper(group_by: &PhysicalGroupBy) -> Vec> { // Update column indices. Since the group by columns come first in the output schema, their @@ -616,7 +540,6 @@ impl AggregateExec { source_to_target_mapping.push((source_expr, target_expr)); } - let aggregation_ordering = calc_aggregation_ordering(&input, &group_by); let required_input_ordering = if new_requirement.is_empty() { None } else { @@ -638,7 +561,6 @@ impl AggregateExec { columns_map, source_to_target_mapping, metrics: ExecutionPlanMetricsSet::new(), - aggregation_ordering, required_input_ordering, limit: None, partition_search_mode, @@ -1205,18 +1127,16 @@ pub(crate) fn evaluate_group_by( #[cfg(test)] mod tests { use super::*; - use crate::aggregates::GroupByOrderMode::{FullyOrdered, PartiallyOrdered}; use crate::aggregates::{ - get_finest_requirement, get_working_mode, AggregateExec, AggregateMode, - PhysicalGroupBy, + get_finest_requirement, AggregateExec, AggregateMode, PhysicalGroupBy, }; use crate::coalesce_batches::CoalesceBatchesExec; use crate::coalesce_partitions::CoalescePartitionsExec; use crate::common; use crate::expressions::{col, Avg}; use crate::memory::MemoryExec; + use crate::test::assert_is_pending; use crate::test::exec::{assert_strong_count_converges_to_zero, BlockingExec}; - use crate::test::{assert_is_pending, mem_exec}; use crate::{ DisplayAs, ExecutionPlan, Partitioning, RecordBatchStream, SendableRecordBatchStream, Statistics, @@ -1257,96 +1177,6 @@ mod tests { Ok(schema) } - /// 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, - } - } - - #[tokio::test] - async fn test_get_working_mode() -> Result<()> { - let test_schema = create_test_schema()?; - // Source is sorted by a ASC NULLS FIRST, b ASC NULLS FIRST, c ASC NULLS FIRST - // Column d, e is not ordered. - let sort_exprs = vec![ - sort_expr("a", &test_schema), - sort_expr("b", &test_schema), - sort_expr("c", &test_schema), - ]; - let input = mem_exec(1).with_sort_information(vec![sort_exprs]); - let input = Arc::new(input) as _; - - // test cases consists of vector of tuples. Where each tuple represents a single test case. - // First field in the tuple is Vec where each element in the vector represents GROUP BY columns - // For instance `vec!["a", "b"]` corresponds to GROUP BY a, b - // Second field in the tuple is Option, which corresponds to expected algorithm mode. - // None represents that existing ordering is not sufficient to run executor with any one of the algorithms - // (We need to add SortExec to be able to run it). - // Some(GroupByOrderMode) represents, we can run algorithm with existing ordering; and algorithm should work in - // GroupByOrderMode. - let options = SortOptions::default(); - let test_cases = vec![ - (vec!["a"], Some((FullyOrdered, vec![(0, options)]))), - (vec!["b"], None), - (vec!["c"], None), - ( - vec!["b", "a"], - Some((FullyOrdered, vec![(1, options), (0, options)])), - ), - (vec!["c", "b"], None), - (vec!["c", "a"], Some((PartiallyOrdered, vec![(1, options)]))), - ( - vec!["c", "b", "a"], - Some((FullyOrdered, vec![(2, options), (1, options), (0, options)])), - ), - (vec!["d", "a"], Some((PartiallyOrdered, vec![(1, options)]))), - (vec!["d", "b"], None), - (vec!["d", "c"], None), - ( - vec!["d", "b", "a"], - Some((PartiallyOrdered, vec![(2, options), (1, options)])), - ), - (vec!["d", "c", "b"], None), - ( - vec!["d", "c", "a"], - Some((PartiallyOrdered, vec![(2, options)])), - ), - ( - vec!["d", "c", "b", "a"], - Some(( - PartiallyOrdered, - vec![(3, options), (2, options), (1, options)], - )), - ), - ]; - for (case_idx, test_case) in test_cases.iter().enumerate() { - let (group_by_columns, expected) = &test_case; - let mut group_by_exprs = vec![]; - for col_name in group_by_columns { - group_by_exprs.push((col(col_name, &test_schema)?, col_name.to_string())); - } - let group_bys = PhysicalGroupBy::new_single(group_by_exprs); - let res = get_working_mode(&input, &group_bys); - assert_eq!( - res, *expected, - "Unexpected result for in unbounded test case#: {case_idx:?}, case: {test_case:?}" - ); - } - - Ok(()) - } - /// some mock data to aggregates fn some_data() -> (Arc, Vec) { // define a schema. diff --git a/datafusion/physical-plan/src/aggregates/order/mod.rs b/datafusion/physical-plan/src/aggregates/order/mod.rs index f0b49872b1c5..7fc5f5dbb9bb 100644 --- a/datafusion/physical-plan/src/aggregates/order/mod.rs +++ b/datafusion/physical-plan/src/aggregates/order/mod.rs @@ -18,9 +18,8 @@ use arrow_array::ArrayRef; use arrow_schema::Schema; use datafusion_common::Result; -use datafusion_physical_expr::EmitTo; - -use super::{AggregationOrdering, GroupByOrderMode}; +use datafusion_physical_expr::equivalence::PartitionSearchMode; +use datafusion_physical_expr::{EmitTo, PhysicalSortExpr}; mod full; mod partial; @@ -43,23 +42,17 @@ impl GroupOrdering { /// Create a `GroupOrdering` for the the specified ordering pub fn try_new( input_schema: &Schema, - ordering: &AggregationOrdering, + mode: &PartitionSearchMode, + ordering: &[PhysicalSortExpr], ) -> Result { - let AggregationOrdering { - mode, - order_indices, - ordering, - } = ordering; - Ok(match mode { - GroupByOrderMode::PartiallyOrdered => { + PartitionSearchMode::Linear => GroupOrdering::None, + PartitionSearchMode::PartiallySorted(order_indices) => { let partial = GroupOrderingPartial::try_new(input_schema, order_indices, ordering)?; GroupOrdering::Partial(partial) } - GroupByOrderMode::FullyOrdered => { - GroupOrdering::Full(GroupOrderingFull::new()) - } + PartitionSearchMode::Sorted => GroupOrdering::Full(GroupOrderingFull::new()), }) } diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index d773533ad6a3..9b5f83e3bbde 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -330,15 +330,12 @@ impl GroupedHashAggregateStream { let name = format!("GroupedHashAggregateStream[{partition}]"); let reservation = MemoryConsumer::new(name).register(context.memory_pool()); - let group_ordering = agg - .aggregation_ordering - .as_ref() - .map(|aggregation_ordering| { - GroupOrdering::try_new(&group_schema, aggregation_ordering) - }) - // return error if any - .transpose()? - .unwrap_or(GroupOrdering::None); + let out_ordering = agg.output_ordering.as_deref().unwrap_or(&[]); + let group_ordering = GroupOrdering::try_new( + &group_schema, + &agg.partition_search_mode, + out_ordering, + )?; let group_values = new_group_values(group_schema)?; timer.done(); From e1b14885918be93eda2b4cddc3f8481934c5a267 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 3 Oct 2023 13:47:45 +0300 Subject: [PATCH 019/122] Simplifications --- .../physical-plan/src/aggregates/mod.rs | 129 ++++++++++-------- 1 file changed, 69 insertions(+), 60 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 1cedf65e6f45..f9b6868e2cce 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -369,6 +369,49 @@ fn get_finest_requirement OrderingEquivalenceProperties>( Ok(finest_req) } +/// Calculates search_mode for the aggregation +fn get_aggregate_search_mode( + group_by: &PhysicalGroupBy, + input: &Arc, + aggr_expr: &mut [Arc], + order_by_expr: &mut [Option], + ordering_req: &mut Vec, +) -> Result { + let groupby_exprs = group_by + .expr + .iter() + .map(|(item, _)| item.clone()) + .collect::>(); + let mut partition_search_mode = PartitionSearchMode::Linear; + let input_oeq = input.ordering_equivalence_properties(); + if !group_by.is_single() || groupby_exprs.is_empty() { + return Ok(partition_search_mode); + } + + if let Some((should_reverse, mode)) = + input_oeq.get_window_mode(&groupby_exprs, ordering_req)? + { + let all_reversible = aggr_expr + .iter() + .all(|expr| !is_order_sensitive(expr) || expr.reverse_expr().is_some()); + if should_reverse && all_reversible { + izip!(aggr_expr.iter_mut(), order_by_expr.iter_mut()).for_each( + |(aggr, order_by)| { + if let Some(reverse) = aggr.reverse_expr() { + *aggr = reverse; + } else { + unreachable!(); + } + *order_by = order_by.as_ref().map(|ob| reverse_order_bys(ob)); + }, + ); + *ordering_req = reverse_order_bys(ordering_req); + } + partition_search_mode = mode; + } + Ok(partition_search_mode) +} + #[allow(dead_code)] fn print_plan(plan: &Arc) { let formatted = displayable(plan.as_ref()).indent(true).to_string(); @@ -441,74 +484,38 @@ impl AggregateExec { }) }) .collect::>(); - // let mut aggregator_reverse_reqs = None; - // Currently we support order-sensitive aggregation only in `Single` mode. - // For `Final` and `FinalPartitioned` modes, we cannot guarantee they will receive - // data according to ordering requirements. As long as we cannot produce correct result - // in `Final` mode, it is not important to produce correct result in `Partial` mode. - // We only support `Single` mode, where we are sure that output produced is final, and it - // is produced in a single step. - let requirement = get_finest_requirement(&mut aggr_expr, &mut order_by_expr, || { input.ordering_equivalence_properties() })?; - let req = requirement.unwrap_or(vec![]); + let mut ordering_req = requirement.unwrap_or(vec![]); + let partition_search_mode = get_aggregate_search_mode( + &group_by, + &input, + &mut aggr_expr, + &mut order_by_expr, + &mut ordering_req, + )?; + + // get group by exprs let groupby_exprs = group_by .expr .iter() .map(|(item, _)| item.clone()) .collect::>(); - let mut partition_search_mode = PartitionSearchMode::Linear; - let input_oeq = input.ordering_equivalence_properties(); - let mut new_requirement: Vec = vec![]; - if group_by.is_single() && !groupby_exprs.is_empty() { - println!("groupby_exprs: {:?}, req:{:?}", groupby_exprs, req); - println!("input_oeq: {:?}", input_oeq); - let indices = get_ordered_partition_by_indices(&groupby_exprs, &input); - let elem = indices - .into_iter() - .map(|idx| PhysicalSortRequirement { - expr: groupby_exprs[idx].clone(), - options: None, - }) - .collect::>(); - new_requirement.extend(elem); - if let Some((should_reverse, mode)) = - input_oeq.get_window_mode(&groupby_exprs, &req)? - { - let all_reversible = aggr_expr.iter().all(|expr| { - !is_order_sensitive(expr) || expr.reverse_expr().is_some() - }); - if should_reverse && all_reversible { - izip!(aggr_expr.iter_mut(), order_by_expr.iter_mut()).for_each( - |(aggr, order_by)| { - if let Some(reverse) = aggr.reverse_expr() { - *aggr = reverse; - } else { - unreachable!(); - } - *order_by = order_by.as_ref().map(|ob| reverse_order_bys(ob)); - }, - ); - let reverse_req = PhysicalSortRequirement::from_sort_exprs( - reverse_order_bys(&req).iter(), - ); - new_requirement.extend(reverse_req); - } else { - let req = PhysicalSortRequirement::from_sort_exprs(&req); - new_requirement.extend(req); - } - partition_search_mode = mode; - } else { - let req = PhysicalSortRequirement::from_sort_exprs(&req); - new_requirement.extend(req); - } - } else { - let req = PhysicalSortRequirement::from_sort_exprs(&req); - new_requirement.extend(req); - } - + // If existing ordering satisfies a prefix of groupby expression, prefix requirement + // with this section. In this case, group by will work more efficient + let indices = get_ordered_partition_by_indices(&groupby_exprs, &input); + let mut new_requirement = indices + .into_iter() + .map(|idx| PhysicalSortRequirement { + expr: groupby_exprs[idx].clone(), + options: None, + }) + .collect::>(); + // Postfix ordering requirement of the aggregation to the requirement. + let req = PhysicalSortRequirement::from_sort_exprs(&ordering_req); + new_requirement.extend(req); new_requirement = collapse_lex_req(new_requirement); // construct a map from the input columns to the output columns of the Aggregation @@ -546,7 +553,9 @@ impl AggregateExec { Some(new_requirement) }; - let aggregate_oeq = input_oeq.project(&source_to_target_mapping, schema.clone()); + let aggregate_oeq = input + .ordering_equivalence_properties() + .project(&source_to_target_mapping, schema.clone()); let output_ordering = aggregate_oeq.oeq_group().output_ordering(); Ok(AggregateExec { From aa019b354ec6012bddf75ff7d8147e196723452e Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 3 Oct 2023 14:58:30 +0300 Subject: [PATCH 020/122] Simplifications --- .../enforce_distribution.rs | 17 +- .../src/physical_optimizer/enforce_sorting.rs | 1024 +---------------- .../src/physical_optimizer/sort_pushdown.rs | 12 +- .../core/src/physical_optimizer/utils.rs | 7 +- datafusion/physical-expr/src/equivalence.rs | 8 +- .../physical-plan/src/aggregates/mod.rs | 1 + datafusion/sqllogictest/test_files/joins.slt | 125 ++ datafusion/sqllogictest/test_files/window.slt | 2 + 8 files changed, 161 insertions(+), 1035 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index f160606c5943..0bc6b18860b5 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -53,7 +53,6 @@ use datafusion_physical_expr::expressions::{Column, NoOp}; use datafusion_physical_expr::utils::map_columns_before_projection; use datafusion_physical_expr::{ expr_list_eq_strict_order, OrderingEquivalenceProperties, PhysicalExpr, - PhysicalSortRequirement, }; use datafusion_physical_plan::unbounded_output; @@ -1209,6 +1208,15 @@ fn replace_order_preserving_variants_helper( exec_tree.plan.clone().with_new_children(updated_children) } +#[allow(dead_code)] +fn print_plan(plan: &Arc) { + let formatted = crate::physical_plan::displayable(plan.as_ref()) + .indent(true) + .to_string(); + let actual: Vec<&str> = formatted.trim().lines().collect(); + println!("{:#?}", actual); +} + /// This function checks whether we need to add additional data exchange /// operators to satisfy distribution requirements. Since this function /// takes care of such requirements, we should avoid manually adding data @@ -1358,10 +1366,7 @@ fn ensure_distribution( // make sure ordering requirements are still satisfied after. if ordering_satisfied { // Make sure to satisfy ordering requirement: - let sort_expr = PhysicalSortRequirement::to_sort_exprs( - required_input_ordering.clone(), - ); - add_sort_above(&mut child, sort_expr, None)?; + add_sort_above(&mut child, required_input_ordering, None)?; } } // Stop tracking distribution changing operators @@ -1699,7 +1704,7 @@ mod tests { use datafusion_physical_expr::expressions::{BinaryExpr, Literal}; use datafusion_physical_expr::{ expressions, expressions::binary, expressions::lit, expressions::Column, - LexOrdering, PhysicalExpr, PhysicalSortExpr, + LexOrdering, PhysicalExpr, PhysicalSortExpr, PhysicalSortRequirement, }; /// Models operators like BoundedWindowExec that require an input diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index fbd8e99e56fd..13712791484b 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -407,7 +407,11 @@ fn parallelize_sorts( let mut prev_layer = plan.clone(); update_child_to_remove_coalesce(&mut prev_layer, &mut coalesce_onwards[0])?; let (sort_exprs, fetch) = get_sort_exprs(&plan)?; - add_sort_above(&mut prev_layer, sort_exprs.to_vec(), fetch)?; + add_sort_above( + &mut prev_layer, + &PhysicalSortRequirement::from_sort_exprs(sort_exprs), + fetch, + )?; let spm = SortPreservingMergeExec::new(sort_exprs.to_vec(), prev_layer) .with_fetch(fetch); return Ok(Transformed::Yes(PlanWithCorrespondingCoalescePartitions { @@ -459,17 +463,15 @@ fn ensure_sorting( (Some(required_ordering), Some(_)) => { // println!("child"); // print_plan(&child); + // println!("required_ordering: {:?}", required_ordering); + // println!("child.ordering_equivalence_properties(): {:?}", child.ordering_equivalence_properties()); if !child .ordering_equivalence_properties() .ordering_satisfy_requirement_concrete(&required_ordering) { // Make sure we preserve the ordering requirements: update_child_to_remove_unnecessary_sort(child, sort_onwards, &plan)?; - // println!("child after update"); - // print_plan(&child); - let sort_expr = - PhysicalSortRequirement::to_sort_exprs(required_ordering); - add_sort_above(child, sort_expr, None)?; + add_sort_above(child, &required_ordering, None)?; if is_sort(child) { *sort_onwards = Some(ExecTree::new(child.clone(), idx, vec![])); } else { @@ -479,8 +481,7 @@ fn ensure_sorting( } (Some(required), None) => { // Ordering requirement is not met, we should add a `SortExec` to the plan. - let sort_expr = PhysicalSortRequirement::to_sort_exprs(required); - add_sort_above(child, sort_expr, None)?; + add_sort_above(child, &required, None)?; *sort_onwards = Some(ExecTree::new(child.clone(), idx, vec![])); } (None, Some(_)) => { @@ -628,9 +629,8 @@ fn analyze_window_sort_removal( .required_input_ordering() .swap_remove(0) .unwrap_or(vec![]); - let sort_expr = PhysicalSortRequirement::to_sort_exprs(reqs); // Satisfy the ordering requirement so that the window can run: - add_sort_above(&mut window_child, sort_expr, None)?; + add_sort_above(&mut window_child, &reqs, None)?; let uses_bounded_memory = window_expr.iter().all(|e| e.uses_bounded_memory()); let input_schema = window_child.schema(); @@ -2262,1007 +2262,3 @@ mod tests { Ok(()) } } - -#[cfg(test)] -mod tmp_tests { - use crate::physical_optimizer::utils::get_plan_string; - use crate::physical_plan::{collect, displayable, ExecutionPlan}; - use crate::prelude::SessionContext; - use arrow::util::pretty::print_batches; - use datafusion_common::Result; - use datafusion_execution::config::SessionConfig; - use std::sync::Arc; - - fn print_plan(plan: &Arc) -> Result<()> { - let formatted = displayable(plan.as_ref()).indent(true).to_string(); - let actual: Vec<&str> = formatted.trim().lines().collect(); - println!("{:#?}", actual); - Ok(()) - } - - #[tokio::test] - async fn test_query() -> Result<()> { - let config = SessionConfig::new().with_target_partitions(1); - let ctx = SessionContext::with_config(config); - - ctx.sql("CREATE TABLE tab0(col0 INTEGER, col1 INTEGER, col2 INTEGER)") - .await?; - - let sql = "SELECT l.col0, LAST_VALUE(r.col1 ORDER BY r.col0) as last_col1 - FROM tab0 as l - JOIN tab0 as r - ON l.col0 = r.col0 - GROUP BY l.col0, l.col1, l.col2 - ORDER BY l.col0;"; - - let msg = format!("Creating logical plan for '{sql}'"); - let dataframe = ctx.sql(sql).await.expect(&msg); - let physical_plan = dataframe.create_physical_plan().await?; - print_plan(&physical_plan)?; - let actual = collect(physical_plan.clone(), ctx.task_ctx()).await?; - print_batches(&actual)?; - - let expected_optimized_lines: Vec<&str> = vec![ - "ProjectionExec: expr=[col0@0 as col0, LAST_VALUE(r.col1) ORDER BY [r.col0 ASC NULLS LAST]@3 as last_col1]", - " AggregateExec: mode=Single, gby=[col0@0 as col0, col1@1 as col1, col2@2 as col2], aggr=[LAST_VALUE(r.col1)], ordering_mode=PartiallySorted([0])", - " SortExec: expr=[col0@3 ASC NULLS LAST]", - " CoalesceBatchesExec: target_batch_size=8192", - " HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(col0@0, col0@0)]", - " MemoryExec: partitions=1, partition_sizes=[0]", - " MemoryExec: partitions=1, partition_sizes=[0]", - ]; - - // Get string representation of the plan - let actual = get_plan_string(&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" - ); - - Ok(()) - } - - #[tokio::test] - async fn test_query2() -> Result<()> { - let config = SessionConfig::new().with_target_partitions(2); - let ctx = SessionContext::with_config(config); - - ctx.sql("CREATE TABLE tab0(col0 INTEGER, col1 INTEGER, col2 INTEGER)") - .await?; - - let sql = "SELECT l.col0, LAST_VALUE(r.col1 ORDER BY r.col0) as last_col1 - FROM tab0 as l - JOIN tab0 as r - ON l.col0 = r.col0 - GROUP BY l.col0, l.col1, l.col2 - ORDER BY l.col0;"; - - let msg = format!("Creating logical plan for '{sql}'"); - let dataframe = ctx.sql(sql).await.expect(&msg); - let physical_plan = dataframe.create_physical_plan().await?; - print_plan(&physical_plan)?; - let actual = collect(physical_plan.clone(), ctx.task_ctx()).await?; - print_batches(&actual)?; - - let expected_optimized_lines: Vec<&str> = vec![ - "SortPreservingMergeExec: [col0@0 ASC NULLS LAST]", - " SortExec: expr=[col0@0 ASC NULLS LAST]", - " ProjectionExec: expr=[col0@0 as col0, LAST_VALUE(r.col1) ORDER BY [r.col0 ASC NULLS LAST]@3 as last_col1]", - " AggregateExec: mode=FinalPartitioned, gby=[col0@0 as col0, col1@1 as col1, col2@2 as col2], aggr=[LAST_VALUE(r.col1)]", - " CoalesceBatchesExec: target_batch_size=8192", - " RepartitionExec: partitioning=Hash([col0@0, col1@1, col2@2], 2), input_partitions=2", - " AggregateExec: mode=Partial, gby=[col0@0 as col0, col1@1 as col1, col2@2 as col2], aggr=[LAST_VALUE(r.col1)], ordering_mode=PartiallySorted([0])", - " SortExec: expr=[col0@3 ASC NULLS LAST]", - " CoalesceBatchesExec: target_batch_size=8192", - " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(col0@0, col0@0)]", - " CoalesceBatchesExec: target_batch_size=8192", - " RepartitionExec: partitioning=Hash([col0@0], 2), input_partitions=2", - " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[0]", - " CoalesceBatchesExec: target_batch_size=8192", - " RepartitionExec: partitioning=Hash([col0@0], 2), input_partitions=2", - " RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1", - " MemoryExec: partitions=1, partition_sizes=[0]", - ]; - - // Get string representation of the plan - let actual = get_plan_string(&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" - ); - - Ok(()) - } - - #[tokio::test] - async fn test_query3() -> Result<()> { - let config = SessionConfig::new().with_target_partitions(1); - let ctx = SessionContext::with_config(config); - - ctx.sql( - "CREATE EXTERNAL TABLE multiple_ordered_table ( - a0 INTEGER, - a INTEGER, - b INTEGER, - c INTEGER, - d INTEGER - ) - STORED AS CSV - WITH HEADER ROW - WITH ORDER (a ASC) - WITH ORDER (b ASC) - WITH ORDER (c ASC) - LOCATION '../core/tests/data/window_2.csv'", - ) - .await?; - - let sql = "SELECT (b+a+c) AS result - FROM multiple_ordered_table - ORDER BY result;"; - - let msg = format!("Creating logical plan for '{sql}'"); - let dataframe = ctx.sql(sql).await.expect(&msg); - let physical_plan = dataframe.create_physical_plan().await?; - print_plan(&physical_plan)?; - let actual = collect(physical_plan.clone(), ctx.task_ctx()).await?; - print_batches(&actual)?; - - let expected_optimized_lines: Vec<&str> = vec![ - "ProjectionExec: expr=[b@1 + a@0 + c@2 as result]", - " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], output_ordering=[a@0 ASC NULLS LAST], has_header=true", - ]; - - // Get string representation of the plan - let actual = get_plan_string(&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" - ); - - Ok(()) - } - - #[tokio::test] - async fn test_query4() -> Result<()> { - let config = SessionConfig::new().with_target_partitions(1); - let ctx = SessionContext::with_config(config); - - ctx.sql( - "CREATE EXTERNAL TABLE aggregate_test_100 ( - c1 VARCHAR NOT NULL, - c2 TINYINT NOT NULL, - c3 SMALLINT NOT NULL, - c4 SMALLINT, - c5 INT, - c6 BIGINT NOT NULL, - c7 SMALLINT NOT NULL, - c8 INT NOT NULL, - c9 BIGINT UNSIGNED NOT NULL, - c10 VARCHAR NOT NULL, - c11 FLOAT NOT NULL, - c12 DOUBLE NOT NULL, - c13 VARCHAR NOT NULL - ) - STORED AS CSV - WITH HEADER ROW - LOCATION '../../testing/data/csv/aggregate_test_100.csv'", - ) - .await?; - - let sql = "SELECT c3, - SUM(c9) OVER(ORDER BY c3+c4 DESC, c9 DESC, c2 ASC) as sum1, - SUM(c9) OVER(ORDER BY c3+c4 ASC, c9 ASC ) as sum2 - FROM aggregate_test_100 - LIMIT 5"; - - let msg = format!("Creating logical plan for '{sql}'"); - let dataframe = ctx.sql(sql).await.expect(&msg); - let physical_plan = dataframe.create_physical_plan().await?; - print_plan(&physical_plan)?; - let actual = collect(physical_plan.clone(), ctx.task_ctx()).await?; - print_batches(&actual)?; - - let expected_optimized_lines: Vec<&str> = vec![ - "ProjectionExec: expr=[c3@0 as c3, 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@3 as sum1, 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@4 as sum2]", - " 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)) }]", - " ProjectionExec: expr=[c3@1 as c3, c4@2 as c4, c9@3 as c9, 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@4 as 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]", - " 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 }], mode=[Sorted]", - " SortExec: expr=[c3@1 + c4@2 DESC,c9@3 DESC,c2@0 ASC NULLS LAST]", - " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2, c3, c4, c9], has_header=true", - ]; - - // Get string representation of the plan - let actual = get_plan_string(&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" - ); - - Ok(()) - } - - #[tokio::test] - async fn test_query5() -> Result<()> { - let config = SessionConfig::new().with_target_partitions(1); - let ctx = SessionContext::with_config(config); - - ctx.sql( - "CREATE EXTERNAL TABLE annotated_data ( - a0 INTEGER, - a INTEGER, - b INTEGER, - c INTEGER, - d INTEGER - ) - STORED AS CSV - WITH HEADER ROW - WITH ORDER (a ASC NULLS FIRST, b ASC, c ASC) - LOCATION '../core/tests/data/window_2.csv'", - ) - .await?; - - let sql = "SELECT * - FROM annotated_data as l_table - JOIN (SELECT *, ROW_NUMBER() OVER() as rn1 - FROM annotated_data) as r_table - ON l_table.a = r_table.a - ORDER BY r_table.rn1"; - - let msg = format!("Creating logical plan for '{sql}'"); - let dataframe = ctx.sql(sql).await.expect(&msg); - let physical_plan = dataframe.create_physical_plan().await?; - print_plan(&physical_plan)?; - let _actual = collect(physical_plan.clone(), ctx.task_ctx()).await?; - // print_batches(&actual)?; - - let expected_optimized_lines: Vec<&str> = vec![ - "CoalesceBatchesExec: target_batch_size=8192", - " HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@1, a@1)]", - " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true", - " ProjectionExec: expr=[a0@0 as a0, a@1 as a, b@2 as b, c@3 as c, d@4 as d, ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING@5 as rn1]", - " BoundedWindowAggExec: wdw=[ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: \"ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING\", 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)) }], mode=[Sorted]", - " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true", - ]; - - // Get string representation of the plan - let actual = get_plan_string(&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" - ); - - Ok(()) - } - - #[tokio::test] - async fn test_query6() -> Result<()> { - let config = SessionConfig::new().with_target_partitions(1); - let ctx = SessionContext::with_config(config); - - ctx.sql( - "CREATE EXTERNAL TABLE annotated_data_finite2 ( - a0 INTEGER, - a INTEGER, - b INTEGER, - c INTEGER, - d INTEGER - ) - STORED AS CSV - WITH HEADER ROW - WITH ORDER (a ASC, b ASC, c ASC) - LOCATION '../core/tests/data/window_2.csv'", - ) - .await?; - - let sql = "SELECT a, b, c, - SUM(c) OVER(PARTITION BY a, d ORDER BY b, c ASC ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING) as sum1, - SUM(c) OVER(PARTITION BY a, d ORDER BY b, c ASC ROWS BETWEEN 1 FOLLOWING AND 5 FOLLOWING) as sum2, - SUM(c) OVER(PARTITION BY d ORDER BY a, b, c ASC ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING) as sum3, - SUM(c) OVER(PARTITION BY d ORDER BY a, b, c ASC ROWS BETWEEN 5 PRECEDING AND 1 PRECEDING) as sum4, - SUM(c) OVER(PARTITION BY a, b ORDER BY c ASC ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING) as sum5, - SUM(c) OVER(PARTITION BY a, b ORDER BY c ASC ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING) as sum6, - SUM(c) OVER(PARTITION BY b, a ORDER BY c ASC ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING) as sum7, - SUM(c) OVER(PARTITION BY b, a ORDER BY c ASC ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING) as sum8, - SUM(c) OVER(PARTITION BY a, b, d ORDER BY c ASC ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING) as sum9, - SUM(c) OVER(PARTITION BY a, b, d ORDER BY c ASC ROWS BETWEEN 5 PRECEDING AND CURRENT ROW) as sum10, - SUM(c) OVER(PARTITION BY b, a, d ORDER BY c ASC ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING) as sum11, - SUM(c) OVER(PARTITION BY b, a, d ORDER BY c ASC ROWS BETWEEN CURRENT ROW AND 1 FOLLOWING) as sum12 - FROM annotated_data_finite2 - ORDER BY c - LIMIT 5"; - - let msg = format!("Creating logical plan for '{sql}'"); - let dataframe = ctx.sql(sql).await.expect(&msg); - let physical_plan = dataframe.create_physical_plan().await?; - print_plan(&physical_plan)?; - let _actual = collect(physical_plan.clone(), ctx.task_ctx()).await?; - // print_batches(&actual)?; - - let expected_optimized_lines: Vec<&str> = vec![ - "GlobalLimitExec: skip=0, fetch=5", - " SortExec: fetch=5, expr=[c@2 ASC NULLS LAST]", - " ProjectionExec: expr=[a@1 as a, b@2 as b, c@3 as c, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@9 as sum1, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 1 FOLLOWING AND 5 FOLLOWING@10 as sum2, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@15 as sum3, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 1 PRECEDING@16 as sum4, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@5 as sum5, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING@6 as sum6, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@11 as sum7, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING@12 as sum8, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@7 as sum9, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND CURRENT ROW@8 as sum10, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING@13 as sum11, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN CURRENT ROW AND 1 FOLLOWING@14 as sum12]", - " BoundedWindowAggExec: wdw=[SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: \"SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(2)), end_bound: Following(UInt64(1)) }, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 1 PRECEDING: Ok(Field { name: \"SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.d] ORDER BY [annotated_data_finite2.a ASC NULLS LAST, annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 1 PRECEDING\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: Preceding(UInt64(1)) }], mode=[Sorted]", - " SortExec: expr=[d@4 ASC NULLS LAST,a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST]", - " BoundedWindowAggExec: wdw=[SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: \"SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(2)), end_bound: Following(UInt64(1)) }, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN CURRENT ROW AND 1 FOLLOWING: Ok(Field { name: \"SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN CURRENT ROW AND 1 FOLLOWING\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: CurrentRow, end_bound: Following(UInt64(1)) }], mode=[Sorted]", - " SortExec: expr=[b@2 ASC NULLS LAST,a@1 ASC NULLS LAST,d@4 ASC NULLS LAST,c@3 ASC NULLS LAST]", - " BoundedWindowAggExec: wdw=[SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: \"SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(2)), end_bound: Following(UInt64(1)) }, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING: Ok(Field { name: \"SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.b, annotated_data_finite2.a] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: Following(UInt64(5)) }], mode=[Sorted]", - " SortExec: expr=[b@2 ASC NULLS LAST,a@1 ASC NULLS LAST,c@3 ASC NULLS LAST]", - " BoundedWindowAggExec: wdw=[SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: \"SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(2)), end_bound: Following(UInt64(1)) }, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 1 FOLLOWING AND 5 FOLLOWING: Ok(Field { name: \"SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.b ASC NULLS LAST, annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 1 FOLLOWING AND 5 FOLLOWING\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Following(UInt64(1)), end_bound: Following(UInt64(5)) }], mode=[Sorted]", - " SortExec: expr=[a@1 ASC NULLS LAST,d@4 ASC NULLS LAST,b@2 ASC NULLS LAST,c@3 ASC NULLS LAST]", - " BoundedWindowAggExec: wdw=[SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: \"SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(2)), end_bound: Following(UInt64(1)) }, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND CURRENT ROW: Ok(Field { name: \"SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b, annotated_data_finite2.d] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND CURRENT ROW\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: CurrentRow }], mode=[Sorted]", - " SortExec: expr=[a@1 ASC NULLS LAST,b@2 ASC NULLS LAST,d@4 ASC NULLS LAST,c@3 ASC NULLS LAST]", - " BoundedWindowAggExec: wdw=[SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING: Ok(Field { name: \"SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 2 PRECEDING AND 1 FOLLOWING\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(2)), end_bound: Following(UInt64(1)) }, SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING: Ok(Field { name: \"SUM(annotated_data_finite2.c) PARTITION BY [annotated_data_finite2.a, annotated_data_finite2.b] ORDER BY [annotated_data_finite2.c ASC NULLS LAST] ROWS BETWEEN 5 PRECEDING AND 5 FOLLOWING\", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Rows, start_bound: Preceding(UInt64(5)), end_bound: Following(UInt64(5)) }], mode=[Sorted]", - " ProjectionExec: expr=[CAST(c@2 AS Int64) as CAST(annotated_data_finite2.c AS Int64)annotated_data_finite2.c, a@0 as a, b@1 as b, c@2 as c, d@3 as d]", - " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true", - ]; - - // Get string representation of the plan - let actual = get_plan_string(&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" - ); - - Ok(()) - } - - #[tokio::test] - async fn test_query7() -> Result<()> { - let config = SessionConfig::new().with_target_partitions(1); - let ctx = SessionContext::with_config(config); - - ctx.sql( - "CREATE TABLE t1(t1_id INT, t1_name TEXT, t1_int INT) AS VALUES - (11, 'a', 1), - (22, 'b', 2), - (33, 'c', 3), - (44, 'd', 4);", - ) - .await?; - - ctx.sql( - "CREATE TABLE t2(t2_id INT, t2_name TEXT, t2_int INT) AS VALUES - (11, 'z', 3), - (22, 'y', 1), - (44, 'x', 3), - (55, 'w', 3);", - ) - .await?; - - let sql = - "SELECT t1_id, (SELECT count(*) FROM t2 WHERE t2.t2_int = t1.t1_int) from t1"; - - let msg = format!("Creating logical plan for '{sql}'"); - let dataframe = ctx.sql(sql).await.expect(&msg); - let physical_plan = dataframe.create_physical_plan().await?; - print_plan(&physical_plan)?; - let _actual = collect(physical_plan.clone(), ctx.task_ctx()).await?; - // print_batches(&actual)?; - - let expected_optimized_lines: Vec<&str> = vec![ - "ProjectionExec: expr=[t1_id@0 as t1_id, CASE WHEN __always_true@4 IS NULL THEN 0 ELSE COUNT(*)@2 END as COUNT(*)]", - " ProjectionExec: expr=[t1_id@3 as t1_id, t1_int@4 as t1_int, COUNT(*)@0 as COUNT(*), t2_int@1 as t2_int, __always_true@2 as __always_true]", - " CoalesceBatchesExec: target_batch_size=8192", - " HashJoinExec: mode=CollectLeft, join_type=Right, on=[(t2_int@1, t1_int@1)]", - " ProjectionExec: expr=[COUNT(*)@2 as COUNT(*), t2_int@0 as t2_int, __always_true@1 as __always_true]", - " AggregateExec: mode=Final, gby=[t2_int@0 as t2_int, __always_true@1 as __always_true], aggr=[COUNT(*)]", - " AggregateExec: mode=Partial, gby=[t2_int@0 as t2_int, true as __always_true], aggr=[COUNT(*)]", - " MemoryExec: partitions=1, partition_sizes=[1]", - " MemoryExec: partitions=1, partition_sizes=[1]", - ]; - - // Get string representation of the plan - let actual = get_plan_string(&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" - ); - - Ok(()) - } - - #[tokio::test] - async fn test_query8() -> Result<()> { - let config = SessionConfig::new().with_target_partitions(1); - let ctx = SessionContext::with_config(config); - - ctx.sql( - "CREATE UNBOUNDED EXTERNAL TABLE annotated_data_infinite2 ( - a0 INTEGER, - a INTEGER, - b INTEGER, - c INTEGER, - d INTEGER - ) - STORED AS CSV - WITH HEADER ROW - WITH ORDER (a ASC, b ASC, c ASC) - LOCATION '../core/tests/data/window_2.csv'", - ) - .await?; - - let sql = "SELECT a, d, - SUM(c ORDER BY a DESC) as summation1 - FROM annotated_data_infinite2 - GROUP BY d, a"; - - let msg = format!("Creating logical plan for '{sql}'"); - let dataframe = ctx.sql(sql).await.expect(&msg); - let physical_plan = dataframe.create_physical_plan().await?; - print_plan(&physical_plan)?; - let _actual = collect(physical_plan.clone(), ctx.task_ctx()).await?; - // print_batches(&actual)?; - - let expected_optimized_lines: Vec<&str> = vec![ - "ProjectionExec: expr=[a@1 as a, d@0 as d, SUM(annotated_data_infinite2.c) ORDER BY [annotated_data_infinite2.a DESC NULLS FIRST]@2 as summation1]", - " AggregateExec: mode=Single, gby=[d@2 as d, a@0 as a], aggr=[SUM(annotated_data_infinite2.c)], ordering_mode=PartiallySorted([1])", - " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c, d], infinite_source=true, output_ordering=[a@0 ASC NULLS LAST], has_header=true", - ]; - - // Get string representation of the plan - let actual = get_plan_string(&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" - ); - - Ok(()) - } - - #[tokio::test] - async fn test_query9() -> Result<()> { - let config = SessionConfig::new().with_target_partitions(1); - let ctx = SessionContext::with_config(config); - - ctx.sql( - "CREATE EXTERNAL TABLE aggregate_test_100 ( - c1 VARCHAR NOT NULL, - c2 TINYINT NOT NULL, - c3 SMALLINT NOT NULL, - c4 SMALLINT, - c5 INT, - c6 BIGINT NOT NULL, - c7 SMALLINT NOT NULL, - c8 INT NOT NULL, - c9 INT UNSIGNED NOT NULL, - c10 BIGINT UNSIGNED NOT NULL, - c11 FLOAT NOT NULL, - c12 DOUBLE NOT NULL, - c13 VARCHAR NOT NULL - ) - STORED AS CSV - WITH HEADER ROW - LOCATION '../../testing/data/csv/aggregate_test_100.csv'", - ) - .await?; - - let sql = "WITH indices AS ( - SELECT 1 AS idx UNION ALL - SELECT 2 AS idx UNION ALL - SELECT 3 AS idx UNION ALL - SELECT 4 AS idx UNION ALL - SELECT 5 AS idx - ) - SELECT data.arr[indices.idx] as element, array_length(data.arr) as array_len, dummy - FROM ( - SELECT array_agg(distinct c2) as arr, count(1) as dummy FROM aggregate_test_100 - ) data - CROSS JOIN indices - ORDER BY 1"; - - // let sql = "SELECT array_agg(distinct c2) as arr, count(1) as dummy FROM aggregate_test_100"; - - let msg = format!("Creating logical plan for '{sql}'"); - let dataframe = ctx.sql(sql).await.expect(&msg); - let physical_plan = dataframe.create_physical_plan().await?; - print_plan(&physical_plan)?; - let _actual = collect(physical_plan.clone(), ctx.task_ctx()).await?; - // print_batches(&actual)?; - - let expected_optimized_lines: Vec<&str> = vec![ - "SortPreservingMergeExec: [element@0 ASC NULLS LAST]", - " SortExec: expr=[element@0 ASC NULLS LAST]", - " ProjectionExec: expr=[(arr@0).[idx@2] as element, array_length(arr@0) as array_len, dummy@1 as dummy]", - " CrossJoinExec", - " ProjectionExec: expr=[ARRAY_AGG(DISTINCT aggregate_test_100.c2)@0 as arr, COUNT(Int64(1))@1 as dummy]", - " AggregateExec: mode=Single, gby=[], aggr=[ARRAY_AGG(DISTINCT aggregate_test_100.c2), COUNT(Int64(1))]", - " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/testing/data/csv/aggregate_test_100.csv]]}, projection=[c2], has_header=true", - " UnionExec", - " ProjectionExec: expr=[1 as idx]", - " EmptyExec: produce_one_row=true", - " ProjectionExec: expr=[2 as idx]", - " EmptyExec: produce_one_row=true", - " ProjectionExec: expr=[3 as idx]", - " EmptyExec: produce_one_row=true", - " ProjectionExec: expr=[4 as idx]", - " EmptyExec: produce_one_row=true", - " ProjectionExec: expr=[5 as idx]", - " EmptyExec: produce_one_row=true", - ]; - - // Get string representation of the plan - let actual = get_plan_string(&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" - ); - - Ok(()) - } - - #[tokio::test] - async fn test_query10() -> Result<()> { - let config = SessionConfig::new().with_target_partitions(4); - let ctx = SessionContext::with_config(config); - - ctx.sql( - "CREATE EXTERNAL TABLE aggregate_test_100 ( - c1 VARCHAR NOT NULL, - c2 TINYINT NOT NULL, - c3 SMALLINT NOT NULL, - c4 SMALLINT, - c5 INT, - c6 BIGINT NOT NULL, - c7 SMALLINT NOT NULL, - c8 INT NOT NULL, - c9 INT UNSIGNED NOT NULL, - c10 BIGINT UNSIGNED NOT NULL, - c11 FLOAT NOT NULL, - c12 DOUBLE NOT NULL, - c13 VARCHAR NOT NULL - ) - STORED AS CSV - WITH HEADER ROW - LOCATION '../../testing/data/csv/aggregate_test_100.csv'", - ) - .await?; - - let sql = "SELECT c1, ROW_NUMBER() OVER (PARTITION BY c1) as rn1 FROM aggregate_test_100 ORDER BY c1 ASC"; - - // let sql = "SELECT array_agg(distinct c2) as arr, count(1) as dummy FROM aggregate_test_100"; - - let msg = format!("Creating logical plan for '{sql}'"); - let dataframe = ctx.sql(sql).await.expect(&msg); - let physical_plan = dataframe.create_physical_plan().await?; - print_plan(&physical_plan)?; - let _actual = collect(physical_plan.clone(), ctx.task_ctx()).await?; - // print_batches(&actual)?; - - let expected_optimized_lines: Vec<&str> = vec![ - "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() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: \"ROW_NUMBER() PARTITION BY [aggregate_test_100.c1] ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING\", 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)) }], mode=[Sorted]", - " SortExec: expr=[c1@0 ASC NULLS LAST]", - " CoalesceBatchesExec: target_batch_size=8192", - " RepartitionExec: partitioning=Hash([c1@0], 4), input_partitions=4", - " RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1", - " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1], has_header=true", - ]; - - // Get string representation of the plan - let actual = get_plan_string(&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" - ); - - Ok(()) - } - - #[tokio::test] - async fn test_query11() -> Result<()> { - let config = SessionConfig::new().with_target_partitions(4); - let ctx = SessionContext::with_config(config); - - ctx.sql( - "CREATE EXTERNAL TABLE annotated_data_finite2 ( - a0 INTEGER, - a INTEGER, - b INTEGER, - c INTEGER, - d INTEGER - ) - STORED AS CSV - WITH HEADER ROW - WITH ORDER (a ASC, b ASC, c ASC) - LOCATION '../core/tests/data/window_2.csv'", - ) - .await?; - - let sql = "SELECT * - FROM annotated_data_finite2 - WHERE a=0 - ORDER BY b, c;"; - - // let sql = "SELECT array_agg(distinct c2) as arr, count(1) as dummy FROM aggregate_test_100"; - - let msg = format!("Creating logical plan for '{sql}'"); - let dataframe = ctx.sql(sql).await.expect(&msg); - let physical_plan = dataframe.create_physical_plan().await?; - print_plan(&physical_plan)?; - let _actual = collect(physical_plan.clone(), ctx.task_ctx()).await?; - // print_batches(&actual)?; - - let expected_optimized_lines: Vec<&str> = vec![ - "SortPreservingMergeExec: [b@2 ASC NULLS LAST,c@3 ASC NULLS LAST]", - " CoalesceBatchesExec: target_batch_size=8192", - " FilterExec: a@1 = 0", - " RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1", - " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC NULLS LAST, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true", - ]; - - // Get string representation of the plan - let actual = get_plan_string(&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" - ); - - Ok(()) - } - - #[tokio::test] - async fn test_query12() -> Result<()> { - let config = SessionConfig::new() - .with_target_partitions(4) - .with_repartition_joins(false); - let ctx = SessionContext::with_config(config); - - ctx.sql( - "CREATE EXTERNAL TABLE annotated_data ( - a0 INTEGER, - a INTEGER, - b INTEGER, - c INTEGER, - d INTEGER - ) - STORED AS CSV - WITH HEADER ROW - WITH ORDER (a ASC, b ASC, c ASC) - LOCATION '../core/tests/data/window_2.csv'", - ) - .await?; - - let sql = "SELECT t2.a - FROM annotated_data as t1 - INNER JOIN annotated_data as t2 - ON t1.c = t2.c ORDER BY t2.a - LIMIT 5"; - - // let sql = "SELECT array_agg(distinct c2) as arr, count(1) as dummy FROM aggregate_test_100"; - - let msg = format!("Creating logical plan for '{sql}'"); - let dataframe = ctx.sql(sql).await.expect(&msg); - let physical_plan = dataframe.create_physical_plan().await?; - print_plan(&physical_plan)?; - let _actual = collect(physical_plan.clone(), ctx.task_ctx()).await?; - // print_batches(&actual)?; - - let expected_optimized_lines: Vec<&str> = vec![ - "GlobalLimitExec: skip=0, fetch=5", - " SortPreservingMergeExec: [a@0 ASC NULLS LAST], fetch=5", - " ProjectionExec: expr=[a@1 as a]", - " CoalesceBatchesExec: target_batch_size=8192", - " HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(c@0, c@1)]", - " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[c], has_header=true", - " RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1", - " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c], output_ordering=[a@0 ASC NULLS LAST], has_header=true", - ]; - - // Get string representation of the plan - let actual = get_plan_string(&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" - ); - - Ok(()) - } - - #[tokio::test] - async fn test_query13() -> Result<()> { - // TODO: Add this test case to .slt - let config = SessionConfig::new().with_target_partitions(1); - let ctx = SessionContext::with_config(config); - - ctx.sql( - "CREATE EXTERNAL TABLE annotated_data_infinite2 ( - a0 INTEGER, - a INTEGER, - b INTEGER, - c INTEGER, - d INTEGER - ) - STORED AS CSV - WITH HEADER ROW - WITH ORDER (a ASC, b ASC, c ASC) - LOCATION '../core/tests/data/window_2.csv'", - ) - .await?; - - let sql = "SELECT l.a, LAST_VALUE(r.b ORDER BY r.a) as last_col1 - FROM annotated_data_infinite2 as l - JOIN annotated_data_infinite2 as r - ON l.a = r.a - GROUP BY l.a, l.b, l.c - ORDER BY l.a;"; - - let msg = format!("Creating logical plan for '{sql}'"); - let dataframe = ctx.sql(sql).await.expect(&msg); - let physical_plan = dataframe.create_physical_plan().await?; - print_plan(&physical_plan)?; - // let actual = collect(physical_plan.clone(), ctx.task_ctx()).await?; - // print_batches(&actual)?; - - let expected_optimized_lines: Vec<&str> = vec![ - "ProjectionExec: expr=[a@0 as a, LAST_VALUE(r.b) ORDER BY [r.a ASC NULLS LAST]@3 as last_col1]", - " AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[LAST_VALUE(r.b)], ordering_mode=PartiallySorted([0])", - " CoalesceBatchesExec: target_batch_size=8192", - " HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@0, a@0)]", - " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true", - " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], has_header=true", - ]; - - // Get string representation of the plan - let actual = get_plan_string(&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" - ); - Ok(()) - } - - #[tokio::test] - async fn test_query14() -> Result<()> { - // TODO: Add this test case to .slt - let config = SessionConfig::new() - .with_target_partitions(4) - .with_bounded_order_preserving_variants(true); - let ctx = SessionContext::with_config(config); - - ctx.sql( - "CREATE EXTERNAL TABLE annotated_data_infinite2 ( - a0 INTEGER, - a INTEGER, - b INTEGER, - c INTEGER, - d INTEGER - ) - STORED AS CSV - WITH HEADER ROW - WITH ORDER (a ASC, b ASC, c ASC) - LOCATION '../core/tests/data/window_2.csv'", - ) - .await?; - - let sql = "SELECT l.a, LAST_VALUE(r.b ORDER BY r.a) as last_col1 - FROM annotated_data_infinite2 as l - JOIN annotated_data_infinite2 as r - ON l.a = r.a - GROUP BY l.a, l.b, l.c - ORDER BY l.a;"; - - let msg = format!("Creating logical plan for '{sql}'"); - let dataframe = ctx.sql(sql).await.expect(&msg); - let physical_plan = dataframe.create_physical_plan().await?; - print_plan(&physical_plan)?; - let actual = collect(physical_plan.clone(), ctx.task_ctx()).await?; - print_batches(&actual)?; - - // TODO: make plan below without sort - let expected_optimized_lines: Vec<&str> = vec![ - "SortPreservingMergeExec: [a@0 ASC NULLS LAST]", - " SortExec: expr=[a@0 ASC NULLS LAST]", - " ProjectionExec: expr=[a@0 as a, LAST_VALUE(r.b) ORDER BY [r.a ASC NULLS LAST]@3 as last_col1]", - " AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[LAST_VALUE(r.b)]", - " CoalesceBatchesExec: target_batch_size=8192", - " RepartitionExec: partitioning=Hash([a@0, b@1, c@2], 4), input_partitions=4", - " AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[LAST_VALUE(r.b)], ordering_mode=PartiallySorted([0])", - " CoalesceBatchesExec: target_batch_size=8192", - " HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a@0)]", - " CoalesceBatchesExec: target_batch_size=8192", - " RepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4", - " RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1", - " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true", - " CoalesceBatchesExec: target_batch_size=8192", - " SortPreservingRepartitionExec: partitioning=Hash([a@0], 4), input_partitions=4", - " RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1", - " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], has_header=true", - ]; - - // Get string representation of the plan - let actual = get_plan_string(&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" - ); - Ok(()) - } - - // oeq bug - #[tokio::test] - async fn test_query15() -> Result<()> { - let config = SessionConfig::new().with_target_partitions(1); - let ctx = SessionContext::with_config(config); - - ctx.sql( - "CREATE EXTERNAL TABLE lineitem ( - l_a0 INTEGER, - l_a INTEGER, - l_b INTEGER, - l_c INTEGER, - l_d INTEGER - ) - STORED AS CSV - WITH HEADER ROW - WITH ORDER (l_a ASC) - LOCATION 'tests/data/window_2.csv'", - ) - .await?; - - ctx.sql( - "CREATE EXTERNAL TABLE orders ( - o_a0 INTEGER, - o_a INTEGER, - o_b INTEGER, - o_c INTEGER, - o_d INTEGER - ) - STORED AS CSV - WITH HEADER ROW - WITH ORDER (o_a ASC) - LOCATION 'tests/data/window_2.csv'", - ) - .await?; - - let sql = "SELECT LAST_VALUE(l_d ORDER BY l_a) AS amount_usd - FROM lineitem - INNER JOIN ( - SELECT *, ROW_NUMBER() OVER (ORDER BY o_a) as row_n FROM orders - ) - ON o_d = l_d AND - l_a >= o_a - 10 - GROUP BY row_n - ORDER BY row_n"; - - let msg = format!("Creating logical plan for '{sql}'"); - let dataframe = ctx.sql(sql).await.expect(&msg); - let physical_plan = dataframe.create_physical_plan().await?; - print_plan(&physical_plan)?; - - let expected = vec![ - "ProjectionExec: expr=[amount_usd@0 as amount_usd]", - " ProjectionExec: expr=[LAST_VALUE(lineitem.l_d) ORDER BY [lineitem.l_a ASC NULLS LAST]@1 as amount_usd, row_n@0 as row_n]", - " AggregateExec: mode=Single, gby=[row_n@2 as row_n], aggr=[LAST_VALUE(lineitem.l_d)], ordering_mode=Sorted", - " ProjectionExec: expr=[l_a@0 as l_a, l_d@1 as l_d, row_n@4 as row_n]", - " CoalesceBatchesExec: target_batch_size=8192", - " HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(l_d@1, o_d@1)], filter=CAST(l_a@0 AS Int64) >= CAST(o_a@1 AS Int64) - 10", - " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[l_a, l_d], output_ordering=[l_a@0 ASC NULLS LAST], has_header=true", - " ProjectionExec: expr=[o_a@0 as o_a, o_d@1 as o_d, ROW_NUMBER() ORDER BY [orders.o_a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as row_n]", - " BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [orders.o_a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: \"ROW_NUMBER() ORDER BY [orders.o_a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW\", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow }], mode=[Sorted]", - " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[o_a, o_d], output_ordering=[o_a@0 ASC NULLS LAST], has_header=true", - ]; - // Get string representation of the plan - let actual = get_plan_string(&physical_plan); - assert_eq!( - expected, actual, - "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" - ); - - let actual = collect(physical_plan, ctx.task_ctx()).await?; - print_batches(&actual)?; - Ok(()) - } - - #[tokio::test] - async fn test_query16() -> Result<()> { - let config = SessionConfig::new().with_target_partitions(1); - let ctx = SessionContext::with_config(config); - - ctx.sql( - "CREATE EXTERNAL TABLE multiple_ordered_table ( - a0 INTEGER, - a INTEGER, - b INTEGER, - c INTEGER, - d INTEGER - ) - STORED AS CSV - WITH HEADER ROW - WITH ORDER (a ASC, b ASC) - WITH ORDER (c ASC) - LOCATION '../core/tests/data/window_2.csv'", - ) - .await?; - - let sql = "SELECT LAST_VALUE(l.d ORDER BY l.a) AS amount_usd - FROM multiple_ordered_table AS l - INNER JOIN ( - SELECT *, ROW_NUMBER() OVER (ORDER BY r.a) as row_n FROM multiple_ordered_table AS r - ) - ON l.d = r.d AND - l.a >= r.a - 10 - GROUP BY row_n - ORDER BY row_n"; - - let msg = format!("Creating logical plan for '{sql}'"); - let dataframe = ctx.sql(sql).await.expect(&msg); - let physical_plan = dataframe.create_physical_plan().await?; - print_plan(&physical_plan)?; - - let expected = vec![ - "ProjectionExec: expr=[amount_usd@0 as amount_usd]", - " ProjectionExec: expr=[LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST]@1 as amount_usd, row_n@0 as row_n]", - " AggregateExec: mode=Single, gby=[row_n@2 as row_n], aggr=[LAST_VALUE(l.d)], ordering_mode=Sorted", - " ProjectionExec: expr=[a@0 as a, d@1 as d, row_n@4 as row_n]", - " CoalesceBatchesExec: target_batch_size=8192", - " HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d@1, d@1)], filter=CAST(a@0 AS Int64) >= CAST(a@1 AS Int64) - 10", - " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", - " ProjectionExec: expr=[a@0 as a, d@1 as d, ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as row_n]", - " BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: \"ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW\", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow }], mode=[Sorted]", - " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true", - ]; - // Get string representation of the plan - let actual = get_plan_string(&physical_plan); - assert_eq!( - expected, actual, - "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" - ); - - let actual = collect(physical_plan, ctx.task_ctx()).await?; - print_batches(&actual)?; - Ok(()) - } - - #[tokio::test] - async fn test_query17() -> Result<()> { - let config = SessionConfig::new().with_target_partitions(1); - let ctx = SessionContext::with_config(config); - - ctx.sql( - "CREATE EXTERNAL TABLE multiple_ordered_table ( - a0 INTEGER, - a INTEGER, - b INTEGER, - c INTEGER, - d INTEGER - ) - STORED AS CSV - WITH HEADER ROW - WITH ORDER (a ASC, b ASC) - WITH ORDER (c ASC) - LOCATION '../core/tests/data/window_2.csv'", - ) - .await?; - - let sql = "SELECT MIN(d) OVER(ORDER BY c ASC) as min1, - MAX(d) OVER(PARTITION BY b, a ORDER BY c ASC) as max1 - FROM multiple_ordered_table"; - - let msg = format!("Creating logical plan for '{sql}'"); - let dataframe = ctx.sql(sql).await.expect(&msg); - let physical_plan = dataframe.create_physical_plan().await?; - print_plan(&physical_plan)?; - - let expected = vec![ - "ProjectionExec: expr=[MIN(multiple_ordered_table.d) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as min1, MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as max1]", - " BoundedWindowAggExec: wdw=[MIN(multiple_ordered_table.d) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: \"MIN(multiple_ordered_table.d) ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow }], mode=[Sorted]", - " ProjectionExec: expr=[c@2 as c, d@3 as d, MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]", - " BoundedWindowAggExec: wdw=[MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: \"MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW\", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow }], mode=[Sorted]", - " CsvExec: file_groups={1 group: [[Users/akurmustafa/projects/synnada/arrow-datafusion-synnada/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], has_header=true", - ]; - // Get string representation of the plan - let actual = get_plan_string(&physical_plan); - assert_eq!( - expected, actual, - "\n**Optimized Plan Mismatch\n\nexpected:\n\n{expected:#?}\nactual:\n\n{actual:#?}\n\n" - ); - - let actual = collect(physical_plan, ctx.task_ctx()).await?; - print_batches(&actual)?; - Ok(()) - } -} diff --git a/datafusion/core/src/physical_optimizer/sort_pushdown.rs b/datafusion/core/src/physical_optimizer/sort_pushdown.rs index fe1236472dd6..fc7b30da0b56 100644 --- a/datafusion/core/src/physical_optimizer/sort_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/sort_pushdown.rs @@ -132,11 +132,9 @@ pub(crate) fn pushdown_sorts( .ordering_satisfy_requirement(parent_required) { // If the current plan is a SortExec, modify it to satisfy parent requirements: - let parent_required_expr = PhysicalSortRequirement::to_sort_exprs( - parent_required.ok_or_else(err)?.iter().cloned(), - ); + let parent_requirement = parent_required.ok_or_else(err)?; new_plan = sort_exec.input().clone(); - add_sort_above(&mut new_plan, parent_required_expr, sort_exec.fetch())?; + add_sort_above(&mut new_plan, parent_requirement, sort_exec.fetch())?; }; let required_ordering = new_plan .output_ordering() @@ -177,11 +175,9 @@ pub(crate) fn pushdown_sorts( })) } else { // Can not push down requirements, add new SortExec: - let parent_required_expr = PhysicalSortRequirement::to_sort_exprs( - parent_required.ok_or_else(err)?.iter().cloned(), - ); let mut new_plan = plan.clone(); - add_sort_above(&mut new_plan, parent_required_expr, None)?; + let parent_requirement = parent_required.ok_or_else(err)?; + add_sort_above(&mut new_plan, parent_requirement, None)?; Ok(Transformed::Yes(SortPushDown::init(new_plan))) } } diff --git a/datafusion/core/src/physical_optimizer/utils.rs b/datafusion/core/src/physical_optimizer/utils.rs index 32197aa7cfeb..9c70a208b510 100644 --- a/datafusion/core/src/physical_optimizer/utils.rs +++ b/datafusion/core/src/physical_optimizer/utils.rs @@ -31,7 +31,7 @@ use crate::physical_plan::union::UnionExec; use crate::physical_plan::windows::{BoundedWindowAggExec, WindowAggExec}; use crate::physical_plan::{displayable, ExecutionPlan}; -use datafusion_physical_expr::PhysicalSortExpr; +use datafusion_physical_expr::PhysicalSortRequirement; /// This object implements a tree that we use while keeping track of paths /// leading to [`SortExec`]s. @@ -100,14 +100,15 @@ pub(crate) fn get_children_exectrees( /// given ordering requirements while preserving the original partitioning. pub fn add_sort_above( node: &mut Arc, - sort_expr: Vec, + sort_requirement: &[PhysicalSortRequirement], fetch: Option, ) -> Result<()> { // If the ordering requirement is already satisfied, do not add a sort. if !node .ordering_equivalence_properties() - .ordering_satisfy(Some(&sort_expr)) + .ordering_satisfy_requirement_concrete(sort_requirement) { + let sort_expr = PhysicalSortRequirement::to_sort_exprs(sort_requirement.to_vec()); let new_sort = SortExec::new(sort_expr, node.clone()).with_fetch(fetch); *node = Arc::new(if node.output_partitioning().partition_count() > 1 { diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 4632293ae474..ec1cb952a6dc 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -859,6 +859,8 @@ impl OrderingEquivalenceProperties { pub fn ordering_satisfy_concrete(&self, required: &[PhysicalSortExpr]) -> bool { let required_normalized = self.normalize_sort_exprs(required); let provided_normalized = self.oeq_group().output_ordering().unwrap_or(vec![]); + println!("required_normalized: {:?}", required_normalized); + println!("provided_normalized: {:?}", provided_normalized); if required_normalized.len() > provided_normalized.len() { return false; } @@ -932,7 +934,6 @@ impl OrderingEquivalenceProperties { /// provided [`PhysicalSortExpr`]s. pub fn ordering_satisfy_requirement( &self, - // provided: Option<&[PhysicalSortExpr]>, required: Option<&[PhysicalSortRequirement]>, ) -> bool { match required { @@ -949,6 +950,8 @@ impl OrderingEquivalenceProperties { ) -> bool { let provided_normalized = self.oeq_group().output_ordering().unwrap_or(vec![]); let required_normalized = self.normalize_sort_requirements(required); + println!("req required_normalized: {:?}", required_normalized); + println!("req provided_normalized: {:?}", provided_normalized); if required_normalized.len() > provided_normalized.len() { return false; } @@ -1046,8 +1049,6 @@ impl OrderingEquivalenceProperties { // prefer None. Instead of Linear. return Ok(None); } - println!("req1:{:?}", req); - println!("partition_by_oeq: {:?}", partition_by_oeq); if partition_by_oeq.ordering_satisfy_requirement_concrete(&req) { // Window can be run with existing ordering return Ok(Some((false, partition_search_mode))); @@ -1056,7 +1057,6 @@ impl OrderingEquivalenceProperties { PhysicalSortRequirement::from_sort_exprs(&reverse_order_bys(&orderby_keys)); let req = [partition_by_reqs, reverse_order_by_reqs].concat(); let req = collapse_lex_req(req); - println!("req2:{:?}", req); if partition_by_oeq.ordering_satisfy_requirement_concrete(&req) { // Window can be run with existing ordering, if the ordering requirements would be reversed return Ok(Some((true, partition_search_mode))); diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index f9b6868e2cce..4a341a8d6ab4 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -517,6 +517,7 @@ impl AggregateExec { let req = PhysicalSortRequirement::from_sort_exprs(&ordering_req); new_requirement.extend(req); new_requirement = collapse_lex_req(new_requirement); + println!("new_requirement: {:?}", new_requirement); // construct a map from the input columns to the output columns of the Aggregation let mut columns_map: HashMap> = HashMap::new(); diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index 01c0131fdb62..6ddf570501c7 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -3279,6 +3279,127 @@ CoalesceBatchesExec: target_batch_size=4096 ------BoundedWindowAggExec: wdw=[ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "ROW_NUMBER() ROWS BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING", 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)) }], mode=[Sorted] --------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a0, a, b, c, d], output_ordering=[a@1 ASC, b@2 ASC NULLS LAST, c@3 ASC NULLS LAST], has_header=true +query TT +EXPLAIN SELECT l.a, LAST_VALUE(r.b ORDER BY r.a ASC NULLS FIRST) as last_col1 +FROM annotated_data as l +JOIN annotated_data as r +ON l.a = r.a +GROUP BY l.a, l.b, l.c +ORDER BY l.a ASC NULLS FIRST; +---- +logical_plan +Sort: l.a ASC NULLS FIRST +--Projection: l.a, LAST_VALUE(r.b) ORDER BY [r.a ASC NULLS FIRST] AS last_col1 +----Aggregate: groupBy=[[l.a, l.b, l.c]], aggr=[[LAST_VALUE(r.b) ORDER BY [r.a ASC NULLS FIRST]]] +------Inner Join: l.a = r.a +--------SubqueryAlias: l +----------TableScan: annotated_data projection=[a, b, c] +--------SubqueryAlias: r +----------TableScan: annotated_data projection=[a, b] +physical_plan +ProjectionExec: expr=[a@0 as a, LAST_VALUE(r.b) ORDER BY [r.a ASC NULLS FIRST]@3 as last_col1] +--AggregateExec: mode=Single, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[LAST_VALUE(r.b)], ordering_mode=PartiallySorted([0]) +----CoalesceBatchesExec: target_batch_size=4096 +------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(a@0, a@0)] +--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], output_ordering=[a@0 ASC, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true +--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC, b@1 ASC NULLS LAST], has_header=true + +# create a table where there more than one valid ordering +# that describes table. +statement ok +CREATE EXTERNAL TABLE multiple_ordered_table ( + a0 INTEGER, + a INTEGER, + b INTEGER, + c INTEGER, + d INTEGER +) +STORED AS CSV +WITH HEADER ROW +WITH ORDER (a ASC, b ASC) +WITH ORDER (c ASC) +LOCATION '../core/tests/data/window_2.csv'; + +query TT +EXPLAIN SELECT LAST_VALUE(l.d ORDER BY l.a) AS amount_usd +FROM multiple_ordered_table AS l +INNER JOIN ( + SELECT *, ROW_NUMBER() OVER (ORDER BY r.a) as row_n FROM multiple_ordered_table AS r +) +ON l.d = r.d AND + l.a >= r.a - 10 +GROUP BY row_n +ORDER BY row_n +---- +logical_plan +Projection: amount_usd +--Sort: row_n ASC NULLS LAST +----Projection: LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST] AS amount_usd, row_n +------Aggregate: groupBy=[[row_n]], aggr=[[LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST]]] +--------Projection: l.a, l.d, row_n +----------Inner Join: l.d = r.d Filter: CAST(l.a AS Int64) >= CAST(r.a AS Int64) - Int64(10) +------------SubqueryAlias: l +--------------TableScan: multiple_ordered_table projection=[a, d] +------------Projection: r.a, r.d, ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS row_n +--------------WindowAggr: windowExpr=[[ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +----------------SubqueryAlias: r +------------------TableScan: multiple_ordered_table projection=[a, d] +physical_plan +ProjectionExec: expr=[amount_usd@0 as amount_usd] +--ProjectionExec: expr=[LAST_VALUE(l.d) ORDER BY [l.a ASC NULLS LAST]@1 as amount_usd, row_n@0 as row_n] +----AggregateExec: mode=Single, gby=[row_n@2 as row_n], aggr=[LAST_VALUE(l.d)], ordering_mode=Sorted +------ProjectionExec: expr=[a@0 as a, d@1 as d, row_n@4 as row_n] +--------CoalesceBatchesExec: target_batch_size=4096 +----------HashJoinExec: mode=CollectLeft, join_type=Inner, on=[(d@1, d@1)], filter=CAST(a@0 AS Int64) >= CAST(a@1 AS Int64) - 10 +------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true +------------ProjectionExec: expr=[a@0 as a, d@1 as d, ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as row_n] +--------------BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [r.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow }], mode=[Sorted] +----------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true + +# run query above in multiple partitions +statement ok +set datafusion.execution.target_partitions = 2; + +# use bounded variants +statement ok +set datafusion.optimizer.bounded_order_preserving_variants = true; + +query TT +EXPLAIN SELECT l.a, LAST_VALUE(r.b ORDER BY r.a ASC NULLS FIRST) as last_col1 +FROM annotated_data as l +JOIN annotated_data as r +ON l.a = r.a +GROUP BY l.a, l.b, l.c +ORDER BY l.a ASC NULLS FIRST; +---- +logical_plan +Sort: l.a ASC NULLS FIRST +--Projection: l.a, LAST_VALUE(r.b) ORDER BY [r.a ASC NULLS FIRST] AS last_col1 +----Aggregate: groupBy=[[l.a, l.b, l.c]], aggr=[[LAST_VALUE(r.b) ORDER BY [r.a ASC NULLS FIRST]]] +------Inner Join: l.a = r.a +--------SubqueryAlias: l +----------TableScan: annotated_data projection=[a, b, c] +--------SubqueryAlias: r +----------TableScan: annotated_data projection=[a, b] +physical_plan +SortPreservingMergeExec: [a@0 ASC] +--SortExec: expr=[a@0 ASC] +----ProjectionExec: expr=[a@0 as a, LAST_VALUE(r.b) ORDER BY [r.a ASC NULLS FIRST]@3 as last_col1] +------AggregateExec: mode=FinalPartitioned, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[LAST_VALUE(r.b)] +--------CoalesceBatchesExec: target_batch_size=4096 +----------RepartitionExec: partitioning=Hash([a@0, b@1, c@2], 2), input_partitions=2 +------------AggregateExec: mode=Partial, gby=[a@0 as a, b@1 as b, c@2 as c], aggr=[LAST_VALUE(r.b)], ordering_mode=PartiallySorted([0]) +--------------CoalesceBatchesExec: target_batch_size=4096 +----------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(a@0, a@0)] +------------------CoalesceBatchesExec: target_batch_size=4096 +--------------------RepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 +----------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], output_ordering=[a@0 ASC, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true +------------------CoalesceBatchesExec: target_batch_size=4096 +--------------------SortPreservingRepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 +----------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 +------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC, b@1 ASC NULLS LAST], has_header=true + #### # Config teardown #### @@ -3292,5 +3413,9 @@ set datafusion.optimizer.prefer_hash_join = true; statement ok set datafusion.execution.target_partitions = 2; +statement ok +set datafusion.optimizer.bounded_order_preserving_variants = false; + statement ok drop table annotated_data; + diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 4e8bae754764..04ee8c624d21 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -3397,6 +3397,8 @@ WITH ORDER (a ASC, b ASC) WITH ORDER (c ASC) LOCATION '../core/tests/data/window_2.csv'; +# All of the window execs in the physical plan should work in the +# sorted mode. query TT EXPLAIN SELECT MIN(d) OVER(ORDER BY c ASC) as min1, MAX(d) OVER(PARTITION BY b, a ORDER BY c ASC) as max1 From 055dc916541ddc2795de356cd14e1126f89a7514 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 3 Oct 2023 15:05:36 +0300 Subject: [PATCH 021/122] Minor changes --- .../enforce_distribution.rs | 9 ------ .../src/physical_optimizer/enforce_sorting.rs | 30 ------------------- .../physical-plan/src/aggregates/mod.rs | 7 ----- 3 files changed, 46 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index 0bc6b18860b5..42ac0e69dd9d 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -1208,15 +1208,6 @@ fn replace_order_preserving_variants_helper( exec_tree.plan.clone().with_new_children(updated_children) } -#[allow(dead_code)] -fn print_plan(plan: &Arc) { - let formatted = crate::physical_plan::displayable(plan.as_ref()) - .indent(true) - .to_string(); - let actual: Vec<&str> = formatted.trim().lines().collect(); - println!("{:#?}", actual); -} - /// This function checks whether we need to add additional data exchange /// operators to satisfy distribution requirements. Since this function /// takes care of such requirements, we should avoid manually adding data diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index 13712791484b..f5172eb4b7ed 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -307,15 +307,6 @@ impl TreeNode for PlanWithCorrespondingCoalescePartitions { } } -#[allow(dead_code)] -fn print_plan(plan: &Arc) { - let formatted = crate::physical_plan::displayable(plan.as_ref()) - .indent(true) - .to_string(); - let actual: Vec<&str> = formatted.trim().lines().collect(); - println!("{:#?}", actual); -} - /// The boolean flag `repartition_sorts` defined in the config indicates /// whether we elect to transform [`CoalescePartitionsExec`] + [`SortExec`] cascades /// into [`SortExec`] + [`SortPreservingMergeExec`] cascades, which enables us to @@ -352,10 +343,8 @@ impl PhysicalOptimizerRule for EnforceSorting { // Execute a top-down traversal to exploit sort push-down opportunities // missed by the bottom-up traversal: - // print_plan(&updated_plan.plan); let sort_pushdown = SortPushDown::init(updated_plan.plan); let adjusted = sort_pushdown.transform_down(&pushdown_sorts)?; - // print_plan(&adjusted.plan); Ok(adjusted.plan) } @@ -447,7 +436,6 @@ fn ensure_sorting( let plan = requirements.plan; let mut children = plan.children(); let mut sort_onwards = requirements.sort_onwards; - // print_plan(&plan); if let Some(result) = analyze_immediate_sort_removal(&plan, &sort_onwards) { return Ok(Transformed::Yes(result)); } @@ -461,10 +449,6 @@ fn ensure_sorting( let physical_ordering = child.output_ordering(); match (required_ordering, physical_ordering) { (Some(required_ordering), Some(_)) => { - // println!("child"); - // print_plan(&child); - // println!("required_ordering: {:?}", required_ordering); - // println!("child.ordering_equivalence_properties(): {:?}", child.ordering_equivalence_properties()); if !child .ordering_equivalence_properties() .ordering_satisfy_requirement_concrete(&required_ordering) @@ -526,20 +510,6 @@ fn analyze_immediate_sort_removal( ) -> Option { if let Some(sort_exec) = plan.as_any().downcast_ref::() { let sort_input = sort_exec.input().clone(); - // println!("sort input"); - // print_plan(&plan); - // println!( - // "sort_input.output_ordering(): {:?}", - // sort_input.output_ordering() - // ); - // println!( - // "sort_exec.output_ordering(): {:?}", - // sort_exec.output_ordering() - // ); - // println!( - // "sort_input.ordering_equivalence_properties(): {:?}", - // sort_input.ordering_equivalence_properties() - // ); // If this sort is unnecessary, we should remove it: if sort_input diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 4a341a8d6ab4..314405431820 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -412,13 +412,6 @@ fn get_aggregate_search_mode( Ok(partition_search_mode) } -#[allow(dead_code)] -fn print_plan(plan: &Arc) { - let formatted = displayable(plan.as_ref()).indent(true).to_string(); - let actual: Vec<&str> = formatted.trim().lines().collect(); - println!("{:#?}", actual); -} - /// Check whether group by expression contains all of the expression inside `requirement` // As an example Group By (c,b,a) contains all of the expressions in the `requirement`: (a ASC, b DESC) fn group_by_contains_all_requirements( From 2f63ac7e3010c452dc1a1e051c1d113f9b2aba0b Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 3 Oct 2023 15:29:04 +0300 Subject: [PATCH 022/122] Simplifications --- datafusion/physical-expr/src/equivalence.rs | 178 ++++---------------- datafusion/physical-plan/src/joins/utils.rs | 9 +- 2 files changed, 39 insertions(+), 148 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index ec1cb952a6dc..c2f0972266d0 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -358,13 +358,17 @@ impl OrderingEquivalentGroup { self.remove_redundant_entries(); } + /// Removes redundant orderings from the state. + /// For instance, If we already know that + /// ordering: [a ASC, b ASC, c DESC] is valid for the schema. + /// There is no need to keep ordering [a ASC, b ASC] in the state. fn remove_redundant_entries(&mut self) { // Make sure there is no redundant entry let mut res: Vec = vec![]; for ordering in self.iter() { let mut is_inside = false; for item in &mut res { - if let Some(finer) = get_finer(item, ordering) { + if let Some(finer) = Self::get_finer_strict(item, ordering) { *item = finer; is_inside = true; } @@ -424,6 +428,31 @@ impl OrderingEquivalentGroup { } OrderingEquivalentGroup::new(res) } + + /// Adds `offset` value to the index of each expression inside `OrderingEquivalentGroup`. + pub fn add_offset(&self, offset: usize) -> Result { + let res = self + .inner + .iter() + .map(|ordering| add_offset_to_lex_ordering(ordering, offset)) + .collect::>>()?; + Ok(OrderingEquivalentGroup::new(res)) + } + + /// Return finer ordering between lhs and rhs. + fn get_finer_strict( + lhs: &[PhysicalSortExpr], + rhs: &[PhysicalSortExpr], + ) -> Option> { + if izip!(lhs.iter(), rhs.iter()).all(|(lhs, rhs)| lhs.eq(rhs)) { + if lhs.len() > rhs.len() { + return Some(lhs.to_vec()); + } else { + return Some(rhs.to_vec()); + } + } + None + } } /// `OrderingEquivalenceProperties` keeps track of columns that describe the @@ -517,8 +546,8 @@ impl OrderingEquivalenceProperties { for ordering in normalized_ordering.into_iter() { let mut is_inside = false; for item in &mut res { - if let Some(finer) = get_finer(item, &ordering) { - *item = finer; + if let Some(finer) = self.get_finer_ordering(item, &ordering) { + *item = finer.to_vec(); is_inside = true; } } @@ -527,7 +556,7 @@ impl OrderingEquivalenceProperties { } } self.oeq_group = OrderingEquivalentGroup::new(res); - // TODO: Add redundant entry check + self.oeq_group.remove_redundant_entries(); } /// Add physical expression that have constant value to the `self.constants` @@ -1076,147 +1105,10 @@ pub enum PartitionSearchMode { Sorted, } +// Stores the mapping between source expression and target expression during projection +// Indices in the vector corresponds to index after projection. type ProjectionMapping = Vec<(Arc, Arc)>; -/// Adds `offset` value to the index of each expression inside `self.head` and `self.others`. -pub fn add_offset( - in_data: &OrderingEquivalentGroup, - offset: usize, -) -> Result { - let res = in_data - .iter() - .map(|ordering| add_offset_to_lex_ordering(ordering, offset)) - .collect::>>()?; - Ok(OrderingEquivalentGroup::new(res)) -} - -fn get_finer( - lhs: &[PhysicalSortExpr], - rhs: &[PhysicalSortExpr], -) -> Option> { - if izip!(lhs.iter(), rhs.iter()).all(|(lhs, rhs)| lhs.eq(rhs)) { - if lhs.len() > rhs.len() { - return Some(lhs.to_vec()); - } else { - return Some(rhs.to_vec()); - } - } - None -} - -// impl OrderingEquivalentClass { -// /// This function updates ordering equivalences with alias information. -// /// For instance, assume columns `a` and `b` are aliases (a as b), and -// /// orderings `a ASC` and `c DESC` are equivalent. Here, we replace column -// /// `a` with `b` in ordering equivalence expressions. After this function, -// /// `a ASC`, `c DESC` will be converted to the `b ASC`, `c DESC`. -// fn update_with_aliases( -// &mut self, -// oeq_alias_map: &[(Column, Column)], -// fields: &Fields, -// ) { -// let is_head_invalid = self.head.iter().any(|sort_expr| { -// collect_columns(&sort_expr.expr) -// .iter() -// .any(|col| is_column_invalid_in_new_schema(col, fields)) -// }); -// // If head is invalidated, update head with alias expressions -// if is_head_invalid { -// self.head = update_with_alias(self.head.clone(), oeq_alias_map); -// } else { -// let new_oeq_expr = update_with_alias(self.head.clone(), oeq_alias_map); -// self.insert(new_oeq_expr); -// } -// for ordering in self.others.clone().into_iter() { -// self.insert(update_with_alias(ordering, oeq_alias_map)); -// } -// } -// -// /// Adds `offset` value to the index of each expression inside `self.head` and `self.others`. -// pub fn add_offset(&self, offset: usize) -> Result { -// let head = add_offset_to_lex_ordering(self.head(), offset)?; -// let others = self -// .others() -// .iter() -// .map(|ordering| add_offset_to_lex_ordering(ordering, offset)) -// .collect::>>()?; -// Ok(OrderingEquivalentClass::new(head, others)) -// } -// -// // /// This function normalizes `OrderingEquivalenceProperties` according to `eq_properties`. -// // /// More explicitly, it makes sure that expressions in `oeq_class` are head entries -// // /// in `eq_properties`, replacing any non-head entries with head entries if necessary. -// // pub fn normalize_with_equivalence_properties( -// // &self, -// // eq_properties: &EquivalenceProperties, -// // ) -> OrderingEquivalentClass { -// // let head = eq_properties.normalize_sort_exprs(self.head()); -// // -// // let others = self -// // .others() -// // .iter() -// // .map(|other| eq_properties.normalize_sort_exprs(other)) -// // .collect(); -// // -// // EquivalentClass::new(head, others) -// // } -// -// /// Prefix with existing ordering. -// pub fn prefix_ordering_equivalent_class_with_existing_ordering( -// &self, -// existing_ordering: &[PhysicalSortExpr], -// ) -> OrderingEquivalentClass { -// // let existing_ordering = eq_properties.normalize_sort_exprs(existing_ordering); -// // let normalized_head = eq_properties.normalize_sort_exprs(self.head()); -// let normalized_head = self.head(); -// let updated_head = merge_vectors(&existing_ordering, &normalized_head); -// let updated_others = self -// .others() -// .iter() -// .map(|ordering| { -// // let normalized_ordering = eq_properties.normalize_sort_exprs(ordering); -// let normalized_ordering = ordering; -// merge_vectors(&existing_ordering, &normalized_ordering) -// }) -// .collect(); -// OrderingEquivalentClass::new(updated_head, updated_others) -// } -// -// fn get_finer( -// lhs: &[PhysicalSortExpr], -// rhs: &[PhysicalSortExpr], -// ) -> Option> { -// if izip!(lhs.iter(), rhs.iter()).all(|(lhs, rhs)| lhs.eq(rhs)) { -// if lhs.len() > rhs.len() { -// return Some(lhs.to_vec()); -// } else { -// return Some(rhs.to_vec()); -// } -// } -// None -// } -// -// fn add_new_ordering(&mut self, ordering: &[PhysicalSortExpr]) { -// let mut is_redundant = false; -// let mut new_res = vec![]; -// for existing_ordering in self.iter() { -// if let Some(finer) = Self::get_finer(existing_ordering, ordering) { -// // existing_ordering = finer; -// new_res.push(finer); -// is_redundant = true; -// } else { -// new_res.push(existing_ordering.to_vec()); -// } -// } -// if !is_redundant { -// new_res.push(ordering.to_vec()); -// } -// let head = new_res[0].clone(); -// let others = new_res[1..].to_vec(); -// *self = OrderingEquivalentClass::new(head, others); -// } -// } - /// This is a builder object facilitating incremental construction /// for ordering equivalences. pub struct OrderingEquivalenceBuilder { diff --git a/datafusion/physical-plan/src/joins/utils.rs b/datafusion/physical-plan/src/joins/utils.rs index fb92bed5b3ed..3775fc91bcb7 100644 --- a/datafusion/physical-plan/src/joins/utils.rs +++ b/datafusion/physical-plan/src/joins/utils.rs @@ -49,7 +49,7 @@ use datafusion_physical_expr::{ use crate::joins::hash_join_utils::{build_filter_input_order, SortedFilterExpr}; use datafusion_physical_expr::equivalence::{ - add_offset, combine_join_equivalence_properties2, OrderingEquivalentGroup, + combine_join_equivalence_properties2, OrderingEquivalentGroup, }; use datafusion_physical_expr::intervals::ExprIntervalGraph; use datafusion_physical_expr::utils::merge_vectors; @@ -247,19 +247,18 @@ pub fn calculate_join_output_ordering( /// it can thereafter safely be used for ordering equivalence normalization. fn get_updated_right_ordering_equivalent_class( join_type: &JoinType, - right_oeq_class: &OrderingEquivalentGroup, + right_oeq_group: &OrderingEquivalentGroup, left_columns_len: usize, ) -> Result { match join_type { // In these modes, indices of the right schema should be offset by // the left table size. JoinType::Inner | JoinType::Left | JoinType::Full | JoinType::Right => { - let right_oeq_class = add_offset(right_oeq_class, left_columns_len)?; - return Ok(right_oeq_class); + return right_oeq_group.add_offset(left_columns_len) } _ => {} }; - Ok(right_oeq_class.clone()) + Ok(right_oeq_group.clone()) } /// Calculate ordering equivalence properties for the given join operation. From e383e4630d0c1d23df07b47fa20fe5256aab2cca Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 3 Oct 2023 17:37:13 +0300 Subject: [PATCH 023/122] All tests pass, fix bug --- datafusion/physical-expr/src/equivalence.rs | 196 ++++++++---------- .../physical-plan/src/aggregates/mod.rs | 3 +- .../physical-plan/src/repartition/mod.rs | 4 - .../src/windows/bounded_window_agg_exec.rs | 6 - datafusion/physical-plan/src/windows/mod.rs | 1 - datafusion/sqllogictest/test_files/window.slt | 15 +- 6 files changed, 97 insertions(+), 128 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index c2f0972266d0..3409ff298413 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -222,9 +222,7 @@ impl EquivalentGroups { .iter() .map(|sort_req| self.normalize_sort_requirement(sort_req.clone())) .collect::>(); - // TODO: Add deduplication check here after normalization collapse_lex_req(normalized_sort_reqs) - // collapse_vec(normalized_sort_reqs) } /// Similar to the \[`normalize_sort_requirements`] this function normalizes @@ -307,6 +305,20 @@ impl EquivalentGroups { projection_eq_groups.remove_redundant_entries(); projection_eq_groups } + + /// Returns the equivalent group that contains `expr` + /// If none of the groups contains `expr`, returns None. + fn get_equivalent_group( + &self, + expr: &Arc, + ) -> Option>> { + for eq_class in self.iter() { + if physical_exprs_contains(eq_class, expr) { + return Some(eq_class.to_vec()); + } + } + None + } } /// `LexOrdering` stores the lexicographical ordering for a schema. @@ -542,21 +554,8 @@ impl OrderingEquivalenceProperties { PhysicalSortRequirement::to_sort_exprs(req) }) .collect::>>(); - let mut res: Vec = vec![]; - for ordering in normalized_ordering.into_iter() { - let mut is_inside = false; - for item in &mut res { - if let Some(finer) = self.get_finer_ordering(item, &ordering) { - *item = finer.to_vec(); - is_inside = true; - } - } - if !is_inside { - res.push(ordering); - } - } - self.oeq_group = OrderingEquivalentGroup::new(res); - self.oeq_group.remove_redundant_entries(); + // Create new oeq group normalized according to equivalent groups. + self.oeq_group = OrderingEquivalentGroup::new(normalized_ordering); } /// Add physical expression that have constant value to the `self.constants` @@ -570,6 +569,7 @@ impl OrderingEquivalenceProperties { self } + /// Get schema. pub fn schema(&self) -> SchemaRef { self.schema.clone() } @@ -611,8 +611,6 @@ impl OrderingEquivalenceProperties { &self, sort_reqs: &[PhysicalSortRequirement], ) -> Vec { - // println!("sort_reqs: {:?}", sort_reqs); - // println!("self.oeq_class: {:?}", self.oeq_class); let normalized_sort_reqs = self.eq_groups.normalize_sort_requirements(sort_reqs); let normalized_sort_reqs = prune_sort_reqs_with_constants(&normalized_sort_reqs, &self.constants); @@ -631,9 +629,6 @@ impl OrderingEquivalenceProperties { let ranges = get_compatible_ranges(&normalized_sort_reqs, &item); let mut offset: i64 = 0; for Range { start, end } in ranges { - // let head = PhysicalSortRequirement::from_sort_exprs(&self.oeq_class[0]); - // let head = self.normalize_with_eq_classes(&head); - // let mut head = prune_sort_reqs_with_constants(&head, &self.constants); let mut head = prune_sort_reqs_with_constants(&first_entry, &self.constants); let updated_start = (start as i64 + offset) as usize; @@ -653,9 +648,6 @@ impl OrderingEquivalenceProperties { } normalized_sort_reqs = simplify_lex_req(normalized_sort_reqs, &self.oeq_group); - // let res = collapse_lex_req(normalized_sort_reqs); - // // println!("normalzied sort_reqs:{:?}", res); - // res collapse_lex_req(normalized_sort_reqs) } @@ -670,49 +662,34 @@ impl OrderingEquivalenceProperties { .any(|ordering| ordering[0].eq(leading_ordering)) } - fn get_eq_class_group( - old_eq_class: &EquivalentGroups, - expr: &Arc, - ) -> Option>> { - for eq_class in old_eq_class.iter() { - if physical_exprs_contains(eq_class, expr) { - return Some(eq_class.to_vec()); - } - } - None - } - - fn get_corresponding_expr( - old_eq_class: &EquivalentGroups, + /// Projects given expression according to mapping in the `source_to_target_mapping`. + /// If expression is not valid after projection returns `None`. + fn project_expr( + &self, source_to_target_mapping: &ProjectionMapping, expr: &Arc, ) -> Option> { let children = expr.children(); if children.is_empty() { for (source, target) in source_to_target_mapping.iter() { - if source.eq(expr) - // || old_eq_class - // .iter() - // .any(|eq_class| eq_class.iter().any(|item| item.eq(expr))) - { + // if source matches expr, expr can be projected + if source.eq(expr) { return Some(target.clone()); - } else if let Some(group) = Self::get_eq_class_group(old_eq_class, source) - { + } + // if equivalent group of source contains expr, expr can be projected + else if let Some(group) = self.eq_groups.get_equivalent_group(source) { if physical_exprs_contains(&group, expr) { return Some(target.clone()); } } } + // After projection, expression is not valid. None - } else if let Some(children) = children + } + // All of the childrens can be projected + else if let Some(children) = children .into_iter() - .map(|child| { - Self::get_corresponding_expr( - old_eq_class, - source_to_target_mapping, - &child, - ) - }) + .map(|child| self.project_expr(source_to_target_mapping, &child)) .collect::>>() { Some(expr.clone().with_new_children(children).unwrap()) @@ -721,27 +698,28 @@ impl OrderingEquivalenceProperties { } } - fn get_projected_ordering( - old_eq_class: &EquivalentGroups, + /// Projects given ordering according to mapping in the `source_to_target_mapping`. + /// If ordering is not valid after projection returns `None`. + fn project_ordering( + &self, source_to_target_mapping: &ProjectionMapping, ordering: &[PhysicalSortExpr], ) -> Option> { - // println!("old_eq_class: {:?}", old_eq_class); - // println!("ordering: {:?}", ordering); let mut res = vec![]; for order in ordering { - // println!("order.expr:{:?}", order.expr); - if let Some(new_expr) = Self::get_corresponding_expr( - old_eq_class, - source_to_target_mapping, - &order.expr, - ) { - // println!("new_expr:{:?}", new_expr); + if let Some(new_expr) = + self.project_expr(source_to_target_mapping, &order.expr) + { res.push(PhysicalSortExpr { expr: new_expr, options: order.options, }) } else { + // Expression is not valid, rest of the ordering shouldn't be projected also. + // e.g if input ordering is [a ASC, b ASC, c ASC], and column b is not valid + // after projection + // we should return projected ordering as [a ASC] not as [a ASC, c ASC] even if + // column c is valid after projection. break; } } @@ -752,32 +730,27 @@ impl OrderingEquivalenceProperties { } } + /// Projects `OrderingEquivalenceProperties` according to mapping given in `source_to_target_mapping`. pub fn project( &self, source_to_target_mapping: &ProjectionMapping, output_schema: SchemaRef, ) -> OrderingEquivalenceProperties { - // println!("source_to_target_mapping: {:?}", source_to_target_mapping); let mut projected_properties = OrderingEquivalenceProperties::new(output_schema); let projected_eq_groups = self.eq_groups.project(source_to_target_mapping); projected_properties.eq_groups = projected_eq_groups; - // println!("old oeq class: {:?}", oeq_class); - let new_ordering = self + let projected_orderings = self .oeq_group .iter() - .filter_map(|order| { - Self::get_projected_ordering( - &self.eq_groups, - source_to_target_mapping, - order, - ) - }) + .filter_map(|order| self.project_ordering(source_to_target_mapping, order)) .collect::>(); - // println!("new_ordering: {:?}", new_ordering); - if !new_ordering.is_empty() { - projected_properties.oeq_group = OrderingEquivalentGroup::new(new_ordering); + + // if empty, no need to track projected_orderings. + if !projected_orderings.is_empty() { + projected_properties.oeq_group = + OrderingEquivalentGroup::new(projected_orderings); } for (source, target) in source_to_target_mapping { @@ -798,28 +771,28 @@ impl OrderingEquivalenceProperties { } } } - // TODO: Add redundancy check, for ordering equivalences - + // Remove redundant entries from ordering group if any. + // projected_properties.oeq_group.remove_redundant_entries(); projected_properties } + /// Re-creates `OrderingEquivalenceProperties` given that + /// schema is re-ordered by `sort_expr` in the argument. pub fn with_reorder( mut self, sort_expr: Vec, ) -> OrderingEquivalenceProperties { - // TODO: In some cases, existing ordering equivalences may still be valid add this analysis - // Equivalences and constants are still valid after reorder - let sort_expr = sort_expr - .into_iter() - .map(|PhysicalSortExpr { expr, options }| { - let new_expr = self.eq_groups.normalize_expr(expr); - PhysicalSortExpr { - expr: new_expr, - options, - } - }) - .collect::>(); - // TODO: Add deduplicate + // TODO: In some cases, existing ordering equivalences may still be valid add this analysis. + + // Normalize sort_expr according to equivalences + let sort_expr = self.eq_groups.normalize_sort_exprs(&sort_expr); + + // Remove redundant entries from the lex ordering. + let sort_expr = collapse_lex_ordering(sort_expr); + + // Reset ordering equivalent group with the new ordering. + // Constants, and equivalent groups are still valid after re-sort. + // Hence only `oeq_group` is overwritten. self.oeq_group = OrderingEquivalentGroup::new(vec![sort_expr]); self } @@ -870,6 +843,7 @@ impl OrderingEquivalenceProperties { } } + /// Empties the `oeq_group` inside self, When existing orderings are invalidated. pub fn with_empty_ordering_equivalence(mut self) -> OrderingEquivalenceProperties { self.oeq_group = OrderingEquivalentGroup::empty(); self @@ -888,8 +862,6 @@ impl OrderingEquivalenceProperties { pub fn ordering_satisfy_concrete(&self, required: &[PhysicalSortExpr]) -> bool { let required_normalized = self.normalize_sort_exprs(required); let provided_normalized = self.oeq_group().output_ordering().unwrap_or(vec![]); - println!("required_normalized: {:?}", required_normalized); - println!("provided_normalized: {:?}", provided_normalized); if required_normalized.len() > provided_normalized.len() { return false; } @@ -979,8 +951,6 @@ impl OrderingEquivalenceProperties { ) -> bool { let provided_normalized = self.oeq_group().output_ordering().unwrap_or(vec![]); let required_normalized = self.normalize_sort_requirements(required); - println!("req required_normalized: {:?}", required_normalized); - println!("req provided_normalized: {:?}", provided_normalized); if required_normalized.len() > provided_normalized.len() { return false; } @@ -1040,7 +1010,6 @@ impl OrderingEquivalenceProperties { orderby_keys: &[PhysicalSortExpr], ) -> Result> { let partitionby_exprs = self.eq_groups.normalize_exprs(partitionby_exprs); - let partition_by_oeq = self.clone().with_constants(partitionby_exprs.clone()); let mut orderby_keys = self.eq_groups.normalize_sort_exprs(orderby_keys); // Keep the order by expressions that are not inside partition by expressions. orderby_keys.retain(|sort_expr| { @@ -1078,6 +1047,8 @@ impl OrderingEquivalenceProperties { // prefer None. Instead of Linear. return Ok(None); } + // Treat partition by exprs as constant. During analysis of requirements are satisfied. + let partition_by_oeq = self.clone().with_constants(partitionby_exprs.clone()); if partition_by_oeq.ordering_satisfy_requirement_concrete(&req) { // Window can be run with existing ordering return Ok(Some((false, partition_search_mode))); @@ -1134,6 +1105,9 @@ impl OrderingEquivalenceBuilder { mut self, new_ordering_eq_properties: OrderingEquivalenceProperties, ) -> Self { + self.ordering_eq_properties + .eq_groups + .extend(new_ordering_eq_properties.eq_groups); self.ordering_eq_properties .extend(new_ordering_eq_properties.oeq_group); self @@ -1149,11 +1123,6 @@ impl OrderingEquivalenceBuilder { self } - // pub fn with_equivalences(mut self, new_eq_properties: EquivalenceProperties) -> Self { - // self.eq_properties = new_eq_properties; - // self - // } - pub fn add_equal_conditions( &mut self, new_equivalent_ordering: Vec, @@ -1314,6 +1283,23 @@ pub fn collapse_lex_req(input: LexOrderingReq) -> LexOrderingReq { output } +/// This function constructs a duplicate-free `LexOrdering` by filtering out duplicate +/// entries that have same physical expression inside the given vector `input`. +/// `vec![a ASC, a DESC]` is collapsed to the `vec![a ASC]`. Since +/// when same expression is already seen before, following expressions are redundant. +pub fn collapse_lex_ordering(input: LexOrdering) -> LexOrdering { + let mut output = vec![]; + for item in input { + if output + .iter() + .all(|elem: &PhysicalSortExpr| !elem.expr.eq(&item.expr)) + { + output.push(item); + } + } + output +} + /// This function simplifies lexicographical ordering requirement /// inside `input` by removing postfix lexicographical requirements /// that satisfy global ordering (occurs inside the ordering equivalent class) @@ -1543,8 +1529,6 @@ pub fn combine_join_equivalence_properties2( let new_lhs = Arc::new(lhs.clone()) as _; let new_rhs = Arc::new(Column::new(rhs.name(), rhs.index() + left_columns_len)) as _; - // (new_lhs, new_rhs) - // println!("new_lhs: {:?}, new_rhs: {:?}", new_lhs, new_rhs); out_properties.add_equal_conditions((&new_lhs, &new_rhs)); }); } @@ -1987,7 +1971,7 @@ mod tests { }, ], ]); - // All of the orderings [a ASC, [d ASC, f ASC], [e ASC]] + // All of the orderings [a ASC], [d ASC, f ASC], [e ASC]] // are valid for the table // Also Columns a and c are equal diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 314405431820..64614564a052 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -22,7 +22,7 @@ use crate::aggregates::{ }; use crate::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use crate::{ - displayable, DisplayFormatType, Distribution, ExecutionPlan, Partitioning, + DisplayFormatType, Distribution, ExecutionPlan, Partitioning, SendableRecordBatchStream, Statistics, }; @@ -510,7 +510,6 @@ impl AggregateExec { let req = PhysicalSortRequirement::from_sort_exprs(&ordering_req); new_requirement.extend(req); new_requirement = collapse_lex_req(new_requirement); - println!("new_requirement: {:?}", new_requirement); // construct a map from the input columns to the output columns of the Aggregation let mut columns_map: HashMap> = HashMap::new(); diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 270580a7ca86..0e3bfd5db73b 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -455,10 +455,6 @@ impl ExecutionPlan for RepartitionExec { } } - // fn equivalence_properties(&self) -> EquivalenceProperties { - // self.input.equivalence_properties() - // } - fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { if !self.maintains_input_order()[0] { self.input diff --git a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs index 6d3d4f3a6454..f75f3708743b 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -283,10 +283,6 @@ impl ExecutionPlan for BoundedWindowAggExec { } } - // fn equivalence_properties(&self) -> EquivalenceProperties { - // self.input().equivalence_properties() - // } - /// Get the OrderingEquivalenceProperties within the plan fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { window_ordering_equivalence(&self.schema, &self.input, &self.window_expr) @@ -315,8 +311,6 @@ impl ExecutionPlan for BoundedWindowAggExec { context: Arc, ) -> Result { let input = self.input.execute(partition, context)?; - // println!("window self.input.ordering_equivalence_properties(): {:?}", self.input.ordering_equivalence_properties()); - // println!("window self.ordering_equivalence_properties(): {:?}", self.ordering_equivalence_properties()); let search_mode = self.get_search_algo()?; let stream = Box::pin(BoundedWindowAggStream::new( self.schema.clone(), diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index 0ad0b3ccbbdf..7a5bdba51013 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -335,7 +335,6 @@ pub(crate) fn window_ordering_equivalence( // We need to update the schema, so we can not directly use // `input.ordering_equivalence_properties()`. let mut builder = OrderingEquivalenceBuilder::new(schema.clone()) - // .with_equivalences(input.equivalence_properties()) .with_existing_ordering(input.output_ordering().map(|elem| elem.to_vec())) .extend(input.ordering_equivalence_properties()); diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 04ee8c624d21..b23cf391c82b 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -2136,15 +2136,12 @@ ProjectionExec: expr=[c9@1 as c9, SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER B ----BoundedWindowAggExec: wdw=[SUM(t1.c9) PARTITION BY [t1.c2, t1.c1_alias] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(t1.c9) PARTITION BY [t1.c2, t1.c1_alias] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", 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)) }], mode=[Sorted] ------ProjectionExec: expr=[c2@0 as c2, c9@2 as c9, c1_alias@3 as c1_alias, SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING@4 as SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING, SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@5 as SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING, SUM(t1.c9) PARTITION BY [t1.c2, t1.c1_alias] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING@6 as SUM(t1.c9) PARTITION BY [t1.c2, t1.c1_alias] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING] --------WindowAggExec: wdw=[SUM(t1.c9) PARTITION BY [t1.c2, t1.c1_alias] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(t1.c9) PARTITION BY [t1.c2, t1.c1_alias] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING", 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(NULL)) }] -----------SortExec: expr=[c2@0 ASC NULLS LAST,c1_alias@3 ASC NULLS LAST,c9@2 ASC NULLS LAST,c8@1 ASC NULLS LAST] -------------ProjectionExec: expr=[c2@1 as c2, c8@2 as c8, c9@3 as c9, c1_alias@4 as c1_alias, SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING@5 as SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING, SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@6 as SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING] ---------------BoundedWindowAggExec: wdw=[SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", 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)) }], mode=[Sorted] -----------------WindowAggExec: wdw=[SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING", 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(NULL)) }] -------------------SortExec: expr=[c1@0 ASC NULLS LAST,c2@1 ASC NULLS LAST,c9@3 ASC NULLS LAST,c8@2 ASC NULLS LAST] ---------------------ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, c8@2 as c8, c9@3 as c9, c1@0 as c1_alias] -----------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c8, c9], has_header=true - - +----------ProjectionExec: expr=[c2@1 as c2, c8@2 as c8, c9@3 as c9, c1_alias@4 as c1_alias, SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING@5 as SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING, SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING@6 as SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING] +------------BoundedWindowAggExec: wdw=[SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING: Ok(Field { name: "SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND 5 FOLLOWING", 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)) }], mode=[Sorted] +--------------WindowAggExec: wdw=[SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING: Ok(Field { name: "SUM(t1.c9) PARTITION BY [t1.c1, t1.c2] ORDER BY [t1.c9 ASC NULLS LAST, t1.c8 ASC NULLS LAST] ROWS BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING", 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(NULL)) }] +----------------SortExec: expr=[c1@0 ASC NULLS LAST,c2@1 ASC NULLS LAST,c9@3 ASC NULLS LAST,c8@2 ASC NULLS LAST] +------------------ProjectionExec: expr=[c1@0 as c1, c2@1 as c2, c8@2 as c8, c9@3 as c9, c1@0 as c1_alias] +--------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c1, c2, c8, c9], has_header=true query IIIII SELECT c9, From 85890c19f84b0ed0489c7ffda59c64bde6a2ca30 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 3 Oct 2023 17:52:00 +0300 Subject: [PATCH 024/122] Remove unnecessary code --- .../core/src/physical_optimizer/sort_pushdown.rs | 4 ++-- datafusion/physical-plan/src/aggregates/mod.rs | 10 ---------- datafusion/physical-plan/src/coalesce_batches.rs | 4 ---- datafusion/physical-plan/src/coalesce_partitions.rs | 4 ---- datafusion/physical-plan/src/joins/cross_join.rs | 10 ---------- .../physical-plan/src/joins/nested_loop_join.rs | 12 ------------ .../physical-plan/src/joins/sort_merge_join.rs | 12 ------------ .../physical-plan/src/joins/symmetric_hash_join.rs | 12 ------------ datafusion/physical-plan/src/lib.rs | 5 ----- datafusion/physical-plan/src/limit.rs | 8 -------- datafusion/physical-plan/src/sorts/sort.rs | 4 ---- .../physical-plan/src/sorts/sort_preserving_merge.rs | 4 ---- datafusion/physical-plan/src/unnest.rs | 4 ---- .../physical-plan/src/windows/window_agg_exec.rs | 4 ---- 14 files changed, 2 insertions(+), 95 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/sort_pushdown.rs b/datafusion/core/src/physical_optimizer/sort_pushdown.rs index fc7b30da0b56..72be8dad731f 100644 --- a/datafusion/core/src/physical_optimizer/sort_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/sort_pushdown.rs @@ -264,13 +264,13 @@ fn pushdown_requirement_to_children( // && plan.ordering_equivalence_properties().get_finer_ordering() // ) { - println!("parent_required if:{:?}", parent_required); + // println!("parent_required if:{:?}", parent_required); // If the current plan is a leaf node or can not maintain any of the input ordering, can not pushed down requirements. // For RepartitionExec, we always choose to not push down the sort requirements even the RepartitionExec(input_partition=1) could maintain input ordering. // Pushing down is not beneficial Ok(None) } else { - println!("parent_required else:{:?}", parent_required); + // println!("parent_required else:{:?}", parent_required); Ok(Some( maintains_input_order .iter() diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 64614564a052..5979bcae121c 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -811,16 +811,6 @@ impl ExecutionPlan for AggregateExec { vec![self.required_input_ordering.clone()] } - // fn equivalence_properties(&self) -> EquivalenceProperties { - // let mut new_properties = EquivalenceProperties::new(self.schema()); - // project_equivalence_properties( - // self.input.equivalence_properties(), - // &self.columns_map, - // &mut new_properties, - // ); - // new_properties - // } - fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { self.input .ordering_equivalence_properties() diff --git a/datafusion/physical-plan/src/coalesce_batches.rs b/datafusion/physical-plan/src/coalesce_batches.rs index a6f8cd8be86b..a49de9eb5f02 100644 --- a/datafusion/physical-plan/src/coalesce_batches.rs +++ b/datafusion/physical-plan/src/coalesce_batches.rs @@ -136,10 +136,6 @@ impl ExecutionPlan for CoalesceBatchesExec { vec![false] } - // fn equivalence_properties(&self) -> EquivalenceProperties { - // self.input.equivalence_properties() - // } - fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { self.input.ordering_equivalence_properties() } diff --git a/datafusion/physical-plan/src/coalesce_partitions.rs b/datafusion/physical-plan/src/coalesce_partitions.rs index a01451ec0d3c..846b8a2f258f 100644 --- a/datafusion/physical-plan/src/coalesce_partitions.rs +++ b/datafusion/physical-plan/src/coalesce_partitions.rs @@ -100,10 +100,6 @@ impl ExecutionPlan for CoalescePartitionsExec { None } - // fn equivalence_properties(&self) -> EquivalenceProperties { - // self.input.equivalence_properties() - // } - fn benefits_from_input_partitioning(&self) -> Vec { vec![false] } diff --git a/datafusion/physical-plan/src/joins/cross_join.rs b/datafusion/physical-plan/src/joins/cross_join.rs index 3a7e8f0c2b41..8b541a7bc107 100644 --- a/datafusion/physical-plan/src/joins/cross_join.rs +++ b/datafusion/physical-plan/src/joins/cross_join.rs @@ -215,16 +215,6 @@ impl ExecutionPlan for CrossJoinExec { None } - // fn equivalence_properties(&self) -> EquivalenceProperties { - // let left_columns_len = self.left.schema().fields.len(); - // cross_join_equivalence_properties( - // self.left.equivalence_properties(), - // self.right.equivalence_properties(), - // left_columns_len, - // self.schema(), - // ) - // } - fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { combine_join_ordering_equivalence_properties( &JoinType::Full, diff --git a/datafusion/physical-plan/src/joins/nested_loop_join.rs b/datafusion/physical-plan/src/joins/nested_loop_join.rs index 00a27a771e81..1d76b429821a 100644 --- a/datafusion/physical-plan/src/joins/nested_loop_join.rs +++ b/datafusion/physical-plan/src/joins/nested_loop_join.rs @@ -190,18 +190,6 @@ impl ExecutionPlan for NestedLoopJoinExec { distribution_from_join_type(&self.join_type) } - // fn equivalence_properties(&self) -> EquivalenceProperties { - // let left_columns_len = self.left.schema().fields.len(); - // combine_join_equivalence_properties( - // self.join_type, - // self.left.equivalence_properties(), - // self.right.equivalence_properties(), - // left_columns_len, - // &[], // empty join keys - // self.schema(), - // ) - // } - fn children(&self) -> Vec> { vec![self.left.clone(), self.right.clone()] } diff --git a/datafusion/physical-plan/src/joins/sort_merge_join.rs b/datafusion/physical-plan/src/joins/sort_merge_join.rs index 9cf035a2d167..cb02a6a2702b 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join.rs @@ -282,18 +282,6 @@ impl ExecutionPlan for SortMergeJoinExec { Self::maintains_input_order(self.join_type) } - // fn equivalence_properties(&self) -> EquivalenceProperties { - // let left_columns_len = self.left.schema().fields.len(); - // combine_join_equivalence_properties( - // self.join_type, - // self.left.equivalence_properties(), - // self.right.equivalence_properties(), - // left_columns_len, - // self.on(), - // self.schema(), - // ) - // } - fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { combine_join_ordering_equivalence_properties( &self.join_type, diff --git a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs index 58dd26fc439a..23addd479c92 100644 --- a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs +++ b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs @@ -432,18 +432,6 @@ impl ExecutionPlan for SymmetricHashJoinExec { None } - // fn equivalence_properties(&self) -> EquivalenceProperties { - // let left_columns_len = self.left.schema().fields.len(); - // combine_join_equivalence_properties( - // self.join_type, - // self.left.equivalence_properties(), - // self.right.equivalence_properties(), - // left_columns_len, - // self.on(), - // self.schema(), - // ) - // } - fn children(&self) -> Vec> { vec![self.left.clone(), self.right.clone()] } diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs index 92c7b8761a00..d07ad839a0f6 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -147,11 +147,6 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { .collect() } - // /// Get the EquivalenceProperties within the plan - // fn equivalence_properties(&self) -> EquivalenceProperties { - // EquivalenceProperties::new(self.schema()) - // } - /// Get the OrderingEquivalenceProperties within the plan fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { OrderingEquivalenceProperties::new(self.schema()) diff --git a/datafusion/physical-plan/src/limit.rs b/datafusion/physical-plan/src/limit.rs index 64177b4c6541..b2d98097caf7 100644 --- a/datafusion/physical-plan/src/limit.rs +++ b/datafusion/physical-plan/src/limit.rs @@ -133,10 +133,6 @@ impl ExecutionPlan for GlobalLimitExec { self.input.output_ordering() } - // fn equivalence_properties(&self) -> EquivalenceProperties { - // self.input.equivalence_properties() - // } - fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { self.input.ordering_equivalence_properties() } @@ -314,10 +310,6 @@ impl ExecutionPlan for LocalLimitExec { vec![true] } - // fn equivalence_properties(&self) -> EquivalenceProperties { - // self.input.equivalence_properties() - // } - fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { self.input.ordering_equivalence_properties() } diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index ab929649db1f..46b209f2f27b 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -825,10 +825,6 @@ impl ExecutionPlan for SortExec { Some(&self.expr) } - // fn equivalence_properties(&self) -> EquivalenceProperties { - // self.input.equivalence_properties() - // } - fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { let input_oeq = self.input.ordering_equivalence_properties(); // println!("sort input: {:?}", input_oeq); diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index 6a48a9454200..10b95de413e0 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -170,10 +170,6 @@ impl ExecutionPlan for SortPreservingMergeExec { vec![true] } - // fn equivalence_properties(&self) -> EquivalenceProperties { - // self.input.equivalence_properties() - // } - fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { self.input.ordering_equivalence_properties() } diff --git a/datafusion/physical-plan/src/unnest.rs b/datafusion/physical-plan/src/unnest.rs index 112f6a50c426..ebb337cd98ef 100644 --- a/datafusion/physical-plan/src/unnest.rs +++ b/datafusion/physical-plan/src/unnest.rs @@ -136,10 +136,6 @@ impl ExecutionPlan for UnnestExec { None } - // fn equivalence_properties(&self) -> EquivalenceProperties { - // self.input.equivalence_properties() - // } - fn execute( &self, partition: usize, diff --git a/datafusion/physical-plan/src/windows/window_agg_exec.rs b/datafusion/physical-plan/src/windows/window_agg_exec.rs index 7b0fe245fcfb..492166e58c97 100644 --- a/datafusion/physical-plan/src/windows/window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/window_agg_exec.rs @@ -214,10 +214,6 @@ impl ExecutionPlan for WindowAggExec { } } - // fn equivalence_properties(&self) -> EquivalenceProperties { - // self.input().equivalence_properties() - // } - /// Get the OrderingEquivalenceProperties within the plan fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { window_ordering_equivalence(&self.schema, &self.input, &self.window_expr) From 9c703bd0ac92153f508cb236be63d704f245c12d Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 3 Oct 2023 18:16:51 +0300 Subject: [PATCH 025/122] Simplifications --- datafusion/physical-expr/src/equivalence.rs | 217 +++++++++----------- 1 file changed, 95 insertions(+), 122 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 3409ff298413..3c9976e152ed 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -44,10 +44,13 @@ pub struct EquivalentGroups { } impl EquivalentGroups { + /// Creates an empty ordering equivalent group fn empty() -> Self { EquivalentGroups { inner: vec![] } } + /// Creates ordering equivalent groups from given vectors + /// Each vector corresponds to a group fn new(entries: Vec>>) -> Self { let mut res = EquivalentGroups { inner: entries }; res.remove_redundant_entries(); @@ -100,12 +103,16 @@ impl EquivalentGroups { } } (Some(group_idx), None) => { + // Extend existing group with new entry self.inner[group_idx].push(second.clone()); } (None, Some(group_idx)) => { + // Extend existing group with new entry self.inner[group_idx].push(first.clone()); } (None, None) => { + // None of the expressions, is among existing groups + // Create a new group. self.inner.push(vec![first.clone(), second.clone()]); } } @@ -124,11 +131,11 @@ impl EquivalentGroups { (unique_eq_group.len() > 1).then_some(unique_eq_group) }) .collect(); - + // Bridge groups that have common expressions self.bridge_groups() } - /// This utils bridges groups that have common entry + /// This utils bridges groups that have common expressions fn bridge_groups(&mut self) { let mut out_groups = vec![]; for group in &self.inner { @@ -154,7 +161,6 @@ impl EquivalentGroups { } fn extend(&mut self, other: EquivalentGroups) { - // TODO: Add check for redundancy self.inner.extend(other.inner); self.remove_redundant_entries(); } @@ -181,12 +187,10 @@ impl EquivalentGroups { &self, exprs: &[Arc], ) -> Vec> { - let res = exprs + exprs .iter() .map(|expr| self.normalize_expr(expr.clone())) - .collect::>(); - // TODO: Add deduplication check here after normalization - res + .collect::>() } /// This function normalizes `sort_requirement` according to `EquivalenceClasses` in the `self`. @@ -241,24 +245,78 @@ impl EquivalentGroups { PhysicalSortRequirement::to_sort_exprs(normalized_sort_requirement) } - /// Calculate updated version of the expression, according to projection mapping - /// returns `None`, if expression is not valid after projection. - fn get_aliased_expr( + /// Projects given expression according to mapping in the `source_to_target_mapping`. + /// If expression is not valid after projection returns `None`. + fn project_expr( + &self, source_to_target_mapping: &ProjectionMapping, expr: &Arc, ) -> Option> { - for (source, target) in source_to_target_mapping { - if expr.eq(source) { - return Some(target.clone()); + let children = expr.children(); + if children.is_empty() { + for (source, target) in source_to_target_mapping.iter() { + // if source matches expr, expr can be projected + if source.eq(expr) { + return Some(target.clone()); + } + // if equivalent group of source contains expr, expr can be projected + else if let Some(group) = self.get_equivalent_group(source) { + if physical_exprs_contains(&group, expr) { + return Some(target.clone()); + } + } } + // After projection, expression is not valid. + None + } + // All of the childrens can be projected + else if let Some(children) = children + .into_iter() + .map(|child| self.project_expr(source_to_target_mapping, &child)) + .collect::>>() + { + Some(expr.clone().with_new_children(children).unwrap()) + } else { + None + } + } + + /// Projects given ordering according to mapping in the `source_to_target_mapping`. + /// If ordering is not valid after projection returns `None`. + fn project_ordering( + &self, + source_to_target_mapping: &ProjectionMapping, + ordering: &[PhysicalSortExpr], + ) -> Option> { + let mut res = vec![]; + for order in ordering { + if let Some(new_expr) = + self.project_expr(source_to_target_mapping, &order.expr) + { + res.push(PhysicalSortExpr { + expr: new_expr, + options: order.options, + }) + } else { + // Expression is not valid, rest of the ordering shouldn't be projected also. + // e.g if input ordering is [a ASC, b ASC, c ASC], and column b is not valid + // after projection + // we should return projected ordering as [a ASC] not as [a ASC, c ASC] even if + // column c is valid after projection. + break; + } + } + if res.is_empty() { + None + } else { + Some(res) } - None } - /// Construct equivalent groups according to projection mapping - /// Each inner vector contains equivalents sets. Outer vector corresponds to + /// Construct equivalent groups according to projection mapping. + /// In the result, each inner vector contains equivalents sets. Outer vector corresponds to /// distinct equivalent groups - fn get_equivalent_groups( + fn calculate_new_projection_equivalent_groups( source_to_target_mapping: &ProjectionMapping, ) -> Vec>> { // TODO: Convert below algorithm to the version that use HashMap. @@ -291,13 +349,14 @@ impl EquivalentGroups { for eq_class in self.iter() { let new_eq_class = eq_class .iter() - .filter_map(|expr| Self::get_aliased_expr(source_to_target_mapping, expr)) + .filter_map(|expr| self.project_expr(source_to_target_mapping, expr)) .collect::>(); if new_eq_class.len() > 1 { new_eq_classes.push(new_eq_class.clone()); } } - let new_classes = Self::get_equivalent_groups(source_to_target_mapping); + let new_classes = + Self::calculate_new_projection_equivalent_groups(source_to_target_mapping); new_eq_classes.extend(new_classes); let mut projection_eq_groups = EquivalentGroups::new(new_eq_classes); @@ -338,16 +397,20 @@ pub struct OrderingEquivalentGroup { } impl OrderingEquivalentGroup { + /// Creates new empty ordering equivalent group fn empty() -> Self { OrderingEquivalentGroup { inner: vec![] } } + /// Creates new ordering equivalent from given vector pub fn new(entries: Vec) -> Self { let mut res = OrderingEquivalentGroup { inner: entries }; + // Make sure ordering equivalences doesn't contain something redundant res.remove_redundant_entries(); res } + /// Check whether ordering is in the state. pub fn contains(&self, other: &LexOrdering) -> bool { self.inner.contains(other) } @@ -356,16 +419,15 @@ impl OrderingEquivalentGroup { if !self.contains(&other) { self.inner.push(other); } - // TODO: remove below check + // Make sure that after new entry there is no redundant + // entry in the state. self.remove_redundant_entries(); } /// Adds new ordering into the ordering equivalent group. pub fn add_new_orderings(&mut self, orderings: &[LexOrdering]) { for ordering in orderings.iter() { - if !self.contains(ordering) { - self.push(ordering.clone()); - } + self.push(ordering.clone()); } self.remove_redundant_entries(); } @@ -392,6 +454,7 @@ impl OrderingEquivalentGroup { self.inner = res; } + /// Check whether ordering equivalent group is empty pub fn is_empty(&self) -> bool { self.inner.is_empty() } @@ -409,6 +472,7 @@ impl OrderingEquivalentGroup { self.inner.len() } + /// Extend ordering equivalent group with other group pub fn extend(&mut self, other: OrderingEquivalentGroup) { for ordering in other.iter() { if !self.contains(ordering) { @@ -418,6 +482,8 @@ impl OrderingEquivalentGroup { self.remove_redundant_entries(); } + /// Get first ordering entry in the ordering equivalences + /// This is one of the many valid orderings (if available) pub fn output_ordering(&self) -> Option> { self.inner.first().cloned() } @@ -484,7 +550,9 @@ impl OrderingEquivalentGroup { /// and treat `a ASC` and `b DESC` as the same ordering requirement. #[derive(Debug, Clone)] pub struct OrderingEquivalenceProperties { + /// Keeps track of expressions that have equivalent value. eq_groups: EquivalentGroups, + /// Keeps track of valid ordering that satisfied table. oeq_group: OrderingEquivalentGroup, /// Keeps track of expressions that have constant value. constants: Vec>, @@ -662,74 +730,6 @@ impl OrderingEquivalenceProperties { .any(|ordering| ordering[0].eq(leading_ordering)) } - /// Projects given expression according to mapping in the `source_to_target_mapping`. - /// If expression is not valid after projection returns `None`. - fn project_expr( - &self, - source_to_target_mapping: &ProjectionMapping, - expr: &Arc, - ) -> Option> { - let children = expr.children(); - if children.is_empty() { - for (source, target) in source_to_target_mapping.iter() { - // if source matches expr, expr can be projected - if source.eq(expr) { - return Some(target.clone()); - } - // if equivalent group of source contains expr, expr can be projected - else if let Some(group) = self.eq_groups.get_equivalent_group(source) { - if physical_exprs_contains(&group, expr) { - return Some(target.clone()); - } - } - } - // After projection, expression is not valid. - None - } - // All of the childrens can be projected - else if let Some(children) = children - .into_iter() - .map(|child| self.project_expr(source_to_target_mapping, &child)) - .collect::>>() - { - Some(expr.clone().with_new_children(children).unwrap()) - } else { - None - } - } - - /// Projects given ordering according to mapping in the `source_to_target_mapping`. - /// If ordering is not valid after projection returns `None`. - fn project_ordering( - &self, - source_to_target_mapping: &ProjectionMapping, - ordering: &[PhysicalSortExpr], - ) -> Option> { - let mut res = vec![]; - for order in ordering { - if let Some(new_expr) = - self.project_expr(source_to_target_mapping, &order.expr) - { - res.push(PhysicalSortExpr { - expr: new_expr, - options: order.options, - }) - } else { - // Expression is not valid, rest of the ordering shouldn't be projected also. - // e.g if input ordering is [a ASC, b ASC, c ASC], and column b is not valid - // after projection - // we should return projected ordering as [a ASC] not as [a ASC, c ASC] even if - // column c is valid after projection. - break; - } - } - if res.is_empty() { - None - } else { - Some(res) - } - } - /// Projects `OrderingEquivalenceProperties` according to mapping given in `source_to_target_mapping`. pub fn project( &self, @@ -744,7 +744,10 @@ impl OrderingEquivalenceProperties { let projected_orderings = self .oeq_group .iter() - .filter_map(|order| self.project_ordering(source_to_target_mapping, order)) + .filter_map(|order| { + self.eq_groups + .project_ordering(source_to_target_mapping, order) + }) .collect::>(); // if empty, no need to track projected_orderings. @@ -1158,36 +1161,6 @@ impl OrderingEquivalenceBuilder { } } -// /// This function applies the given projection to the given ordering -// /// equivalence properties to compute the resulting (projected) ordering -// /// equivalence properties; e.g. -// /// 1) Adding an alias, which can introduce additional ordering equivalence -// /// properties, as in Projection(a, a as a1, a as a2) extends global ordering -// /// of a to a1 and a2. -// /// 2) Truncate the [`OrderingEquivalentClass`]es that are not in the output schema. -// pub fn project_ordering_equivalence_properties( -// input_eq: OrderingEquivalenceProperties, -// columns_map: &HashMap>, -// output_eq: &mut OrderingEquivalenceProperties, -// ) { -// // Get schema and fields of projection output -// let schema = output_eq.schema(); -// let fields = schema.fields(); -// -// let mut oeq_class = input_eq.oeq_class().clone(); -// let mut oeq_alias_map = vec![]; -// for (column, columns) in columns_map { -// if is_column_invalid_in_new_schema(column, fields) { -// oeq_alias_map.push((column.clone(), columns[0].clone())); -// } -// } -// oeq_class = update_with_aliases(&oeq_class, &oeq_alias_map, fields); -// -// if oeq_class.len() > 1 { -// output_eq.extend(oeq_class); -// } -// } - /// Update `ordering` if it contains cast expression with target column /// after projection, if there is no cast expression among `ordering` expressions, /// returns `None`. From 93b7dc2f033bc1dd7007676374e5dc9451ee8a3d Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 3 Oct 2023 18:19:43 +0300 Subject: [PATCH 026/122] Minor changes --- datafusion/physical-expr/src/equivalence.rs | 3 +++ 1 file changed, 3 insertions(+) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 3c9976e152ed..cabd8e64bd49 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -608,6 +608,8 @@ impl OrderingEquivalenceProperties { } /// Normalizes state according to equivalent classes + /// This util makes sure that all of the entries that have have equivalent groups among the ordering equivalent group + /// uses representative expression of corresponding equivalent group (first entry in the group). fn normalize_state(&mut self) { let normalized_ordering = self .oeq_group @@ -624,6 +626,7 @@ impl OrderingEquivalenceProperties { .collect::>>(); // Create new oeq group normalized according to equivalent groups. self.oeq_group = OrderingEquivalentGroup::new(normalized_ordering); + // TODO: Add normalization with constant also. } /// Add physical expression that have constant value to the `self.constants` From 79b72b9b042a0ae411fa5ede8039c181325f9b28 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 4 Oct 2023 09:40:30 +0300 Subject: [PATCH 027/122] Simplifications --- .../src/physical_optimizer/sort_pushdown.rs | 4 -- datafusion/physical-expr/src/equivalence.rs | 45 ++++++++----------- .../physical-plan/src/aggregates/mod.rs | 1 - .../physical-plan/src/joins/hash_join.rs | 9 ---- datafusion/physical-plan/src/windows/mod.rs | 7 +-- 5 files changed, 21 insertions(+), 45 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/sort_pushdown.rs b/datafusion/core/src/physical_optimizer/sort_pushdown.rs index 72be8dad731f..1fb5078931df 100644 --- a/datafusion/core/src/physical_optimizer/sort_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/sort_pushdown.rs @@ -340,10 +340,6 @@ fn try_pushdown_requirements_to_join( let finer_req = smj .ordering_equivalence_properties() .get_finer_requirement(&new_output_ordering, parent_required.unwrap_or(&[])); - println!( - "finer_req:{:?}, new_output_ordering:{:?}, parent_required:{:?}", - finer_req, new_output_ordering, parent_required - ); let mut should_pushdown = false; if let Some(finer_req) = finer_req { should_pushdown = finer_req.eq(&new_output_ordering); diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index cabd8e64bd49..b5353abea4bd 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -23,7 +23,6 @@ use crate::{ }; use arrow::datatypes::SchemaRef; -use arrow_schema::SortOptions; use crate::physical_expr::{deduplicate_physical_exprs, have_common_entries}; use crate::sort_properties::{ExprOrdering, SortProperties}; @@ -76,6 +75,9 @@ impl EquivalentGroups { self.inner.into_iter() } + /// Adds tuple argument to the equivalent groups + /// It is known that first and second entry in the tuple will have same values in the table. + /// This can arise after filter(a=b), alias(a, a as b), etc. pub fn add_equal_conditions( &mut self, new_conditions: (&Arc, &Arc), @@ -160,6 +162,7 @@ impl EquivalentGroups { self.inner = out_groups; } + /// Extend equivalent group with other equivalent groups fn extend(&mut self, other: EquivalentGroups) { self.inner.extend(other.inner); self.remove_redundant_entries(); @@ -320,6 +323,7 @@ impl EquivalentGroups { source_to_target_mapping: &ProjectionMapping, ) -> Vec>> { // TODO: Convert below algorithm to the version that use HashMap. + // once `Arc` can be stored in `HashMap`. let mut res = vec![]; for (source, target) in source_to_target_mapping { if res.is_empty() { @@ -415,6 +419,7 @@ impl OrderingEquivalentGroup { self.inner.contains(other) } + /// Pushes new ordering to the state. fn push(&mut self, other: LexOrdering) { if !self.contains(&other) { self.inner.push(other); @@ -456,7 +461,7 @@ impl OrderingEquivalentGroup { /// Check whether ordering equivalent group is empty pub fn is_empty(&self) -> bool { - self.inner.is_empty() + self.len() == 0 } pub fn iter(&self) -> impl Iterator { @@ -467,7 +472,7 @@ impl OrderingEquivalentGroup { self.inner.into_iter() } - #[allow(dead_code)] + /// Get length of the entries in the ordering equivalent group fn len(&self) -> usize { self.inner.len() } @@ -555,6 +560,8 @@ pub struct OrderingEquivalenceProperties { /// Keeps track of valid ordering that satisfied table. oeq_group: OrderingEquivalentGroup, /// Keeps track of expressions that have constant value. + /// TODO: We do not need to track constants separately, they can be tracked + /// inside `eq_groups` as `Literal` expressions. constants: Vec>, schema: SchemaRef, } @@ -616,7 +623,9 @@ impl OrderingEquivalenceProperties { .inner .iter() .map(|ordering| { + // Use a representative version of the each equivalent group inside ordering expressions. let ordering = self.eq_groups.normalize_sort_exprs(ordering); + // Prune with constants let req = prune_sort_reqs_with_constants( &PhysicalSortRequirement::from_sort_exprs(&ordering), &self.constants, @@ -626,7 +635,6 @@ impl OrderingEquivalenceProperties { .collect::>>(); // Create new oeq group normalized according to equivalent groups. self.oeq_group = OrderingEquivalentGroup::new(normalized_ordering); - // TODO: Add normalization with constant also. } /// Add physical expression that have constant value to the `self.constants` @@ -683,6 +691,7 @@ impl OrderingEquivalenceProperties { sort_reqs: &[PhysicalSortRequirement], ) -> Vec { let normalized_sort_reqs = self.eq_groups.normalize_sort_requirements(sort_reqs); + // Remove entries that are known to be constant from requirement expression. let normalized_sort_reqs = prune_sort_reqs_with_constants(&normalized_sort_reqs, &self.constants); let mut normalized_sort_reqs = collapse_lex_req(normalized_sort_reqs); @@ -692,16 +701,13 @@ impl OrderingEquivalenceProperties { let first_entry = PhysicalSortRequirement::from_sort_exprs(&self.oeq_group.inner[0]); let first_entry = self.eq_groups.normalize_sort_requirements(&first_entry); - let first_entry = prune_sort_reqs_with_constants(&first_entry, &self.constants); for item in self.oeq_group.iter() { let item = PhysicalSortRequirement::from_sort_exprs(item); let item = self.eq_groups.normalize_sort_requirements(&item); - let item = prune_sort_reqs_with_constants(&item, &self.constants); let ranges = get_compatible_ranges(&normalized_sort_reqs, &item); let mut offset: i64 = 0; for Range { start, end } in ranges { - let mut head = - prune_sort_reqs_with_constants(&first_entry, &self.constants); + let mut head = first_entry.clone(); let updated_start = (start as i64 + offset) as usize; let updated_end = (end as i64 + offset) as usize; let range = end - start; @@ -803,10 +809,9 @@ impl OrderingEquivalenceProperties { self } - pub fn set_satisfy( - &self, - exprs: &[Arc], - ) -> Option> { + /// Check whether any permutation of the argument has a prefix with existing ordering. + /// Return indices that describes ordering and their ordering information. + pub fn set_satisfy(&self, exprs: &[Arc]) -> Option> { let exprs_normalized = self.eq_groups.normalize_exprs(exprs); let mut best = vec![]; @@ -830,15 +835,7 @@ impl OrderingEquivalenceProperties { // these indices, we would match existing ordering. For the example above, // this would produce 1, 0; meaning 1st and 0th entries (a, b) among the // GROUP BY expressions b, a, d match input ordering. - let indices = - get_indices_of_exprs_strict(&ordered_exprs, &exprs_normalized); - best = indices - .iter() - .enumerate() - .map(|(order_idx, &match_idx)| { - (match_idx, ordering[order_idx].options) - }) - .collect(); + best = get_indices_of_exprs_strict(&ordered_exprs, &exprs_normalized) } } @@ -1025,11 +1022,7 @@ impl OrderingEquivalenceProperties { let mut partition_by_reqs: Vec = vec![]; if partitionby_exprs.is_empty() { partition_search_mode = PartitionSearchMode::Sorted; - } else if let Some(indices_and_ordering) = self.set_satisfy(&partitionby_exprs) { - let indices = indices_and_ordering - .iter() - .map(|(idx, _options)| *idx) - .collect::>(); + } else if let Some(indices) = self.set_satisfy(&partitionby_exprs) { let elem = indices .iter() .map(|&idx| PhysicalSortRequirement { diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 5979bcae121c..aa373bf18675 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -336,7 +336,6 @@ fn get_finest_requirement OrderingEquivalenceProperties>( }; if let Some(finest_req) = &mut finest_req { - println!("finest_req{:?}, fn_req:{:?}", finest_req, fn_req); if let Some(finer) = oeq_properties.get_finer_ordering(finest_req, fn_req) { *finest_req = finer.to_vec(); continue; diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index a5960e6e6635..860d29e3e86a 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -406,15 +406,6 @@ impl ExecutionPlan for HashJoinExec { let left_partitions = self.left.output_partitioning().partition_count(); let right_partitions = self.right.output_partitioning().partition_count(); - // for child in self.children(){ - // println!("hash join child ordering_equivalence_properties()\n{:?}", child.ordering_equivalence_properties()); - // } - println!( - "hash join self.ordering_equivalence_properties()\n{:?}", - self.ordering_equivalence_properties() - ); - println!("hash join output ordering: {:?}", self.output_ordering()); - if self.mode == PartitionMode::Partitioned && left_partitions != right_partitions { return internal_err!( diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index 7a5bdba51013..b8b2b443aefe 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -314,14 +314,11 @@ pub(crate) fn get_ordered_partition_by_indices( partition_by_exprs: &[Arc], input: &Arc, ) -> Vec { - if let Some(indices_and_orderings) = input + if let Some(indices) = input .ordering_equivalence_properties() .set_satisfy(partition_by_exprs) { - indices_and_orderings - .into_iter() - .map(|(idx, _options)| idx) - .collect() + indices } else { vec![] } From c70ac05dd87bf40e632a030f46fc700431791a70 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 4 Oct 2023 09:48:08 +0300 Subject: [PATCH 028/122] Move oeq join to methods --- datafusion/common/src/join_type.rs | 28 ++ datafusion/common/src/lib.rs | 2 +- .../src/physical_optimizer/sort_pushdown.rs | 4 +- datafusion/physical-expr/src/equivalence.rs | 294 ++++++++++++++---- .../physical-plan/src/joins/cross_join.rs | 8 +- .../physical-plan/src/joins/hash_join.rs | 24 +- .../src/joins/hash_join_utils.rs | 6 +- .../src/joins/nested_loop_join.rs | 5 +- .../src/joins/sort_merge_join.rs | 12 +- .../src/joins/symmetric_hash_join.rs | 3 +- datafusion/physical-plan/src/joins/utils.rs | 244 +-------------- .../proto/src/physical_plan/from_proto.rs | 3 +- .../proto/src/physical_plan/to_proto.rs | 3 +- 13 files changed, 299 insertions(+), 337 deletions(-) diff --git a/datafusion/common/src/join_type.rs b/datafusion/common/src/join_type.rs index 8d4657f1dc56..c62dda66dc22 100644 --- a/datafusion/common/src/join_type.rs +++ b/datafusion/common/src/join_type.rs @@ -95,3 +95,31 @@ pub enum JoinConstraint { /// Join USING Using, } + +impl Display for JoinSide { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + match self { + JoinSide::Left => write!(f, "left"), + JoinSide::Right => write!(f, "right"), + } + } +} + +/// Used in ColumnIndex to distinguish which side the index is for +#[derive(Debug, Clone, Copy, PartialEq)] +pub enum JoinSide { + /// Left side of the join + Left, + /// Right side of the join + Right, +} + +impl JoinSide { + /// Inverse the join side + pub fn negate(&self) -> Self { + match self { + JoinSide::Left => JoinSide::Right, + JoinSide::Right => JoinSide::Left, + } + } +} diff --git a/datafusion/common/src/lib.rs b/datafusion/common/src/lib.rs index 71782f67046d..d65004f309e1 100644 --- a/datafusion/common/src/lib.rs +++ b/datafusion/common/src/lib.rs @@ -55,7 +55,7 @@ pub use functional_dependencies::{ aggregate_functional_dependencies, get_target_functional_dependencies, Constraints, Dependency, FunctionalDependence, FunctionalDependencies, }; -pub use join_type::{JoinConstraint, JoinType}; +pub use join_type::{JoinConstraint, JoinSide, JoinType}; pub use scalar::{ScalarType, ScalarValue}; pub use schema_reference::{OwnedSchemaReference, SchemaReference}; pub use stats::{ColumnStatistics, Statistics}; diff --git a/datafusion/core/src/physical_optimizer/sort_pushdown.rs b/datafusion/core/src/physical_optimizer/sort_pushdown.rs index 1fb5078931df..0d8faa570b2e 100644 --- a/datafusion/core/src/physical_optimizer/sort_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/sort_pushdown.rs @@ -21,7 +21,7 @@ use crate::physical_optimizer::utils::{ add_sort_above, is_limit, is_sort_preserving_merge, is_union, is_window, }; use crate::physical_plan::filter::FilterExec; -use crate::physical_plan::joins::utils::{calculate_join_output_ordering, JoinSide}; +use crate::physical_plan::joins::utils::calculate_join_output_ordering; use crate::physical_plan::joins::{HashJoinExec, SortMergeJoinExec}; use crate::physical_plan::projection::ProjectionExec; use crate::physical_plan::repartition::RepartitionExec; @@ -29,7 +29,7 @@ use crate::physical_plan::sorts::sort::SortExec; use crate::physical_plan::{with_new_children_if_necessary, ExecutionPlan}; use datafusion_common::tree_node::{Transformed, TreeNode, VisitRecursion}; -use datafusion_common::{plan_err, DataFusionError, Result}; +use datafusion_common::{plan_err, DataFusionError, JoinSide, Result}; use datafusion_expr::JoinType; use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::{PhysicalSortExpr, PhysicalSortRequirement}; diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index b5353abea4bd..6f25b3e5dc7a 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -28,7 +28,7 @@ use crate::physical_expr::{deduplicate_physical_exprs, have_common_entries}; use crate::sort_properties::{ExprOrdering, SortProperties}; use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::utils::longest_consecutive_prefix; -use datafusion_common::{JoinType, Result}; +use datafusion_common::{DataFusionError, JoinSide, JoinType, Result}; use itertools::izip; use std::hash::Hash; use std::ops::Range; @@ -382,6 +382,47 @@ impl EquivalentGroups { } None } + + /// Combine EquivalentGroups of the given join children. + pub fn join( + &self, + join_type: &JoinType, + right_eq_classes: &EquivalentGroups, + left_columns_len: usize, + on: &[(Column, Column)], + ) -> Result { + let mut res = EquivalentGroups::empty(); + match join_type { + JoinType::Inner | JoinType::Left | JoinType::Full | JoinType::Right => { + res.extend(self.clone()); + let updated_eq_classes = right_eq_classes + .iter() + .map(|eq_class| { + add_offset_to_exprs(eq_class.to_vec(), left_columns_len) + }) + .collect::>>()?; + + res.extend(EquivalentGroups::new(updated_eq_classes)); + } + JoinType::LeftSemi | JoinType::LeftAnti => { + res.extend(self.clone()); + } + JoinType::RightSemi | JoinType::RightAnti => { + res.extend(right_eq_classes.clone()); + } + } + // In the inner join, expressions in the on are equal at the resulting table. + if *join_type == JoinType::Inner { + on.iter().for_each(|(lhs, rhs)| { + let new_lhs = Arc::new(lhs.clone()) as _; + let new_rhs = + Arc::new(Column::new(rhs.name(), rhs.index() + left_columns_len)) + as _; + res.add_equal_conditions((&new_lhs, &new_rhs)); + }); + } + Ok(res) + } } /// `LexOrdering` stores the lexicographical ordering for a schema. @@ -604,6 +645,12 @@ impl OrderingEquivalenceProperties { self.normalize_state(); } + /// Add new equivalent group to state. + pub fn add_equivalent_groups(&mut self, other_eq_group: EquivalentGroups) { + self.eq_groups.extend(other_eq_group); + self.normalize_state(); + } + /// Adds new equality group into the equivalent groups. /// If equalities are new, otherwise extends corresponding group. pub fn add_equal_conditions( @@ -1062,6 +1109,87 @@ impl OrderingEquivalenceProperties { } Ok(None) } + + /// Calculate ordering equivalence properties for the given join operation. + pub fn join( + &self, + join_type: &JoinType, + right: &OrderingEquivalenceProperties, + join_schema: SchemaRef, + maintains_input_order: &[bool], + probe_side: Option, + on: &[(Column, Column)], + ) -> Result { + let left_columns_len = self.schema.fields.len(); + let mut new_properties = OrderingEquivalenceProperties::new(join_schema); + + let join_eq_groups = + self.eq_groups() + .join(join_type, right.eq_groups(), left_columns_len, on)?; + new_properties.add_equivalent_groups(join_eq_groups); + + // All joins have 2 children + assert_eq!(maintains_input_order.len(), 2); + let left_maintains = maintains_input_order[0]; + let right_maintains = maintains_input_order[1]; + let left_oeq_class = self.oeq_group(); + let right_oeq_class = right.oeq_group(); + match (left_maintains, right_maintains) { + (true, true) => { + return Err(DataFusionError::Plan( + "Cannot maintain ordering of both sides".to_string(), + )) + } + (true, false) => { + // In this special case, right side ordering can be prefixed with left side ordering. + if let (Some(JoinSide::Left), JoinType::Inner) = (probe_side, join_type) { + let updated_right_oeq = get_updated_right_ordering_equivalent_group( + join_type, + right_oeq_class, + left_columns_len, + )?; + + // Right side ordering equivalence properties should be prepended with + // those of the left side while constructing output ordering equivalence + // properties since stream side is the left side. + // + // If the right table ordering equivalences contain `b ASC`, and the output + // ordering of the left table is `a ASC`, then the ordering equivalence `b ASC` + // for the right table should be converted to `a ASC, b ASC` before it is added + // to the ordering equivalences of the join. + let out_oeq_class = left_oeq_class.join_postfix(&updated_right_oeq); + new_properties.extend(out_oeq_class); + } else { + new_properties.extend(left_oeq_class.clone()); + } + } + (false, true) => { + let updated_right_oeq = get_updated_right_ordering_equivalent_group( + join_type, + right.oeq_group(), + left_columns_len, + )?; + // In this special case, left side ordering can be prefixed with right side ordering. + if let (Some(JoinSide::Right), JoinType::Inner) = (probe_side, join_type) + { + // Left side ordering equivalence properties should be prepended with + // those of the right side while constructing output ordering equivalence + // properties since stream side is the right side. + // + // If the right table ordering equivalences contain `b ASC`, and the output + // ordering of the left table is `a ASC`, then the ordering equivalence `b ASC` + // for the right table should be converted to `a ASC, b ASC` before it is added + // to the ordering equivalences of the join. + let out_oeq_class = updated_right_oeq.join_postfix(left_oeq_class); + new_properties.extend(out_oeq_class); + } else { + new_properties.extend(updated_right_oeq); + } + } + (false, false) => {} + } + Ok(new_properties) + } } #[derive(Debug, Clone, PartialEq)] @@ -1182,15 +1310,6 @@ pub fn update_ordering_equivalence_with_cast( cast_exprs: &[(CastExpr, Column)], input_oeq: &mut OrderingEquivalenceProperties, ) { - // if let Some(cls) = &mut input_oeq.oeq_class { - // for ordering in - // std::iter::once(cls.head().clone()).chain(cls.others().clone().into_iter()) - // { - // if let Some(updated_ordering) = update_with_cast_exprs(cast_exprs, ordering) { - // cls.insert(updated_ordering); - // } - // } - // } let mut new_casts = vec![]; for ordering in input_oeq.oeq_group.iter() { if let Some(updated_ordering) = @@ -1219,20 +1338,6 @@ pub fn ordering_equivalence_properties_helper( oep.extend(OrderingEquivalentGroup::new(eq_orderings.to_vec())); oep } - // oep.extend(Some(OrderingEquivalentClass::new())) - // let first_ordering = if let Some(first) = eq_orderings.first() { - // first - // } else { - // // Return an empty OrderingEquivalenceProperties: - // return oep; - // }; - // // First entry among eq_orderings is the head, skip it: - // for ordering in eq_orderings.iter().skip(1) { - // if !ordering.is_empty() { - // oep.add_ordering_equal_conditions((first_ordering, ordering)) - // } - // } - // oep } /// This function constructs a duplicate-free `LexOrderingReq` by filtering out duplicate @@ -1462,46 +1567,36 @@ pub fn update_ordering( node.state = Some(node.expr.get_ordering(&[])); Ok(Transformed::Yes(node)) } - // Ok(Transformed::Yes(node)) } -/// Combine equivalence properties of the given join inputs. -pub fn combine_join_equivalence_properties2( +/// Update right table ordering equivalences so that: +/// - They point to valid indices at the output of the join schema, and +/// - They are normalized with respect to equivalence columns. +/// +/// To do so, we increment column indices by the size of the left table when +/// join schema consists of a combination of left and right schema (Inner, +/// Left, Full, Right joins). Then, we normalize the sort expressions of +/// ordering equivalences one by one. We make sure that each expression in the +/// ordering equivalence is either: +/// - The head of the one of the equivalent classes, or +/// - Doesn't have an equivalent column. +/// +/// This way; once we normalize an expression according to equivalence properties, +/// it can thereafter safely be used for ordering equivalence normalization. +fn get_updated_right_ordering_equivalent_group( join_type: &JoinType, - left_eq_classes: &EquivalentGroups, - right_eq_classes: &EquivalentGroups, + right_oeq_group: &OrderingEquivalentGroup, left_columns_len: usize, - on: &[(Column, Column)], - out_properties: &mut OrderingEquivalenceProperties, -) -> Result<()> { - let mut res = EquivalentGroups::empty(); +) -> Result { match join_type { + // In these modes, indices of the right schema should be offset by + // the left table size. JoinType::Inner | JoinType::Left | JoinType::Full | JoinType::Right => { - res.extend(left_eq_classes.clone()); - let updated_eq_classes = right_eq_classes - .iter() - .map(|eq_class| add_offset_to_exprs(eq_class.to_vec(), left_columns_len)) - .collect::>>()?; - - res.extend(EquivalentGroups::new(updated_eq_classes)); - } - JoinType::LeftSemi | JoinType::LeftAnti => { - res.extend(left_eq_classes.clone()); - } - JoinType::RightSemi | JoinType::RightAnti => { - res.extend(right_eq_classes.clone()); + return right_oeq_group.add_offset(left_columns_len) } - } - out_properties.eq_groups = res; - if *join_type == JoinType::Inner { - on.iter().for_each(|(lhs, rhs)| { - let new_lhs = Arc::new(lhs.clone()) as _; - let new_rhs = - Arc::new(Column::new(rhs.name(), rhs.index() + left_columns_len)) as _; - out_properties.add_equal_conditions((&new_lhs, &new_rhs)); - }); - } - Ok(()) + _ => {} + }; + Ok(right_oeq_group.clone()) } #[cfg(test)] @@ -1512,7 +1607,7 @@ mod tests { use datafusion_common::Result; use crate::physical_expr::physical_exprs_equal; - use arrow_schema::SortOptions; + use arrow_schema::{Fields, SortOptions}; use std::sync::Arc; // Generate a schema which consists of 5 columns (a, b, c, d, e) @@ -2049,4 +2144,87 @@ mod tests { assert!(physical_exprs_equal(&eq_groups[1], &expected[1])); Ok(()) } + + #[test] + fn test_get_updated_right_ordering_equivalence_properties() -> Result<()> { + let join_type = JoinType::Inner; + + let options = SortOptions::default(); + let right_oeq_class = OrderingEquivalentGroup::new(vec![ + vec![ + PhysicalSortExpr { + expr: Arc::new(Column::new("x", 0)), + options, + }, + PhysicalSortExpr { + expr: Arc::new(Column::new("y", 1)), + options, + }, + ], + vec![ + PhysicalSortExpr { + expr: Arc::new(Column::new("z", 2)), + options, + }, + PhysicalSortExpr { + expr: Arc::new(Column::new("w", 3)), + options, + }, + ], + ]); + + let left_columns_len = 4; + + let fields: Fields = ["a", "b", "c", "d", "x", "y", "z", "w"] + .into_iter() + .map(|name| Field::new(name, DataType::Int32, true)) + .collect(); + + let schema = Schema::new(fields); + let col_a_expr = col("a", &schema)?; + let col_d_expr = col("d", &schema)?; + let col_x_expr = col("x", &schema)?; + let col_y_expr = col("y", &schema)?; + let col_z_expr = col("z", &schema)?; + let col_w_expr = col("w", &schema)?; + + let mut join_eq_properties = OrderingEquivalenceProperties::new(Arc::new(schema)); + join_eq_properties.add_equal_conditions((&col_a_expr, &col_x_expr)); + join_eq_properties.add_equal_conditions((&col_d_expr, &col_w_expr)); + + let result = get_updated_right_ordering_equivalent_group( + &join_type, + &right_oeq_class, + left_columns_len, + )?; + join_eq_properties.extend(result); + let result = join_eq_properties.oeq_group().clone(); + + let expected = OrderingEquivalentGroup::new(vec![ + vec![ + PhysicalSortExpr { + expr: col_a_expr, + options, + }, + PhysicalSortExpr { + expr: col_y_expr, + options, + }, + ], + vec![ + PhysicalSortExpr { + expr: col_z_expr, + options, + }, + PhysicalSortExpr { + expr: col_d_expr, + options, + }, + ], + ]); + + assert_eq!(result, expected); + + Ok(()) + } } diff --git a/datafusion/physical-plan/src/joins/cross_join.rs b/datafusion/physical-plan/src/joins/cross_join.rs index 8b541a7bc107..7a79cd27d952 100644 --- a/datafusion/physical-plan/src/joins/cross_join.rs +++ b/datafusion/physical-plan/src/joins/cross_join.rs @@ -25,7 +25,6 @@ use std::{any::Any, sync::Arc, task::Poll}; use arrow::datatypes::{Fields, Schema, SchemaRef}; use arrow::record_batch::RecordBatch; -use crate::joins::utils::combine_join_ordering_equivalence_properties; use crate::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use crate::DisplayAs; use crate::{ @@ -216,10 +215,11 @@ impl ExecutionPlan for CrossJoinExec { } fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { - combine_join_ordering_equivalence_properties( + let left = self.left.ordering_equivalence_properties(); + let right = self.right.ordering_equivalence_properties(); + left.join( &JoinType::Full, - self.left(), - self.right(), + &right, self.schema(), &[false, false], None, diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index 860d29e3e86a..7dc86755f06b 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -26,8 +26,8 @@ use std::{any::Any, usize, vec}; use crate::joins::utils::{ adjust_indices_by_join_type, apply_join_filter_to_indices, build_batch_from_indices, - calculate_join_output_ordering, combine_join_ordering_equivalence_properties, - get_final_indices_from_bit_map, need_produce_result_in_final, JoinSide, + calculate_join_output_ordering, get_final_indices_from_bit_map, + need_produce_result_in_final, }; use crate::DisplayAs; use crate::{ @@ -63,7 +63,7 @@ use arrow::util::bit_util; use arrow_array::cast::downcast_array; use arrow_schema::ArrowError; use datafusion_common::{ - exec_err, internal_err, plan_err, DataFusionError, JoinType, Result, + exec_err, internal_err, plan_err, DataFusionError, JoinSide, JoinType, Result, }; use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; use datafusion_execution::TaskContext; @@ -365,10 +365,11 @@ impl ExecutionPlan for HashJoinExec { } fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { - combine_join_ordering_equivalence_properties( + let left = self.left.ordering_equivalence_properties(); + let right = self.right.ordering_equivalence_properties(); + left.join( &self.join_type, - &self.left, - &self.right, + &right, self.schema(), &self.maintains_input_order(), Some(Self::probe_side()), @@ -1056,14 +1057,9 @@ mod tests { use hashbrown::raw::RawTable; use crate::{ - common, - expressions::Column, - hash_utils::create_hashes, - joins::{hash_join::build_equal_condition_join_indices, utils::JoinSide}, - memory::MemoryExec, - repartition::RepartitionExec, - test::build_table_i32, - test::exec::MockExec, + common, expressions::Column, hash_utils::create_hashes, + joins::hash_join::build_equal_condition_join_indices, memory::MemoryExec, + repartition::RepartitionExec, test::build_table_i32, test::exec::MockExec, }; use datafusion_execution::config::SessionConfig; use datafusion_execution::runtime_env::{RuntimeConfig, RuntimeEnv}; diff --git a/datafusion/physical-plan/src/joins/hash_join_utils.rs b/datafusion/physical-plan/src/joins/hash_join_utils.rs index 525c1a7145b9..3a2a85c72722 100644 --- a/datafusion/physical-plan/src/joins/hash_join_utils.rs +++ b/datafusion/physical-plan/src/joins/hash_join_utils.rs @@ -24,14 +24,14 @@ use std::ops::IndexMut; use std::sync::Arc; use std::{fmt, usize}; -use crate::joins::utils::{JoinFilter, JoinSide}; +use crate::joins::utils::JoinFilter; use arrow::compute::concat_batches; use arrow::datatypes::{ArrowNativeType, SchemaRef}; use arrow_array::builder::BooleanBufferBuilder; use arrow_array::{ArrowPrimitiveType, NativeAdapter, PrimitiveArray, RecordBatch}; use datafusion_common::tree_node::{Transformed, TreeNode}; -use datafusion_common::{DataFusionError, Result, ScalarValue}; +use datafusion_common::{DataFusionError, JoinSide, Result, ScalarValue}; use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::intervals::{Interval, IntervalBound}; use datafusion_physical_expr::utils::collect_columns; @@ -732,7 +732,7 @@ pub mod tests { use crate::{ expressions::Column, expressions::PhysicalSortExpr, - joins::utils::{ColumnIndex, JoinFilter, JoinSide}, + joins::utils::{ColumnIndex, JoinFilter}, }; use arrow::compute::SortOptions; use arrow::datatypes::{DataType, Field, Schema}; diff --git a/datafusion/physical-plan/src/joins/nested_loop_join.rs b/datafusion/physical-plan/src/joins/nested_loop_join.rs index 1d76b429821a..e881b88c4318 100644 --- a/datafusion/physical-plan/src/joins/nested_loop_join.rs +++ b/datafusion/physical-plan/src/joins/nested_loop_join.rs @@ -24,7 +24,7 @@ use crate::joins::utils::{ build_join_schema, check_join_is_valid, estimate_join_statistics, get_anti_indices, get_anti_u64_indices, get_final_indices_from_bit_map, get_semi_indices, get_semi_u64_indices, partitioned_join_output_partitioning, BuildProbeJoinMetrics, - ColumnIndex, JoinFilter, JoinSide, OnceAsync, OnceFut, + ColumnIndex, JoinFilter, OnceAsync, OnceFut, }; use crate::metrics::{ExecutionPlanMetricsSet, MetricsSet}; use crate::{ @@ -37,7 +37,7 @@ use arrow::array::{ use arrow::datatypes::{Schema, SchemaRef}; use arrow::record_batch::RecordBatch; use arrow::util::bit_util; -use datafusion_common::{exec_err, DataFusionError, Statistics}; +use datafusion_common::{exec_err, DataFusionError, JoinSide, Statistics}; use datafusion_execution::memory_pool::MemoryReservation; use datafusion_expr::JoinType; use datafusion_physical_expr::PhysicalSortExpr; @@ -736,7 +736,6 @@ mod tests { use datafusion_expr::Operator; use datafusion_physical_expr::expressions::BinaryExpr; - use crate::joins::utils::JoinSide; use datafusion_common::{assert_batches_sorted_eq, assert_contains, ScalarValue}; use datafusion_physical_expr::expressions::Literal; use datafusion_physical_expr::PhysicalExpr; diff --git a/datafusion/physical-plan/src/joins/sort_merge_join.rs b/datafusion/physical-plan/src/joins/sort_merge_join.rs index cb02a6a2702b..b9b1724dd8cf 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join.rs @@ -34,8 +34,7 @@ use crate::expressions::Column; use crate::expressions::PhysicalSortExpr; use crate::joins::utils::{ build_join_schema, calculate_join_output_ordering, check_join_is_valid, - combine_join_ordering_equivalence_properties, estimate_join_statistics, - partitioned_join_output_partitioning, JoinOn, JoinSide, + estimate_join_statistics, partitioned_join_output_partitioning, JoinOn, }; use crate::metrics::{ExecutionPlanMetricsSet, MetricBuilder, MetricsSet}; use crate::{ @@ -49,7 +48,7 @@ use arrow::datatypes::{DataType, SchemaRef, TimeUnit}; use arrow::error::ArrowError; use arrow::record_batch::RecordBatch; use datafusion_common::{ - internal_err, not_impl_err, plan_err, DataFusionError, JoinType, Result, + internal_err, not_impl_err, plan_err, DataFusionError, JoinSide, JoinType, Result, }; use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; use datafusion_execution::TaskContext; @@ -283,10 +282,11 @@ impl ExecutionPlan for SortMergeJoinExec { } fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { - combine_join_ordering_equivalence_properties( + let left = self.left.ordering_equivalence_properties(); + let right = self.right.ordering_equivalence_properties(); + left.join( &self.join_type, - &self.left, - &self.right, + &right, self.schema(), &self.maintains_input_order(), Some(Self::probe_side(&self.join_type)), diff --git a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs index 23addd479c92..9333860c8269 100644 --- a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs +++ b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs @@ -49,7 +49,6 @@ use crate::{ utils::{ build_batch_from_indices, build_join_schema, check_join_is_valid, partitioned_join_output_partitioning, ColumnIndex, JoinFilter, JoinOn, - JoinSide, }, }, metrics::{self, ExecutionPlanMetricsSet, MetricBuilder, MetricsSet}, @@ -62,7 +61,7 @@ use arrow::compute::concat_batches; use arrow::datatypes::{Schema, SchemaRef}; use arrow::record_batch::RecordBatch; use datafusion_common::utils::bisect; -use datafusion_common::{internal_err, plan_err, JoinType}; +use datafusion_common::{internal_err, plan_err, JoinSide, JoinType}; use datafusion_common::{DataFusionError, Result}; use datafusion_execution::memory_pool::MemoryConsumer; use datafusion_execution::TaskContext; diff --git a/datafusion/physical-plan/src/joins/utils.rs b/datafusion/physical-plan/src/joins/utils.rs index 3775fc91bcb7..0320dedfe0cd 100644 --- a/datafusion/physical-plan/src/joins/utils.rs +++ b/datafusion/physical-plan/src/joins/utils.rs @@ -19,14 +19,12 @@ use std::cmp::max; use std::collections::HashSet; -use std::fmt::{Display, Formatter}; use std::future::Future; use std::sync::Arc; use std::task::{Context, Poll}; use std::usize; use crate::metrics::{self, ExecutionPlanMetricsSet, MetricBuilder}; -use crate::SchemaRef; use crate::{ColumnStatistics, ExecutionPlan, Partitioning, Statistics}; use arrow::array::{ @@ -39,18 +37,16 @@ use arrow::record_batch::{RecordBatch, RecordBatchOptions}; use datafusion_common::cast::as_boolean_array; use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::{ - exec_err, plan_err, DataFusionError, JoinType, Result, ScalarValue, SharedResult, + exec_err, plan_err, DataFusionError, JoinSide, JoinType, Result, ScalarValue, + SharedResult, }; use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::{ - add_offset_to_lex_ordering, LexOrdering, LexOrderingRef, - OrderingEquivalenceProperties, PhysicalExpr, PhysicalSortExpr, + add_offset_to_lex_ordering, LexOrdering, LexOrderingRef, PhysicalExpr, + PhysicalSortExpr, }; use crate::joins::hash_join_utils::{build_filter_input_order, SortedFilterExpr}; -use datafusion_physical_expr::equivalence::{ - combine_join_equivalence_properties2, OrderingEquivalentGroup, -}; use datafusion_physical_expr::intervals::ExprIntervalGraph; use datafusion_physical_expr::utils::merge_vectors; use futures::future::{BoxFuture, Shared}; @@ -231,153 +227,6 @@ pub fn calculate_join_output_ordering( Ok((!output_ordering.is_empty()).then_some(output_ordering)) } -/// Update right table ordering equivalences so that: -/// - They point to valid indices at the output of the join schema, and -/// - They are normalized with respect to equivalence columns. -/// -/// To do so, we increment column indices by the size of the left table when -/// join schema consists of a combination of left and right schema (Inner, -/// Left, Full, Right joins). Then, we normalize the sort expressions of -/// ordering equivalences one by one. We make sure that each expression in the -/// ordering equivalence is either: -/// - The head of the one of the equivalent classes, or -/// - Doesn't have an equivalent column. -/// -/// This way; once we normalize an expression according to equivalence properties, -/// it can thereafter safely be used for ordering equivalence normalization. -fn get_updated_right_ordering_equivalent_class( - join_type: &JoinType, - right_oeq_group: &OrderingEquivalentGroup, - left_columns_len: usize, -) -> Result { - match join_type { - // In these modes, indices of the right schema should be offset by - // the left table size. - JoinType::Inner | JoinType::Left | JoinType::Full | JoinType::Right => { - return right_oeq_group.add_offset(left_columns_len) - } - _ => {} - }; - Ok(right_oeq_group.clone()) -} - -/// Calculate ordering equivalence properties for the given join operation. -/// TODO: Move this to method as join -pub fn combine_join_ordering_equivalence_properties( - join_type: &JoinType, - left: &Arc, - right: &Arc, - schema: SchemaRef, - maintains_input_order: &[bool], - probe_side: Option, - on: &[(Column, Column)], -) -> Result { - // println!("combine join start"); - let mut new_properties = OrderingEquivalenceProperties::new(schema); - let left_columns_len = left.schema().fields.len(); - let left_oeq_properties = left.ordering_equivalence_properties(); - let right_oeq_properties = right.ordering_equivalence_properties(); - - combine_join_equivalence_properties2( - join_type, - left_oeq_properties.eq_groups(), - right_oeq_properties.eq_groups(), - left_columns_len, - on, - &mut new_properties, - )?; - // All joins have 2 children - assert_eq!(maintains_input_order.len(), 2); - let left_maintains = maintains_input_order[0]; - let right_maintains = maintains_input_order[1]; - let left_oeq_class = left_oeq_properties.oeq_group(); - let right_oeq_class = right_oeq_properties.oeq_group(); - match (left_maintains, right_maintains) { - (true, true) => { - return Err(DataFusionError::Plan( - "Cannot maintain ordering of both sides".to_string(), - )) - } - (true, false) => { - // In this special case, right side ordering can be prefixed with left side ordering. - if let (Some(JoinSide::Left), JoinType::Inner) = (probe_side, join_type) { - let updated_right_oeq = get_updated_right_ordering_equivalent_class( - join_type, - right_oeq_class, - left_columns_len, - )?; - - // Right side ordering equivalence properties should be prepended with - // those of the left side while constructing output ordering equivalence - // properties since stream side is the left side. - // - // If the right table ordering equivalences contain `b ASC`, and the output - // ordering of the left table is `a ASC`, then the ordering equivalence `b ASC` - // for the right table should be converted to `a ASC, b ASC` before it is added - // to the ordering equivalences of the join. - let out_oeq_class = left_oeq_class.join_postfix(&updated_right_oeq); - new_properties.extend(out_oeq_class); - } else { - new_properties.extend(left_oeq_class.clone()); - } - } - (false, true) => { - let updated_right_oeq = get_updated_right_ordering_equivalent_class( - join_type, - right_oeq_properties.oeq_group(), - left_columns_len, - )?; - // In this special case, left side ordering can be prefixed with right side ordering. - if let (Some(JoinSide::Right), JoinType::Inner) = (probe_side, join_type) { - // Left side ordering equivalence properties should be prepended with - // those of the right side while constructing output ordering equivalence - // properties since stream side is the right side. - // - // If the right table ordering equivalences contain `b ASC`, and the output - // ordering of the left table is `a ASC`, then the ordering equivalence `b ASC` - // for the right table should be converted to `a ASC, b ASC` before it is added - // to the ordering equivalences of the join. - let out_oeq_class = updated_right_oeq.join_postfix(left_oeq_class); - new_properties.extend(out_oeq_class); - } else { - new_properties.extend(updated_right_oeq); - } - } - (false, false) => {} - } - // println!("combine join end"); - // println!("new_eq_classes join after: {:?}", new_properties.eq_classes()); - Ok(new_properties) -} - -impl Display for JoinSide { - fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { - match self { - JoinSide::Left => write!(f, "left"), - JoinSide::Right => write!(f, "right"), - } - } -} - -/// Used in ColumnIndex to distinguish which side the index is for -#[derive(Debug, Clone, Copy, PartialEq)] -pub enum JoinSide { - /// Left side of the join - Left, - /// Right side of the join - Right, -} - -impl JoinSide { - /// Inverse the join side - pub fn negate(&self) -> Self { - match self { - JoinSide::Left => JoinSide::Right, - JoinSide::Right => JoinSide::Left, - } - } -} - /// Information about the index and placement (left or right) of the columns #[derive(Debug, Clone)] pub struct ColumnIndex { @@ -1281,7 +1130,6 @@ mod tests { use arrow::{datatypes::DataType, error::ArrowError}; use arrow_schema::SortOptions; use datafusion_common::ScalarValue; - use datafusion_physical_expr::expressions::col; use std::pin::Pin; fn check(left: &[Column], right: &[Column], on: &[(Column, Column)]) -> Result<()> { @@ -1812,90 +1660,6 @@ mod tests { Ok(()) } - #[test] - fn test_get_updated_right_ordering_equivalence_properties() -> Result<()> { - let join_type = JoinType::Inner; - - let options = SortOptions::default(); - let right_oeq_class = OrderingEquivalentGroup::new(vec![ - vec![ - PhysicalSortExpr { - expr: Arc::new(Column::new("x", 0)), - options, - }, - PhysicalSortExpr { - expr: Arc::new(Column::new("y", 1)), - options, - }, - ], - vec![ - PhysicalSortExpr { - expr: Arc::new(Column::new("z", 2)), - options, - }, - PhysicalSortExpr { - expr: Arc::new(Column::new("w", 3)), - options, - }, - ], - ]); - - let left_columns_len = 4; - - let fields: Fields = ["a", "b", "c", "d", "x", "y", "z", "w"] - .into_iter() - .map(|name| Field::new(name, DataType::Int32, true)) - .collect(); - - let schema = Schema::new(fields); - let col_a_expr = col("a", &schema)?; - let col_d_expr = col("d", &schema)?; - let col_x_expr = col("x", &schema)?; - let col_y_expr = col("y", &schema)?; - let col_z_expr = col("z", &schema)?; - let col_w_expr = col("w", &schema)?; - - let mut join_eq_properties = OrderingEquivalenceProperties::new(Arc::new(schema)); - join_eq_properties.add_equal_conditions((&col_a_expr, &col_x_expr)); - join_eq_properties.add_equal_conditions((&col_d_expr, &col_w_expr)); - - // println!("join_eq_properties: {:?}", join_eq_properties); - let result = get_updated_right_ordering_equivalent_class( - &join_type, - &right_oeq_class, - left_columns_len, - )?; - join_eq_properties.extend(result); - let result = join_eq_properties.oeq_group().clone(); - - let expected = OrderingEquivalentGroup::new(vec![ - vec![ - PhysicalSortExpr { - expr: col_a_expr, - options, - }, - PhysicalSortExpr { - expr: col_y_expr, - options, - }, - ], - vec![ - PhysicalSortExpr { - expr: col_z_expr, - options, - }, - PhysicalSortExpr { - expr: col_d_expr, - options, - }, - ], - ]); - - assert_eq!(result, expected); - - Ok(()) - } - #[test] fn test_calculate_join_output_ordering() -> Result<()> { let options = SortOptions::default(); diff --git a/datafusion/proto/src/physical_plan/from_proto.rs b/datafusion/proto/src/physical_plan/from_proto.rs index bb38116e5dba..01b5e5799fac 100644 --- a/datafusion/proto/src/physical_plan/from_proto.rs +++ b/datafusion/proto/src/physical_plan/from_proto.rs @@ -41,7 +41,7 @@ use datafusion::physical_plan::{ functions, Partitioning, }; use datafusion::physical_plan::{ColumnStatistics, PhysicalExpr, Statistics}; -use datafusion_common::{not_impl_err, DataFusionError, Result}; +use datafusion_common::{not_impl_err, DataFusionError, JoinSide, Result}; use object_store::path::Path; use object_store::ObjectMeta; use std::convert::{TryFrom, TryInto}; @@ -52,7 +52,6 @@ use crate::common::proto_error; use crate::convert_required; use crate::logical_plan; use crate::protobuf::physical_expr_node::ExprType; -use datafusion::physical_plan::joins::utils::JoinSide; use datafusion::physical_plan::sorts::sort::SortOptions; impl From<&protobuf::PhysicalColumn> for Column { diff --git a/datafusion/proto/src/physical_plan/to_proto.rs b/datafusion/proto/src/physical_plan/to_proto.rs index cf3dbe26190a..d7e1b4ecc1ae 100644 --- a/datafusion/proto/src/physical_plan/to_proto.rs +++ b/datafusion/proto/src/physical_plan/to_proto.rs @@ -71,9 +71,8 @@ use crate::protobuf::{ use datafusion::logical_expr::BuiltinScalarFunction; use datafusion::physical_expr::expressions::{GetFieldAccessExpr, GetIndexedFieldExpr}; use datafusion::physical_expr::{PhysicalSortExpr, ScalarFunctionExpr}; -use datafusion::physical_plan::joins::utils::JoinSide; use datafusion::physical_plan::udaf::AggregateFunctionExpr; -use datafusion_common::{internal_err, not_impl_err, DataFusionError, Result}; +use datafusion_common::{internal_err, not_impl_err, DataFusionError, JoinSide, Result}; impl TryFrom> for protobuf::PhysicalExprNode { type Error = DataFusionError; From f6f9d4762cc0f41f4cc1e4e9c1d0b98843ce7ce9 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 4 Oct 2023 10:35:43 +0300 Subject: [PATCH 029/122] Simplifications --- datafusion/physical-expr/src/equivalence.rs | 32 ++++++++++++++------- 1 file changed, 22 insertions(+), 10 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 6f25b3e5dc7a..a2347f0088ff 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -26,6 +26,7 @@ use arrow::datatypes::SchemaRef; use crate::physical_expr::{deduplicate_physical_exprs, have_common_entries}; use crate::sort_properties::{ExprOrdering, SortProperties}; +use arrow_schema::SortOptions; use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::utils::longest_consecutive_prefix; use datafusion_common::{DataFusionError, JoinSide, JoinType, Result}; @@ -577,6 +578,18 @@ impl OrderingEquivalentGroup { } None } + + /// Get leading ordering of the expression if it is ordered. + /// `None` means expression is not ordered. + fn get_ordering(&self, expr: &Arc) -> Option { + for ordering in self.iter() { + let leading_ordering = &ordering[0]; + if expr.eq(&leading_ordering.expr) { + return Some(leading_ordering.options); + } + } + None + } } /// `OrderingEquivalenceProperties` keeps track of columns that describe the @@ -1551,17 +1564,16 @@ pub fn update_ordering( Ok(Transformed::Yes(node)) } else if node.expr.as_any().is::() { // We have a Column, which is one of the two possible leaf node types: - // TODO: Make this a method of ordering equivalence - for ordering in ordering_equal_properties.oeq_group().iter() { - let global_ordering = &ordering[0]; - if node.expr.eq(&global_ordering.expr) { - node.state = Some(SortProperties::Ordered(global_ordering.options)); - return Ok(Transformed::Yes(node)); - } + if let Some(options) = ordering_equal_properties + .oeq_group() + .get_ordering(&node.expr) + { + node.state = Some(SortProperties::Ordered(options)); + Ok(Transformed::Yes(node)) + } else { + node.state = None; + Ok(Transformed::No(node)) } - - node.state = None; - Ok(Transformed::No(node)) } else { // We have a Literal, which is the other possible leaf node type: node.state = Some(node.expr.get_ordering(&[])); From 7bd3d23fd4cc65501705c1eae68161ffeba49908 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 4 Oct 2023 10:41:57 +0300 Subject: [PATCH 030/122] Remove redundant code --- datafusion/physical-expr/src/equivalence.rs | 42 +-------------------- datafusion/physical-plan/src/projection.rs | 28 -------------- 2 files changed, 1 insertion(+), 69 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index a2347f0088ff..ca75aa7b9c97 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -use crate::expressions::{CastExpr, Column}; +use crate::expressions::Column; use crate::utils::get_indices_of_exprs_strict; use crate::{ physical_exprs_contains, reverse_order_bys, LexOrdering, LexOrderingRef, @@ -1298,46 +1298,6 @@ impl OrderingEquivalenceBuilder { } } -/// Update `ordering` if it contains cast expression with target column -/// after projection, if there is no cast expression among `ordering` expressions, -/// returns `None`. -fn update_with_cast_exprs( - cast_exprs: &[(CastExpr, Column)], - mut ordering: LexOrdering, -) -> Option { - let mut is_changed = false; - for sort_expr in ordering.iter_mut() { - for (cast_expr, target_col) in cast_exprs.iter() { - if sort_expr.expr.eq(cast_expr.expr()) { - sort_expr.expr = Arc::new(target_col.clone()) as _; - is_changed = true; - } - } - } - is_changed.then_some(ordering) -} - -/// Update cast expressions inside ordering equivalence -/// properties with its target column after projection -pub fn update_ordering_equivalence_with_cast( - cast_exprs: &[(CastExpr, Column)], - input_oeq: &mut OrderingEquivalenceProperties, -) { - let mut new_casts = vec![]; - for ordering in input_oeq.oeq_group.iter() { - if let Some(updated_ordering) = - update_with_cast_exprs(cast_exprs, ordering.clone()) - { - new_casts.push(updated_ordering); - } - } - if !new_casts.is_empty() { - input_oeq - .oeq_group - .extend(OrderingEquivalentGroup::new(new_casts)); - } -} - /// Retrieves the ordering equivalence properties for a given schema and output ordering. pub fn ordering_equivalence_properties_helper( schema: SchemaRef, diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index 04bfbe2daffa..19ec1c510061 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -135,28 +135,10 @@ impl ProjectionExec { }; } - // let orderings = find_orderings_of_exprs( - // &expr, - // input.output_ordering(), - // input.ordering_equivalence_properties(), - // )?; - - // println!("source_to_target_mapping:{:?}", source_to_target_mapping); - // println!("input.ordering_equivalence_properties():{:?}", input.ordering_equivalence_properties()); - // println!("ordering_equivalence_properties():{:?}", input - // .ordering_equivalence_properties() - // .project(&columns_map, &source_to_target_mapping, schema.clone())); - - // let output_ordering = - // validate_output_ordering(output_ordering, &orderings, &expr); - let input_oeq = input.ordering_equivalence_properties(); - // println!("input.ordering_equivalence_properties(): {:?}", input_oeq); let project_oeq = input_oeq.project(&source_to_target_mapping, schema.clone()); let project_orderings = project_oeq.oeq_group(); - // println!("project_oeq: {:?}", project_oeq); let output_ordering = project_orderings.output_ordering(); - // println!("output ordering:{:?}", output_ordering); Ok(Self { expr, @@ -289,16 +271,6 @@ impl ExecutionPlan for ProjectionExec { context: Arc, ) -> Result { trace!("Start ProjectionExec::execute for partition {} of context session_id {} and task_id {:?}", partition, context.session_id(), context.task_id()); - // println!( - // "proj self.input.ordering_equivalence_properties(): {:?}", - // self.input.ordering_equivalence_properties() - // ); - // println!( - // "proj self.ordering_equivalence_properties(): {:?}", - // self.ordering_equivalence_properties() - // ); - // println!("self.source_to_target_mapping: {:?}", self.source_to_target_mapping); - // println!("proj self.output ordering: {:?}", self.output_ordering()); Ok(Box::pin(ProjectionStream { schema: self.schema.clone(), expr: self.expr.iter().map(|x| x.0.clone()).collect(), From a6f18a607d7e747ca327ba87c6475773f4bb4406 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 4 Oct 2023 10:56:52 +0300 Subject: [PATCH 031/122] Minor changes --- datafusion/physical-expr/src/equivalence.rs | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index ca75aa7b9c97..8745db5975a5 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -1518,16 +1518,16 @@ pub fn update_ordering( mut node: ExprOrdering, ordering_equal_properties: &OrderingEquivalenceProperties, ) -> Result> { + let eq_groups = &ordering_equal_properties.eq_groups; + let oeq_group = &ordering_equal_properties.oeq_group; if let Some(children_sort_options) = &node.children_states { // We have an intermediate (non-leaf) node, account for its children: node.state = Some(node.expr.get_ordering(children_sort_options)); Ok(Transformed::Yes(node)) } else if node.expr.as_any().is::() { // We have a Column, which is one of the two possible leaf node types: - if let Some(options) = ordering_equal_properties - .oeq_group() - .get_ordering(&node.expr) - { + let normalized_expr = eq_groups.normalize_expr(node.expr.clone()); + if let Some(options) = oeq_group.get_ordering(&normalized_expr) { node.state = Some(SortProperties::Ordered(options)); Ok(Transformed::Yes(node)) } else { From 8dcc6cdb34722963348a8e0fb04271b8a4cea45c Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 4 Oct 2023 11:33:37 +0300 Subject: [PATCH 032/122] Minor changes --- datafusion/physical-expr/src/utils.rs | 131 ++++++++++---------------- 1 file changed, 50 insertions(+), 81 deletions(-) diff --git a/datafusion/physical-expr/src/utils.rs b/datafusion/physical-expr/src/utils.rs index d73064827355..78e30754248e 100644 --- a/datafusion/physical-expr/src/utils.rs +++ b/datafusion/physical-expr/src/utils.rs @@ -17,8 +17,6 @@ use crate::equivalence::OrderingEquivalenceProperties; use crate::expressions::{BinaryExpr, Column, UnKnownColumn}; -use crate::sort_properties::{ExprOrdering, SortProperties}; -use crate::update_ordering; use crate::{PhysicalExpr, PhysicalSortExpr, PhysicalSortRequirement}; use arrow::array::{make_array, Array, ArrayRef, BooleanArray, MutableArrayData}; @@ -502,57 +500,6 @@ pub fn get_indices_of_matching_sort_exprs_with_order_eq( None } -/// Calculates the output orderings for a set of expressions within the context of a given -/// execution plan. The resulting orderings are all in the type of [`Column`], since these -/// expressions become [`Column`] after the projection step. The expressions having an alias -/// are renamed with those aliases in the returned [`PhysicalSortExpr`]'s. If an expression -/// is found to be unordered, the corresponding entry in the output vector is `None`. -/// -/// # Arguments -/// -/// * `expr` - A slice of tuples containing expressions and their corresponding aliases. -/// -/// * `input_output_ordering` - Output ordering of the input plan. -/// -/// * `input_equal_properties` - Equivalence properties of the columns in the input plan. -/// -/// * `input_ordering_equal_properties` - Ordering equivalence properties of the columns in the input plan. -/// -/// # Returns -/// -/// A `Result` containing a vector of optional [`PhysicalSortExpr`]'s. Each element of the -/// vector corresponds to an expression from the input slice. If an expression can be ordered, -/// the corresponding entry is `Some(PhysicalSortExpr)`. If an expression cannot be ordered, -/// the entry is `None`. -pub fn find_orderings_of_exprs( - expr: &[(Arc, String)], - input_output_ordering: Option<&[PhysicalSortExpr]>, - input_ordering_equal_properties: OrderingEquivalenceProperties, -) -> Result>> { - let mut orderings: Vec> = vec![]; - if let Some(leading_ordering) = - input_output_ordering.and_then(|output_ordering| output_ordering.first()) - { - for (index, (expression, name)) in expr.iter().enumerate() { - let initial_expr = ExprOrdering::new(expression.clone()); - let transformed = initial_expr.transform_up(&|expr| { - update_ordering(expr, leading_ordering, &input_ordering_equal_properties) - })?; - if let Some(SortProperties::Ordered(sort_options)) = transformed.state { - orderings.push(Some(PhysicalSortExpr { - expr: Arc::new(Column::new(name, index)), - options: sort_options, - })); - } else { - orderings.push(None); - } - } - } else { - orderings.extend(expr.iter().map(|_| None)); - } - Ok(orderings) -} - /// Merge left and right sort expressions, checking for duplicates. pub fn merge_vectors( left: &[PhysicalSortExpr], @@ -1309,26 +1256,39 @@ mod tests { Field::new("b", DataType::Int32, true), Field::new("c", DataType::Int32, true), ]); - let orderings = find_orderings_of_exprs( - &[ - (Arc::new(Column::new("b", 1)), "b_new".to_string()), - (Arc::new(Column::new("a", 0)), "a_new".to_string()), - ], - Some(&[PhysicalSortExpr { - expr: Arc::new(Column::new("b", 1)), - options: SortOptions::default(), - }]), - OrderingEquivalenceProperties::new(Arc::new(schema.clone())), - )?; + let mut oeq_properties = + OrderingEquivalenceProperties::new(Arc::new(schema.clone())); + let ordering = vec![PhysicalSortExpr { + expr: Arc::new(Column::new("b", 1)), + options: SortOptions::default(), + }]; + oeq_properties.add_new_orderings(&[ordering]); + let source_to_target_mapping = vec![ + ( + Arc::new(Column::new("b", 1)) as _, + Arc::new(Column::new("b_new", 0)) as _, + ), + ( + Arc::new(Column::new("a", 0)) as _, + Arc::new(Column::new("a_new", 1)) as _, + ), + ]; + let projection_schema = Arc::new(Schema::new(vec![ + Field::new("b_new", DataType::Int32, true), + Field::new("a_new", DataType::Int32, true), + ])); + let projected_oeq = + oeq_properties.project(&source_to_target_mapping, projection_schema); + let orderings = projected_oeq + .oeq_group() + .output_ordering() + .unwrap_or(vec![]); assert_eq!( - vec![ - Some(PhysicalSortExpr { - expr: Arc::new(Column::new("b_new", 0)), - options: SortOptions::default(), - }), - None, - ], + vec![PhysicalSortExpr { + expr: Arc::new(Column::new("b_new", 0)), + options: SortOptions::default(), + }], orderings ); @@ -1337,16 +1297,25 @@ mod tests { Field::new("b", DataType::Int32, true), Field::new("c", DataType::Int32, true), ]); - let orderings = find_orderings_of_exprs( - &[ - (Arc::new(Column::new("c", 2)), "c_new".to_string()), - (Arc::new(Column::new("b", 1)), "b_new".to_string()), - ], - Some(&[]), - OrderingEquivalenceProperties::new(Arc::new(schema)), - )?; - - assert_eq!(vec![None, None], orderings); + let oeq_properties = OrderingEquivalenceProperties::new(Arc::new(schema.clone())); + let source_to_target_mapping = vec![ + ( + Arc::new(Column::new("c", 2)) as _, + Arc::new(Column::new("c_new", 0)) as _, + ), + ( + Arc::new(Column::new("b", 1)) as _, + Arc::new(Column::new("b_new", 1)) as _, + ), + ]; + let projection_schema = Arc::new(Schema::new(vec![ + Field::new("c_new", DataType::Int32, true), + Field::new("b_new", DataType::Int32, true), + ])); + let projected_oeq = + oeq_properties.project(&source_to_target_mapping, projection_schema); + // After projection there is no ordering. + assert!(projected_oeq.oeq_group().output_ordering().is_none()); Ok(()) } From 68ff6ba0994ec0cd7907d7b1e9a575f0c2e0c0d1 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 4 Oct 2023 13:51:36 +0300 Subject: [PATCH 033/122] Simplifications --- datafusion/physical-expr/src/equivalence.rs | 57 +++-- datafusion/physical-expr/src/lib.rs | 5 +- .../physical-expr/src/sort_properties.rs | 57 +---- datafusion/physical-expr/src/utils.rs | 202 +++++------------- .../physical-plan/src/aggregates/mod.rs | 97 +++------ datafusion/physical-plan/src/common.rs | 30 +++ datafusion/physical-plan/src/projection.rs | 59 +---- 7 files changed, 171 insertions(+), 336 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 8745db5975a5..332a90833da2 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -426,6 +426,10 @@ impl EquivalentGroups { } } +/// Stores the mapping between source expression and target expression during projection +/// Indices in the vector corresponds to index after projection. +pub type ProjectionMapping = Vec<(Arc, Arc)>; + /// `LexOrdering` stores the lexicographical ordering for a schema. /// OrderingEquivalentClass keeps track of different alternative orderings than can /// describe the schema. @@ -835,12 +839,8 @@ impl OrderingEquivalenceProperties { expr: target.clone(), options: sort_options, }; - if !projected_properties - .oeq_group - .contains(&vec![sort_expr.clone()]) - { - projected_properties.oeq_group.push(vec![sort_expr]); - } + // Push new ordering to the state. + projected_properties.oeq_group.push(vec![sort_expr]); } } // Remove redundant entries from ordering group if any. @@ -871,7 +871,10 @@ impl OrderingEquivalenceProperties { /// Check whether any permutation of the argument has a prefix with existing ordering. /// Return indices that describes ordering and their ordering information. - pub fn set_satisfy(&self, exprs: &[Arc]) -> Option> { + fn set_satisfy_helper( + &self, + exprs: &[Arc], + ) -> Option> { let exprs_normalized = self.eq_groups.normalize_exprs(exprs); let mut best = vec![]; @@ -895,7 +898,15 @@ impl OrderingEquivalenceProperties { // these indices, we would match existing ordering. For the example above, // this would produce 1, 0; meaning 1st and 0th entries (a, b) among the // GROUP BY expressions b, a, d match input ordering. - best = get_indices_of_exprs_strict(&ordered_exprs, &exprs_normalized) + let indices = + get_indices_of_exprs_strict(&ordered_exprs, &exprs_normalized); + best = indices + .iter() + .enumerate() + .map(|(order_idx, &match_idx)| { + (match_idx, ordering[order_idx].options) + }) + .collect(); } } @@ -906,6 +917,32 @@ impl OrderingEquivalenceProperties { } } + /// Check whether any permutation of the argument has a prefix with existing ordering. + /// Return indices that describes ordering and their ordering information. + pub fn set_satisfy(&self, exprs: &[Arc]) -> Option> { + self.set_satisfy_helper(exprs).map(|indices_and_orders| { + indices_and_orders + .into_iter() + .map(|(idx, _options)| idx) + .collect() + }) + } + + /// Check whether one of the permutation of the exprs satisfies existing ordering. + /// If so, return indices and their orderings. + /// None, indicates that there is no permutation that satisfies ordering. + pub fn set_exactly_satisfy( + &self, + exprs: &[Arc], + ) -> Option> { + if let Some(indices_and_orders) = self.set_satisfy_helper(exprs) { + if indices_and_orders.len() == exprs.len() { + return Some(indices_and_orders); + } + } + None + } + /// Empties the `oeq_group` inside self, When existing orderings are invalidated. pub fn with_empty_ordering_equivalence(mut self) -> OrderingEquivalenceProperties { self.oeq_group = OrderingEquivalentGroup::empty(); @@ -1216,10 +1253,6 @@ pub enum PartitionSearchMode { Sorted, } -// Stores the mapping between source expression and target expression during projection -// Indices in the vector corresponds to index after projection. -type ProjectionMapping = Vec<(Arc, Arc)>; - /// This is a builder object facilitating incremental construction /// for ordering equivalences. pub struct OrderingEquivalenceBuilder { diff --git a/datafusion/physical-expr/src/lib.rs b/datafusion/physical-expr/src/lib.rs index c2557d6a3eb7..104a740cb237 100644 --- a/datafusion/physical-expr/src/lib.rs +++ b/datafusion/physical-expr/src/lib.rs @@ -69,8 +69,7 @@ pub use sort_expr::{ LexOrdering, LexOrderingRef, LexOrderingReq, PhysicalSortExpr, PhysicalSortRequirement, }; -pub use sort_properties::update_ordering; pub use utils::{ - expr_list_eq_any_order, expr_list_eq_strict_order, - normalize_out_expr_with_columns_map, reverse_order_bys, split_conjunction, + expr_list_eq_any_order, expr_list_eq_strict_order, project_out_expr, + reverse_order_bys, split_conjunction, }; diff --git a/datafusion/physical-expr/src/sort_properties.rs b/datafusion/physical-expr/src/sort_properties.rs index d3a24fc8c706..2d5266b2ce80 100644 --- a/datafusion/physical-expr/src/sort_properties.rs +++ b/datafusion/physical-expr/src/sort_properties.rs @@ -17,12 +17,10 @@ use std::{ops::Neg, sync::Arc}; -use crate::expressions::Column; -use crate::utils::get_indices_of_matching_sort_exprs_with_order_eq; -use crate::{OrderingEquivalenceProperties, PhysicalExpr, PhysicalSortExpr}; +use crate::PhysicalExpr; use arrow_schema::SortOptions; -use datafusion_common::tree_node::{Transformed, TreeNode, VisitRecursion}; +use datafusion_common::tree_node::{TreeNode, VisitRecursion}; use datafusion_common::Result; use itertools::Itertools; @@ -220,54 +218,3 @@ impl TreeNode for ExprOrdering { } } } - -/// Calculates the [`SortProperties`] of a given [`ExprOrdering`] node. -/// The node is either a leaf node, or an intermediate node: -/// - If it is a leaf node, the children states are `None`. We directly find -/// the order of the node by looking at the given sort expression and equivalence -/// properties if it is a `Column` leaf, or we mark it as unordered. In the case -/// of a `Literal` leaf, we mark it as singleton so that it can cooperate with -/// some ordered columns at the upper steps. -/// - If it is an intermediate node, the children states matter. Each `PhysicalExpr` -/// and operator has its own rules about how to propagate the children orderings. -/// However, before the children order propagation, it is checked that whether -/// the intermediate node can be directly matched with the sort expression. If there -/// is a match, the sort expression emerges at that node immediately, discarding -/// the order coming from the children. -pub fn update_ordering( - mut node: ExprOrdering, - sort_expr: &PhysicalSortExpr, - ordering_equal_properties: &OrderingEquivalenceProperties, -) -> Result> { - // If we can directly match a sort expr with the current node, we can set - // its state and return early. - // TODO: If there is a PhysicalExpr other than a Column at this node (e.g. - // a BinaryExpr like a + b), and there is an ordering equivalence of - // it (let's say like c + d), we actually can find it at this step. - if sort_expr.expr.eq(&node.expr) { - node.state = Some(SortProperties::Ordered(sort_expr.options)); - return Ok(Transformed::Yes(node)); - } - - if let Some(children_sort_options) = &node.children_states { - // We have an intermediate (non-leaf) node, account for its children: - node.state = Some(node.expr.get_ordering(children_sort_options)); - } else if let Some(column) = node.expr.as_any().downcast_ref::() { - // We have a Column, which is one of the two possible leaf node types: - node.state = get_indices_of_matching_sort_exprs_with_order_eq( - &[sort_expr.clone()], - &[column.clone()], - ordering_equal_properties, - ) - .map(|(sort_options, _)| { - SortProperties::Ordered(SortOptions { - descending: sort_options[0].descending, - nulls_first: sort_options[0].nulls_first, - }) - }); - } else { - // We have a Literal, which is the other possible leaf node type: - node.state = Some(node.expr.get_ordering(&[])); - } - Ok(Transformed::Yes(node)) -} diff --git a/datafusion/physical-expr/src/utils.rs b/datafusion/physical-expr/src/utils.rs index 78e30754248e..04378ddc8565 100644 --- a/datafusion/physical-expr/src/utils.rs +++ b/datafusion/physical-expr/src/utils.rs @@ -15,21 +15,19 @@ // specific language governing permissions and limitations // under the License. -use crate::equivalence::OrderingEquivalenceProperties; use crate::expressions::{BinaryExpr, Column, UnKnownColumn}; -use crate::{PhysicalExpr, PhysicalSortExpr, PhysicalSortRequirement}; +use crate::{PhysicalExpr, PhysicalSortExpr}; use arrow::array::{make_array, Array, ArrayRef, BooleanArray, MutableArrayData}; use arrow::compute::{and_kleene, is_not_null, SlicesIterator}; use arrow::datatypes::SchemaRef; -use arrow_schema::SortOptions; use datafusion_common::tree_node::{ Transformed, TreeNode, TreeNodeRewriter, VisitRecursion, }; -use datafusion_common::utils::longest_consecutive_prefix; use datafusion_common::Result; use datafusion_expr::Operator; +use crate::equivalence::ProjectionMapping; use itertools::Itertools; use petgraph::graph::NodeIndex; use petgraph::stable_graph::StableGraph; @@ -101,24 +99,26 @@ fn split_conjunction_impl<'a>( } } -/// Normalize the output expressions based on Columns Map. +/// Normalize the output expressions based on projection_map. /// -/// If there is a mapping in Columns Map, replace the Column in the output expressions with the 1st Column in the Columns Map. -/// Otherwise, replace the Column with a place holder of [UnKnownColumn] +/// If there is a mapping in projection_map, replace the expression +/// in the output expressions with the expression after mapping. +/// Otherwise, replace the expression with a place holder of [UnKnownColumn] /// -pub fn normalize_out_expr_with_columns_map( +pub fn project_out_expr( expr: Arc, - columns_map: &HashMap>, + projection_map: &ProjectionMapping, ) -> Arc { expr.clone() .transform(&|expr| { - let normalized_form = match expr.as_any().downcast_ref::() { - Some(column) => columns_map - .get(column) - .map(|c| Arc::new(c[0].clone()) as _) - .or_else(|| Some(Arc::new(UnKnownColumn::new(column.name())) as _)), - None => None, - }; + // If expression is not valid after projection. Treat it is as UnknownColumn. + let mut normalized_form = + Some(Arc::new(UnKnownColumn::new(&expr.to_string())) as _); + for (source, target) in projection_map { + if source.eq(&expr) { + normalized_form = Some(target.clone()) + } + } Ok(if let Some(normalized_form) = normalized_form { Transformed::Yes(normalized_form) } else { @@ -409,97 +409,6 @@ pub fn scatter(mask: &BooleanArray, truthy: &dyn Array) -> Result { Ok(make_array(data)) } -/// Return indices of each item in `required_exprs` inside `provided_exprs`. -/// All the items should be found inside `provided_exprs`. Found indices will -/// be a permutation of the range 0, 1, ..., N. For example, \[2,1,0\] is valid -/// (\[0,1,2\] is consecutive), but \[3,1,0\] is not valid (\[0,1,3\] is not -/// consecutive). -fn get_lexicographical_match_indices( - required_exprs: &[Arc], - provided_exprs: &[Arc], -) -> Option> { - let indices_of_equality = get_indices_of_exprs_strict(required_exprs, provided_exprs); - let mut ordered_indices = indices_of_equality.clone(); - ordered_indices.sort(); - let n_match = indices_of_equality.len(); - let first_n = longest_consecutive_prefix(ordered_indices); - (n_match == required_exprs.len() && first_n == n_match && n_match > 0) - .then_some(indices_of_equality) -} - -/// Attempts to find a full match between the required columns to be ordered (lexicographically), and -/// the provided sort options (lexicographically), while considering equivalence properties. -/// -/// It starts by normalizing members of both the required columns and the provided sort options. -/// If a full match is found, returns the sort options and indices of the matches. If no full match is found, -/// the function proceeds to check against ordering equivalence properties. If still no full match is found, -/// the function returns `None`. -pub fn get_indices_of_matching_sort_exprs_with_order_eq( - provided_sorts: &[PhysicalSortExpr], - required_columns: &[Column], - order_eq_properties: &OrderingEquivalenceProperties, -) -> Option<(Vec, Vec)> { - // Create a vector of `PhysicalSortRequirement`s from the required columns: - let sort_requirement_on_requirements = required_columns - .iter() - .map(|required_column| PhysicalSortRequirement { - expr: Arc::new(required_column.clone()) as _, - options: None, - }) - .collect::>(); - - let oeq = OrderingEquivalenceProperties::new(order_eq_properties.schema()); - - let normalized_required = - oeq.normalize_sort_requirements(&sort_requirement_on_requirements); - let normalized_provided = oeq.normalize_sort_requirements( - &PhysicalSortRequirement::from_sort_exprs(provided_sorts.iter()), - ); - - let provided_sorts = normalized_provided - .iter() - .map(|req| req.expr.clone()) - .collect::>(); - - let normalized_required_expr = normalized_required - .iter() - .map(|req| req.expr.clone()) - .collect::>(); - - if let Some(indices_of_equality) = - get_lexicographical_match_indices(&normalized_required_expr, &provided_sorts) - { - return Some(( - indices_of_equality - .iter() - .filter_map(|index| normalized_provided[*index].options) - .collect(), - indices_of_equality, - )); - } - - // We did not find all the expressions, consult ordering equivalence properties: - let oeq_class = order_eq_properties.oeq_group(); - for ordering in oeq_class.iter() { - let order_eq_class_exprs = convert_to_expr(ordering); - if let Some(indices_of_equality) = get_lexicographical_match_indices( - &normalized_required_expr, - &order_eq_class_exprs, - ) { - return Some(( - indices_of_equality - .iter() - .map(|index| ordering[*index].options) - .collect(), - indices_of_equality, - )); - } - } - - // If no match found, return `None`: - None -} - /// Merge left and right sort expressions, checking for duplicates. pub fn merge_vectors( left: &[PhysicalSortExpr], @@ -521,7 +430,7 @@ mod tests { use super::*; use crate::equivalence::{OrderingEquivalenceProperties, OrderingEquivalentGroup}; use crate::expressions::{binary, cast, col, in_list, lit, Column, Literal}; - use crate::PhysicalSortExpr; + use crate::{PhysicalSortExpr, PhysicalSortRequirement}; use arrow::compute::SortOptions; use arrow_array::Int32Array; @@ -1103,7 +1012,17 @@ mod tests { let sort_options = SortOptions::default(); let sort_options_not = SortOptions::default().not(); - let provided_sorts = [ + let required_columns = [ + Arc::new(Column::new("b", 1)) as _, + Arc::new(Column::new("a", 0)) as _, + ]; + let schema = Schema::new(vec![ + Field::new("a", DataType::Int32, true), + Field::new("b", DataType::Int32, true), + ]); + let mut ordering_equal_properties = + OrderingEquivalenceProperties::new(Arc::new(schema)); + ordering_equal_properties.add_new_orderings(&[vec![ PhysicalSortExpr { expr: Arc::new(Column::new("b", 1)), options: sort_options_not, @@ -1112,29 +1031,16 @@ mod tests { expr: Arc::new(Column::new("a", 0)), options: sort_options, }, - ]; - let required_columns = [Column::new("b", 1), Column::new("a", 0)]; - let schema = Schema::new(vec![ - Field::new("a", DataType::Int32, true), - Field::new("b", DataType::Int32, true), - ]); - let ordering_equal_properties = - OrderingEquivalenceProperties::new(Arc::new(schema)); + ]]); assert_eq!( - get_indices_of_matching_sort_exprs_with_order_eq( - &provided_sorts, - &required_columns, - &ordering_equal_properties, - ), - Some((vec![sort_options_not, sort_options], vec![0, 1])) + ordering_equal_properties.set_exactly_satisfy(&required_columns), + Some(vec![(0, sort_options_not), (1, sort_options)]) ); - // required columns are provided in the equivalence classes - let provided_sorts = [PhysicalSortExpr { - expr: Arc::new(Column::new("c", 2)), - options: sort_options, - }]; - let required_columns = [Column::new("b", 1), Column::new("a", 0)]; + let required_columns = [ + Arc::new(Column::new("b", 1)) as _, + Arc::new(Column::new("a", 0)) as _, + ]; let schema = Schema::new(vec![ Field::new("a", DataType::Int32, true), Field::new("b", DataType::Int32, true), @@ -1159,16 +1065,24 @@ mod tests { ], ]); assert_eq!( - get_indices_of_matching_sort_exprs_with_order_eq( - &provided_sorts, - &required_columns, - &ordering_equal_properties, - ), - Some((vec![sort_options_not, sort_options], vec![0, 1])) + ordering_equal_properties.set_exactly_satisfy(&required_columns), + Some(vec![(0, sort_options_not), (1, sort_options)]) ); + let required_columns = [ + Arc::new(Column::new("b", 1)) as _, + Arc::new(Column::new("a", 0)) as _, + ]; + let schema = Schema::new(vec![ + Field::new("a", DataType::Int32, true), + Field::new("b", DataType::Int32, true), + Field::new("c", DataType::Int32, true), + ]); + let mut ordering_equal_properties = + OrderingEquivalenceProperties::new(Arc::new(schema)); + // not satisfied orders - let provided_sorts = [ + ordering_equal_properties.add_new_orderings(&[vec![ PhysicalSortExpr { expr: Arc::new(Column::new("b", 1)), options: sort_options_not, @@ -1181,21 +1095,9 @@ mod tests { expr: Arc::new(Column::new("a", 0)), options: sort_options, }, - ]; - let required_columns = [Column::new("b", 1), Column::new("a", 0)]; - let schema = Schema::new(vec![ - Field::new("a", DataType::Int32, true), - Field::new("b", DataType::Int32, true), - Field::new("c", DataType::Int32, true), - ]); - let ordering_equal_properties = - OrderingEquivalenceProperties::new(Arc::new(schema)); + ]]); assert_eq!( - get_indices_of_matching_sort_exprs_with_order_eq( - &provided_sorts, - &required_columns, - &ordering_equal_properties, - ), + ordering_equal_properties.set_exactly_satisfy(&required_columns), None ); diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index aa373bf18675..22586c2d910c 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -33,16 +33,13 @@ use datafusion_common::{not_impl_err, plan_err, DataFusionError, Result}; use datafusion_execution::TaskContext; use datafusion_expr::Accumulator; use datafusion_physical_expr::{ - expressions::Column, normalize_out_expr_with_columns_map, physical_exprs_contains, - reverse_order_bys, AggregateExpr, LexOrdering, LexOrderingReq, - OrderingEquivalenceProperties, PhysicalExpr, PhysicalSortExpr, - PhysicalSortRequirement, + expressions::Column, physical_exprs_contains, project_out_expr, reverse_order_bys, + AggregateExpr, LexOrdering, LexOrderingReq, OrderingEquivalenceProperties, + PhysicalExpr, PhysicalSortExpr, PhysicalSortRequirement, }; -use datafusion_common::tree_node::{Transformed, TreeNode}; use itertools::{izip, Itertools}; use std::any::Any; -use std::collections::HashMap; use std::sync::Arc; mod group_values; @@ -53,6 +50,7 @@ mod topk; mod topk_stream; use crate::aggregates::topk_stream::GroupedTopKAggregateStream; +use crate::common::calculate_projection_mapping; use crate::windows::get_ordered_partition_by_indices; pub use datafusion_expr::AggregateFunction; use datafusion_physical_expr::aggregate::is_order_sensitive; @@ -213,6 +211,25 @@ impl PhysicalGroupBy { pub fn is_single(&self) -> bool { self.null_expr.is_empty() } + + /// Calculate group by expressions according to input schema. + pub fn input_exprs(&self) -> Vec> { + self.expr + .iter() + .map(|(expr, _alias)| expr.clone()) + .collect::>() + } + + /// This function returns grouping expressions as they occur in the output schema. + fn output_exprs(&self) -> Vec> { + // Update column indices. Since the group by columns come first in the output schema, their + // indices are simply 0..self.group_expr(len). + self.expr + .iter() + .enumerate() + .map(|(index, (_, name))| Arc::new(Column::new(name, index)) as _) + .collect() + } } impl PartialEq for PhysicalGroupBy { @@ -272,9 +289,8 @@ pub struct AggregateExec { /// same as input.schema() but for the final aggregate it will be the same as the input /// to the partial aggregate pub input_schema: SchemaRef, - /// The columns map used to normalize out expressions like Partitioning and PhysicalSortExpr - /// The key is the column from the input schema and the values are the columns from the output schema - columns_map: HashMap>, + /// The source_to_target_mapping used to normalize out expressions like Partitioning and PhysicalSortExpr + /// The key is the expression from the input schema and the value is the expression from the output schema. source_to_target_mapping: Vec<(Arc, Arc)>, /// Execution Metrics metrics: ExecutionPlanMetricsSet, @@ -283,18 +299,6 @@ pub struct AggregateExec { output_ordering: Option, } -/// This function returns grouping expressions as they occur in the output schema. -fn output_group_expr_helper(group_by: &PhysicalGroupBy) -> Vec> { - // Update column indices. Since the group by columns come first in the output schema, their - // indices are simply 0..self.group_expr(len). - group_by - .expr() - .iter() - .enumerate() - .map(|(index, (_, name))| Arc::new(Column::new(name, index)) as _) - .collect() -} - /// This function returns the ordering requirement of the first non-reversible /// order-sensitive aggregate function such as ARRAY_AGG. This requirement serves /// as the initial requirement while calculating the finest requirement among all @@ -417,11 +421,7 @@ fn group_by_contains_all_requirements( group_by: &PhysicalGroupBy, requirement: &LexOrdering, ) -> bool { - let physical_exprs = group_by - .expr() - .iter() - .map(|(expr, _alias)| expr.clone()) - .collect::>(); + let physical_exprs = group_by.input_exprs(); // When we have multiple groups (grouping set) // since group by may be calculated on the subset of the group_by.expr() // it is not guaranteed to have all of the requirements among group by expressions. @@ -490,11 +490,7 @@ impl AggregateExec { )?; // get group by exprs - let groupby_exprs = group_by - .expr - .iter() - .map(|(item, _)| item.clone()) - .collect::>(); + let groupby_exprs = group_by.input_exprs(); // If existing ordering satisfies a prefix of groupby expression, prefix requirement // with this section. In this case, group by will work more efficient let indices = get_ordered_partition_by_indices(&groupby_exprs, &input); @@ -510,34 +506,9 @@ impl AggregateExec { new_requirement.extend(req); new_requirement = collapse_lex_req(new_requirement); - // construct a map from the input columns to the output columns of the Aggregation - let mut columns_map: HashMap> = HashMap::new(); - let mut source_to_target_mapping = vec![]; - let schema_of_input = input.schema(); - for (expr_idx, (expression, name)) in group_by.expr.iter().enumerate() { - if let Some(column) = expression.as_any().downcast_ref::() { - let new_col_idx = schema.index_of(name)?; - let entry = columns_map.entry(column.clone()).or_insert_with(Vec::new); - entry.push(Column::new(name, new_col_idx)); - }; - - let target_expr = - Arc::new(Column::new(name, expr_idx)) as Arc; - let source_expr = expression.clone().transform_down(&|e| match e - .as_any() - .downcast_ref::( - ) { - Some(col) => { - let idx = col.index(); - let matching_input_field = schema_of_input.field(idx); - let matching_input_column = - Column::new(matching_input_field.name(), idx); - Ok(Transformed::Yes(Arc::new(matching_input_column))) - } - None => Ok(Transformed::No(e)), - })?; - source_to_target_mapping.push((source_expr, target_expr)); - } + // construct a map from the input expression to the output expression of the Aggregation group by + let source_to_target_mapping = + calculate_projection_mapping(&group_by.expr, &input.schema())?; let required_input_ordering = if new_requirement.is_empty() { None @@ -559,7 +530,6 @@ impl AggregateExec { input, schema, input_schema, - columns_map, source_to_target_mapping, metrics: ExecutionPlanMetricsSet::new(), required_input_ordering, @@ -586,7 +556,7 @@ impl AggregateExec { /// Grouping expressions as they occur in the output schema pub fn output_group_expr(&self) -> Vec> { - output_group_expr_helper(&self.group_by) + self.group_by.output_exprs() } /// Aggregate expressions @@ -754,10 +724,7 @@ impl ExecutionPlan for AggregateExec { let normalized_exprs = exprs .into_iter() .map(|expr| { - normalize_out_expr_with_columns_map( - expr, - &self.columns_map, - ) + project_out_expr(expr, &self.source_to_target_mapping) }) .collect::>(); Partitioning::Hash(normalized_exprs, part) diff --git a/datafusion/physical-plan/src/common.rs b/datafusion/physical-plan/src/common.rs index c6cfbbfbbac7..6bdb23747c7c 100644 --- a/datafusion/physical-plan/src/common.rs +++ b/datafusion/physical-plan/src/common.rs @@ -23,8 +23,10 @@ use crate::{ColumnStatistics, ExecutionPlan, Statistics}; use arrow::datatypes::Schema; use arrow::ipc::writer::{FileWriter, IpcWriteOptions}; use arrow::record_batch::RecordBatch; +use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::{plan_err, DataFusionError, Result}; use datafusion_execution::memory_pool::MemoryReservation; +use datafusion_physical_expr::equivalence::ProjectionMapping; use datafusion_physical_expr::expressions::{BinaryExpr, Column}; use datafusion_physical_expr::{PhysicalExpr, PhysicalSortExpr}; use futures::{Future, StreamExt, TryStreamExt}; @@ -370,6 +372,34 @@ pub fn batch_byte_size(batch: &RecordBatch) -> usize { batch.get_array_memory_size() } +/// Constructs projection mapping between input and output +pub fn calculate_projection_mapping( + expr: &[(Arc, String)], + input_schema: &Arc, +) -> Result { + // construct a map from the input expressions to the output expression of the Projection + let mut source_to_target_mapping = vec![]; + for (expr_idx, (expression, name)) in expr.iter().enumerate() { + let target_expr = Arc::new(Column::new(name, expr_idx)) as Arc; + + let source_expr = expression.clone().transform_down(&|e| match e + .as_any() + .downcast_ref::() + { + Some(col) => { + let idx = col.index(); + let matching_input_field = input_schema.field(idx); + let matching_input_column = Column::new(matching_input_field.name(), idx); + Ok(Transformed::Yes(Arc::new(matching_input_column))) + } + None => Ok(Transformed::No(e)), + })?; + + source_to_target_mapping.push((source_expr, target_expr)); + } + Ok(source_to_target_mapping) +} + #[cfg(test)] mod tests { use std::ops::Not; diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index 19ec1c510061..5ce17f829658 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -38,11 +38,9 @@ use arrow::record_batch::{RecordBatch, RecordBatchOptions}; use datafusion_common::Result; use datafusion_execution::TaskContext; use datafusion_physical_expr::expressions::Literal; -use datafusion_physical_expr::{ - normalize_out_expr_with_columns_map, OrderingEquivalenceProperties, -}; +use datafusion_physical_expr::{project_out_expr, OrderingEquivalenceProperties}; -use datafusion_common::tree_node::{Transformed, TreeNode}; +use crate::common::calculate_projection_mapping; use futures::stream::{Stream, StreamExt}; use log::trace; @@ -57,9 +55,8 @@ pub struct ProjectionExec { input: Arc, /// The output ordering output_ordering: Option>, - /// The columns map used to normalize out expressions like Partitioning and PhysicalSortExpr - /// The key is the column from the input schema and the values are the columns from the output schema - columns_map: HashMap>, + /// The source_to_target_mapping used to normalize out expressions like Partitioning and PhysicalSortExpr + /// The key is the expression from the input schema and the value is the expression from the output schema source_to_target_mapping: Vec<(Arc, Arc)>, /// Execution metrics metrics: ExecutionPlanMetricsSet, @@ -94,46 +91,9 @@ impl ProjectionExec { input_schema.metadata().clone(), )); - // construct a map from the input columns to the output columns of the Projection - let mut columns_map: HashMap> = HashMap::new(); - let mut source_to_target_mapping = vec![]; - for (expr_idx, (expression, name)) in expr.iter().enumerate() { - let target_expr = - Arc::new(Column::new(name, expr_idx)) as Arc; - - let source_expr = expression.clone().transform_down(&|e| match e - .as_any() - .downcast_ref::( - ) { - Some(col) => { - let idx = col.index(); - let matching_input_field = input_schema.field(idx); - let matching_input_column = - Column::new(matching_input_field.name(), idx); - Ok(Transformed::Yes(Arc::new(matching_input_column))) - } - None => Ok(Transformed::No(e)), - })?; - - source_to_target_mapping.push((source_expr, target_expr)); - } - // println!("source_to_target_mapping: {:?}", source_to_target_mapping); - for (expr_idx, (expression, name)) in expr.iter().enumerate() { - if let Some(column) = expression.as_any().downcast_ref::() { - // For some executors, logical and physical plan schema fields - // are not the same. The information in a `Column` comes from - // the logical plan schema. Therefore, to produce correct results - // we use the field in the input schema with the same index. This - // corresponds to the physical plan `Column`. - let idx = column.index(); - let matching_input_field = input_schema.field(idx); - let matching_input_column = Column::new(matching_input_field.name(), idx); - let entry = columns_map - .entry(matching_input_column) - .or_insert_with(Vec::new); - entry.push(Column::new(name, expr_idx)); - }; - } + // construct a map from the input expressions to the output expression of the Projection + let source_to_target_mapping = + calculate_projection_mapping(&expr, &input_schema)?; let input_oeq = input.ordering_equivalence_properties(); let project_oeq = input_oeq.project(&source_to_target_mapping, schema.clone()); @@ -145,7 +105,6 @@ impl ProjectionExec { schema, input, output_ordering, - columns_map, source_to_target_mapping, metrics: ExecutionPlanMetricsSet::new(), }) @@ -219,9 +178,7 @@ impl ExecutionPlan for ProjectionExec { Partitioning::Hash(exprs, part) => { let normalized_exprs = exprs .into_iter() - .map(|expr| { - normalize_out_expr_with_columns_map(expr, &self.columns_map) - }) + .map(|expr| project_out_expr(expr, &self.source_to_target_mapping)) .collect::>(); Partitioning::Hash(normalized_exprs, part) From 552d4474e9bf1f11488086c984e1b3cbafc8ab7e Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 4 Oct 2023 15:43:50 +0300 Subject: [PATCH 034/122] Simplifications --- .../src/physical_optimizer/sort_pushdown.rs | 2 - datafusion/physical-expr/src/equivalence.rs | 101 +++--------------- datafusion/physical-expr/src/utils.rs | 3 +- .../physical-expr/src/window/built_in.rs | 41 ++++--- datafusion/physical-plan/src/common.rs | 4 + datafusion/physical-plan/src/filter.rs | 14 --- .../src/joins/symmetric_hash_join.rs | 16 +++ .../physical-plan/src/repartition/mod.rs | 1 + datafusion/physical-plan/src/sorts/sort.rs | 8 +- datafusion/physical-plan/src/union.rs | 5 +- .../src/windows/bounded_window_agg_exec.rs | 11 -- datafusion/physical-plan/src/windows/mod.rs | 8 +- 12 files changed, 63 insertions(+), 151 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/sort_pushdown.rs b/datafusion/core/src/physical_optimizer/sort_pushdown.rs index 0d8faa570b2e..77faee84d0ab 100644 --- a/datafusion/core/src/physical_optimizer/sort_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/sort_pushdown.rs @@ -264,13 +264,11 @@ fn pushdown_requirement_to_children( // && plan.ordering_equivalence_properties().get_finer_ordering() // ) { - // println!("parent_required if:{:?}", parent_required); // If the current plan is a leaf node or can not maintain any of the input ordering, can not pushed down requirements. // For RepartitionExec, we always choose to not push down the sort requirements even the RepartitionExec(input_partition=1) could maintain input ordering. // Pushing down is not beneficial Ok(None) } else { - // println!("parent_required else:{:?}", parent_required); Ok(Some( maintains_input_order .iter() diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 332a90833da2..c31cd91efc2d 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -635,9 +635,16 @@ impl OrderingEquivalenceProperties { } } + /// Add OrderingEquivalenceProperties of the other to the state. + pub fn extend(mut self, other: OrderingEquivalenceProperties) -> Self { + self.eq_groups.extend(other.eq_groups); + self.oeq_group.extend(other.oeq_group); + self.with_constants(other.constants) + } + /// Extends `OrderingEquivalenceProperties` by adding ordering inside the `other` /// to the `self.oeq_class`. - pub fn extend(&mut self, other: OrderingEquivalentGroup) { + pub fn add_ordering_equivalent_group(&mut self, other: OrderingEquivalentGroup) { for ordering in other.into_iter() { if !self.oeq_group.contains(&ordering) { self.oeq_group.push(ordering); @@ -1208,9 +1215,9 @@ impl OrderingEquivalenceProperties { // for the right table should be converted to `a ASC, b ASC` before it is added // to the ordering equivalences of the join. let out_oeq_class = left_oeq_class.join_postfix(&updated_right_oeq); - new_properties.extend(out_oeq_class); + new_properties.add_ordering_equivalent_group(out_oeq_class); } else { - new_properties.extend(left_oeq_class.clone()); + new_properties.add_ordering_equivalent_group(left_oeq_class.clone()); } } (false, true) => { @@ -1231,9 +1238,9 @@ impl OrderingEquivalenceProperties { // for the right table should be converted to `a ASC, b ASC` before it is added // to the ordering equivalences of the join. let out_oeq_class = updated_right_oeq.join_postfix(left_oeq_class); - new_properties.extend(out_oeq_class); + new_properties.add_ordering_equivalent_group(out_oeq_class); } else { - new_properties.extend(updated_right_oeq); + new_properties.add_ordering_equivalent_group(updated_right_oeq); } } (false, false) => {} @@ -1253,84 +1260,6 @@ pub enum PartitionSearchMode { Sorted, } -/// This is a builder object facilitating incremental construction -/// for ordering equivalences. -pub struct OrderingEquivalenceBuilder { - // eq_properties: EquivalenceProperties, - ordering_eq_properties: OrderingEquivalenceProperties, - existing_ordering: Vec, - schema: SchemaRef, -} - -impl OrderingEquivalenceBuilder { - pub fn new(schema: SchemaRef) -> Self { - // let eq_properties = EquivalenceProperties::new(schema.clone()); - let ordering_eq_properties = OrderingEquivalenceProperties::new(schema.clone()); - Self { - // eq_properties, - ordering_eq_properties, - existing_ordering: vec![], - schema, - } - } - - pub fn extend( - mut self, - new_ordering_eq_properties: OrderingEquivalenceProperties, - ) -> Self { - self.ordering_eq_properties - .eq_groups - .extend(new_ordering_eq_properties.eq_groups); - self.ordering_eq_properties - .extend(new_ordering_eq_properties.oeq_group); - self - } - - pub fn with_existing_ordering( - mut self, - existing_ordering: Option>, - ) -> Self { - if let Some(existing_ordering) = existing_ordering { - self.existing_ordering = existing_ordering; - } - self - } - - pub fn add_equal_conditions( - &mut self, - new_equivalent_ordering: Vec, - ) { - let mut normalized_out_ordering = vec![]; - for item in &self.existing_ordering { - // To account for ordering equivalences, first normalize the expression: - // let normalized = self.eq_properties.normalize_expr(item.expr.clone()); - normalized_out_ordering.push(PhysicalSortExpr { - expr: item.expr.clone(), - options: item.options, - }); - } - // If there is an existing ordering, add new ordering as an equivalence: - if !normalized_out_ordering.is_empty() { - self.ordering_eq_properties - .add_new_orderings(&[normalized_out_ordering, new_equivalent_ordering]); - } - } - - /// Return a reference to the schema with which this builder was constructed with - pub fn schema(&self) -> &SchemaRef { - &self.schema - } - - /// Return a reference to the existing ordering - pub fn existing_ordering(&self) -> &LexOrdering { - &self.existing_ordering - } - - pub fn build(self) -> OrderingEquivalenceProperties { - self.ordering_eq_properties - } -} - /// Retrieves the ordering equivalence properties for a given schema and output ordering. pub fn ordering_equivalence_properties_helper( schema: SchemaRef, @@ -1341,7 +1270,9 @@ pub fn ordering_equivalence_properties_helper( // Return an empty OrderingEquivalenceProperties: oep } else { - oep.extend(OrderingEquivalentGroup::new(eq_orderings.to_vec())); + oep.add_ordering_equivalent_group(OrderingEquivalentGroup::new( + eq_orderings.to_vec(), + )); oep } } @@ -2202,7 +2133,7 @@ mod tests { &right_oeq_class, left_columns_len, )?; - join_eq_properties.extend(result); + join_eq_properties.add_ordering_equivalent_group(result); let result = join_eq_properties.oeq_group().clone(); let expected = OrderingEquivalentGroup::new(vec![ diff --git a/datafusion/physical-expr/src/utils.rs b/datafusion/physical-expr/src/utils.rs index 04378ddc8565..9c757a2c4a5e 100644 --- a/datafusion/physical-expr/src/utils.rs +++ b/datafusion/physical-expr/src/utils.rs @@ -1130,7 +1130,8 @@ mod tests { options: sort_options, }], ]; - equal_properties.extend(OrderingEquivalentGroup::new(others)); + equal_properties + .add_ordering_equivalent_group(OrderingEquivalentGroup::new(others)); let mut expected_oeq = OrderingEquivalenceProperties::new(Arc::new(schema)); expected_oeq.add_new_orderings(&[ diff --git a/datafusion/physical-expr/src/window/built_in.rs b/datafusion/physical-expr/src/window/built_in.rs index 9bb360ff3699..3e3de61d78dd 100644 --- a/datafusion/physical-expr/src/window/built_in.rs +++ b/datafusion/physical-expr/src/window/built_in.rs @@ -23,12 +23,10 @@ use std::sync::Arc; use super::BuiltInWindowFunctionExpr; use super::WindowExpr; -use crate::equivalence::OrderingEquivalenceBuilder; use crate::expressions::PhysicalSortExpr; -use crate::utils::{convert_to_expr, get_indices_of_exprs_strict}; use crate::window::window_expr::{get_orderby_values, WindowFn}; use crate::window::{PartitionBatches, PartitionWindowAggStates, WindowState}; -use crate::{reverse_order_bys, PhysicalExpr}; +use crate::{reverse_order_bys, OrderingEquivalenceProperties, PhysicalExpr}; use arrow::array::{new_empty_array, ArrayRef}; use arrow::compute::SortOptions; use arrow::datatypes::Field; @@ -75,12 +73,15 @@ impl BuiltInWindowExpr { /// If `self.expr` doesn't have an ordering, ordering equivalence properties /// are not updated. Otherwise, ordering equivalence properties are updated /// by the ordering of `self.expr`. - pub fn add_equal_orderings(&self, builder: &mut OrderingEquivalenceBuilder) { - let schema = builder.schema(); - if let Some(fn_res_ordering) = self.expr.get_result_ordering(schema) { + pub fn add_equal_orderings( + &self, + oeq_properties: &mut OrderingEquivalenceProperties, + ) { + let schema = oeq_properties.schema(); + if let Some(fn_res_ordering) = self.expr.get_result_ordering(&schema) { if self.partition_by.is_empty() { // In the absence of a PARTITION BY, ordering of `self.expr` is global: - builder.add_equal_conditions(vec![fn_res_ordering]); + oeq_properties.add_new_orderings(&[vec![fn_res_ordering]]); } else { // If we have a PARTITION BY, built-in functions can not introduce // a global ordering unless the existing ordering is compatible @@ -88,22 +89,18 @@ impl BuiltInWindowExpr { // expressions and existing ordering expressions are equal (w.r.t. // set equality), we can prefix the ordering of `self.expr` with // the existing ordering. - let existing_ordering = builder.existing_ordering(); - let existing_ordering_exprs = convert_to_expr(existing_ordering); - // Get indices of the PARTITION BY expressions among input ordering expressions: - let pb_indices = get_indices_of_exprs_strict( - &self.partition_by, - &existing_ordering_exprs, - ); - // Existing ordering should match exactly with PARTITION BY expressions. - // There should be no missing/extra entries in the existing ordering. - // Otherwise, prefixing wouldn't work. - if pb_indices.len() == self.partition_by.len() - && pb_indices.len() == existing_ordering.len() + if let Some(indices_and_orders) = + oeq_properties.set_exactly_satisfy(&self.partition_by) { - let mut new_ordering = existing_ordering.to_vec(); - new_ordering.push(fn_res_ordering); - builder.add_equal_conditions(new_ordering); + let mut ordering = indices_and_orders + .into_iter() + .map(|(idx, options)| PhysicalSortExpr { + expr: self.partition_by[idx].clone(), + options, + }) + .collect::>(); + ordering.push(fn_res_ordering); + oeq_properties.add_new_orderings(&[ordering]); } } } diff --git a/datafusion/physical-plan/src/common.rs b/datafusion/physical-plan/src/common.rs index 6bdb23747c7c..805d4d385cd3 100644 --- a/datafusion/physical-plan/src/common.rs +++ b/datafusion/physical-plan/src/common.rs @@ -387,6 +387,10 @@ pub fn calculate_projection_mapping( .downcast_ref::() { Some(col) => { + // Sometimes, expression and its name in the input_schema doesn't match. + // This can cause problems. Hence in here we make sure that expression name + // matches with the name in the inout_schema. + // Conceptually, source_expr and expression should be same. let idx = col.index(); let matching_input_field = input_schema.field(idx); let matching_input_column = Column::new(matching_input_field.name(), idx); diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index 83c948c00ca6..5f5a6e1700ec 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -143,16 +143,6 @@ impl ExecutionPlan for FilterExec { vec![true] } - // fn equivalence_properties(&self) -> EquivalenceProperties { - // // Combine the equal predicates with the input equivalence properties - // let mut input_properties = self.input.equivalence_properties(); - // let (equal_pairs, _ne_pairs) = collect_columns_from_predicate(&self.predicate); - // for new_condition in equal_pairs { - // input_properties.add_equal_conditions(new_condition) - // } - // input_properties - // } - fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { let stats = self.statistics(); // Combine the equal predicates with the input equivalence properties @@ -170,10 +160,6 @@ impl ExecutionPlan for FilterExec { .filter(|column| col_stats[column.index()].is_singleton()) .map(|column| Arc::new(column) as Arc) .collect::>(); - // // let filter_oeq = self.input.ordering_equivalence_properties(); - // let res = filter_oeq.with_constants(constants); - // println!("filter res:{:?}", res); - // res filter_oeq.with_constants(constants) } else { filter_oeq diff --git a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs index 9333860c8269..98e2dd036ef2 100644 --- a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs +++ b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs @@ -69,6 +69,7 @@ use datafusion_physical_expr::intervals::ExprIntervalGraph; use crate::joins::utils::prepare_sorted_exprs; use ahash::RandomState; +use datafusion_physical_expr::OrderingEquivalenceProperties; use futures::stream::{select, BoxStream}; use futures::{Stream, StreamExt}; use hashbrown::HashSet; @@ -431,6 +432,21 @@ impl ExecutionPlan for SymmetricHashJoinExec { None } + fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { + let left = self.left.ordering_equivalence_properties(); + let right = self.right.ordering_equivalence_properties(); + left.join( + &self.join_type, + &right, + self.schema(), + &self.maintains_input_order(), + // Has alternating probe side + None, + self.on(), + ) + .unwrap() + } + fn children(&self) -> Vec> { vec![self.left.clone(), self.right.clone()] } diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 0e3bfd5db73b..3b1a699dcaf1 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -456,6 +456,7 @@ impl ExecutionPlan for RepartitionExec { } fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { + // If ordering is not preserved, reset ordering equivalent group. if !self.maintains_input_order()[0] { self.input .ordering_equivalence_properties() diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index 46b209f2f27b..0893281545d3 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -827,13 +827,7 @@ impl ExecutionPlan for SortExec { fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { let input_oeq = self.input.ordering_equivalence_properties(); - // println!("sort input: {:?}", input_oeq); - // let mut res = OrderingEquivalenceProperties::new(self.schema()); - - // let res = input_oeq.with_reorder(self.expr.to_vec()); - // // println!("sort output: {:?}", res); - // res - + // reset ordering equivalent group with new ordering. input_oeq.with_reorder(self.expr.to_vec()) } diff --git a/datafusion/physical-plan/src/union.rs b/datafusion/physical-plan/src/union.rs index 19c2f81b7a03..b6bc5d6d4ff5 100644 --- a/datafusion/physical-plan/src/union.rs +++ b/datafusion/physical-plan/src/union.rs @@ -235,13 +235,10 @@ impl ExecutionPlan for UnionExec { let mut union_oeq = OrderingEquivalenceProperties::new(self.schema()); for elem in first_oeq.iter() { if child_oeqs.iter().all(|child_oeq| child_oeq.contains(elem)) { - // res.push(elem); - // Search meet instead of exact + // TODO: Search meet instead of exact union_oeq.add_new_orderings(&[elem.clone()]) } } - // let mut union_oeq = OrderingEquivalenceProperties::new(self.schema()); - // union_oeq.add_ordering_equal_conditions() union_oeq } diff --git a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs index f75f3708743b..75b66b175d93 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -68,17 +68,6 @@ use hashbrown::raw::RawTable; use indexmap::IndexMap; use log::debug; -// #[derive(Debug, Clone, PartialEq)] -// /// Specifies partition column properties in terms of input ordering -// pub enum PartitionSearchMode { -// /// None of the columns among the partition columns is ordered. -// Linear, -// /// Some columns of the partition columns are ordered but not all -// PartiallySorted(Vec), -// /// All Partition columns are ordered (Also empty case) -// Sorted, -// } - /// Window execution plan #[derive(Debug)] pub struct BoundedWindowAggExec { diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index b8b2b443aefe..b388240789bf 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -38,7 +38,6 @@ use datafusion_expr::{ PartitionEvaluator, WindowFrame, WindowUDF, }; use datafusion_physical_expr::{ - equivalence::OrderingEquivalenceBuilder, window::{BuiltInWindowFunctionExpr, SlidingAggregateWindowExpr}, AggregateExpr, OrderingEquivalenceProperties, PhysicalSortRequirement, }; @@ -331,18 +330,17 @@ pub(crate) fn window_ordering_equivalence( ) -> OrderingEquivalenceProperties { // We need to update the schema, so we can not directly use // `input.ordering_equivalence_properties()`. - let mut builder = OrderingEquivalenceBuilder::new(schema.clone()) - .with_existing_ordering(input.output_ordering().map(|elem| elem.to_vec())) + let mut window_oeq_properties = OrderingEquivalenceProperties::new(schema.clone()) .extend(input.ordering_equivalence_properties()); for expr in window_expr { if let Some(builtin_window_expr) = expr.as_any().downcast_ref::() { - builtin_window_expr.add_equal_orderings(&mut builder); + builtin_window_expr.add_equal_orderings(&mut window_oeq_properties); } } - builder.build() + window_oeq_properties } /// Constructs the best-fitting windowing operator (a `WindowAggExec` or a From 38f31019251327bea08a80553f3218b464781af4 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 4 Oct 2023 18:25:49 +0300 Subject: [PATCH 035/122] Simplifications --- .github/pull_request_template.md | 2 +- datafusion/physical-expr/src/equivalence.rs | 62 ++++++++++--------- datafusion/physical-expr/src/utils.rs | 14 ++++- .../physical-expr/src/window/built_in.rs | 24 ++++--- 4 files changed, 63 insertions(+), 39 deletions(-) diff --git a/.github/pull_request_template.md b/.github/pull_request_template.md index 442e6e4009f6..06db092d6fc8 100644 --- a/.github/pull_request_template.md +++ b/.github/pull_request_template.md @@ -37,4 +37,4 @@ If there are user-facing changes then we may require documentation to be updated \ No newline at end of file +--> diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index c31cd91efc2d..f49700ad1421 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -878,10 +878,7 @@ impl OrderingEquivalenceProperties { /// Check whether any permutation of the argument has a prefix with existing ordering. /// Return indices that describes ordering and their ordering information. - fn set_satisfy_helper( - &self, - exprs: &[Arc], - ) -> Option> { + pub fn set_satisfy(&self, exprs: &[Arc]) -> Option> { let exprs_normalized = self.eq_groups.normalize_exprs(exprs); let mut best = vec![]; @@ -905,15 +902,7 @@ impl OrderingEquivalenceProperties { // these indices, we would match existing ordering. For the example above, // this would produce 1, 0; meaning 1st and 0th entries (a, b) among the // GROUP BY expressions b, a, d match input ordering. - let indices = - get_indices_of_exprs_strict(&ordered_exprs, &exprs_normalized); - best = indices - .iter() - .enumerate() - .map(|(order_idx, &match_idx)| { - (match_idx, ordering[order_idx].options) - }) - .collect(); + best = get_indices_of_exprs_strict(&ordered_exprs, &exprs_normalized); } } @@ -924,27 +913,44 @@ impl OrderingEquivalenceProperties { } } - /// Check whether any permutation of the argument has a prefix with existing ordering. - /// Return indices that describes ordering and their ordering information. - pub fn set_satisfy(&self, exprs: &[Arc]) -> Option> { - self.set_satisfy_helper(exprs).map(|indices_and_orders| { - indices_and_orders - .into_iter() - .map(|(idx, _options)| idx) - .collect() - }) - } - /// Check whether one of the permutation of the exprs satisfies existing ordering. /// If so, return indices and their orderings. /// None, indicates that there is no permutation that satisfies ordering. pub fn set_exactly_satisfy( &self, exprs: &[Arc], - ) -> Option> { - if let Some(indices_and_orders) = self.set_satisfy_helper(exprs) { - if indices_and_orders.len() == exprs.len() { - return Some(indices_and_orders); + ) -> Option> { + if let Some(indices) = self.set_satisfy(exprs) { + // A permutation of the exprs satisfies one of the existing orderings. + if indices.len() == exprs.len() { + return Some(indices); + } + } + None + } + + /// Get ordering of the expressions in the argument + /// Assumes arguments define lexicographical ordering. + /// None, represents none of the existing ordering satisfy + /// lexicographical ordering of the exprs. + pub fn get_lex_ordering( + &self, + exprs: &[Arc], + ) -> Option> { + let normalized_exprs = self.eq_groups.normalize_exprs(exprs); + for ordering in self.oeq_group.iter() { + if normalized_exprs.len() <= ordering.len() { + let mut ordering_options = vec![]; + for (expr, sort_expr) in izip!(normalized_exprs.iter(), ordering.iter()) { + if sort_expr.expr.eq(expr) { + ordering_options.push(sort_expr.options); + } else { + break; + } + if ordering_options.len() == normalized_exprs.len() { + return Some(ordering_options); + } + } } } None diff --git a/datafusion/physical-expr/src/utils.rs b/datafusion/physical-expr/src/utils.rs index 9c757a2c4a5e..d31d78c885bf 100644 --- a/datafusion/physical-expr/src/utils.rs +++ b/datafusion/physical-expr/src/utils.rs @@ -1034,7 +1034,12 @@ mod tests { ]]); assert_eq!( ordering_equal_properties.set_exactly_satisfy(&required_columns), - Some(vec![(0, sort_options_not), (1, sort_options)]) + Some(vec![0, 1]) + ); + + assert_eq!( + ordering_equal_properties.get_lex_ordering(&required_columns), + Some(vec![sort_options_not, sort_options]) ); let required_columns = [ @@ -1066,7 +1071,12 @@ mod tests { ]); assert_eq!( ordering_equal_properties.set_exactly_satisfy(&required_columns), - Some(vec![(0, sort_options_not), (1, sort_options)]) + Some(vec![0, 1]) + ); + + assert_eq!( + ordering_equal_properties.get_lex_ordering(&required_columns), + Some(vec![sort_options_not, sort_options]) ); let required_columns = [ diff --git a/datafusion/physical-expr/src/window/built_in.rs b/datafusion/physical-expr/src/window/built_in.rs index 3e3de61d78dd..133493452913 100644 --- a/datafusion/physical-expr/src/window/built_in.rs +++ b/datafusion/physical-expr/src/window/built_in.rs @@ -36,6 +36,7 @@ use datafusion_common::{Result, ScalarValue}; use datafusion_expr::window_state::WindowAggState; use datafusion_expr::window_state::WindowFrameContext; use datafusion_expr::WindowFrame; +use itertools::izip; /// A window expr that takes the form of a [`BuiltInWindowFunctionExpr`]. #[derive(Debug)] @@ -89,18 +90,25 @@ impl BuiltInWindowExpr { // expressions and existing ordering expressions are equal (w.r.t. // set equality), we can prefix the ordering of `self.expr` with // the existing ordering. - if let Some(indices_and_orders) = + if let Some(indices) = oeq_properties.set_exactly_satisfy(&self.partition_by) { - let mut ordering = indices_and_orders + let lex_partition_by = indices .into_iter() - .map(|(idx, options)| PhysicalSortExpr { - expr: self.partition_by[idx].clone(), - options, - }) + .map(|idx| self.partition_by[idx].clone()) + .collect::>(); + if let Some(ordering_options) = + oeq_properties.get_lex_ordering(&lex_partition_by) + { + let mut ordering = izip!( + lex_partition_by.into_iter(), + ordering_options.into_iter() + ) + .map(|(expr, options)| PhysicalSortExpr { expr, options }) .collect::>(); - ordering.push(fn_res_ordering); - oeq_properties.add_new_orderings(&[ordering]); + ordering.push(fn_res_ordering); + oeq_properties.add_new_orderings(&[ordering]); + } } } } From 062823f15314d55fa47e0c4290a3df31aacd80cf Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 5 Oct 2023 09:49:26 +0300 Subject: [PATCH 036/122] Move window to util from method, simplifications --- .../src/physical_optimizer/enforce_sorting.rs | 2 +- .../core/src/physical_optimizer/test_utils.rs | 2 +- datafusion/core/src/physical_planner.rs | 2 +- .../core/tests/fuzz_cases/window_fuzz.rs | 4 +- datafusion/physical-expr/src/equivalence.rs | 82 +--------------- .../physical-plan/src/aggregates/mod.rs | 9 +- .../physical-plan/src/aggregates/order/mod.rs | 2 +- .../src/windows/bounded_window_agg_exec.rs | 2 +- datafusion/physical-plan/src/windows/mod.rs | 98 ++++++++++++++++--- datafusion/proto/src/physical_plan/mod.rs | 5 +- 10 files changed, 100 insertions(+), 108 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index f5172eb4b7ed..d35805224c2b 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -59,7 +59,7 @@ use datafusion_common::tree_node::{Transformed, TreeNode, VisitRecursion}; use datafusion_common::{plan_err, DataFusionError}; use datafusion_physical_expr::{PhysicalSortExpr, PhysicalSortRequirement}; -use datafusion_physical_expr::equivalence::PartitionSearchMode; +use datafusion_physical_plan::windows::PartitionSearchMode; use itertools::izip; /// This rule inspects [`SortExec`]'s in the given physical plan and removes the diff --git a/datafusion/core/src/physical_optimizer/test_utils.rs b/datafusion/core/src/physical_optimizer/test_utils.rs index 56b640266549..85692d0131dc 100644 --- a/datafusion/core/src/physical_optimizer/test_utils.rs +++ b/datafusion/core/src/physical_optimizer/test_utils.rs @@ -46,7 +46,7 @@ use datafusion_physical_expr::expressions::col; use datafusion_physical_expr::{PhysicalExpr, PhysicalSortExpr}; use async_trait::async_trait; -use datafusion_physical_expr::equivalence::PartitionSearchMode; +use datafusion_physical_plan::windows::PartitionSearchMode; async fn register_current_csv( ctx: &SessionContext, diff --git a/datafusion/core/src/physical_planner.rs b/datafusion/core/src/physical_planner.rs index 0e9f4e4f87bc..e2481880c6eb 100644 --- a/datafusion/core/src/physical_planner.rs +++ b/datafusion/core/src/physical_planner.rs @@ -87,8 +87,8 @@ use datafusion_expr::expr_rewriter::{unalias, unnormalize_cols}; use datafusion_expr::logical_plan::builder::wrap_projection_for_join_if_necessary; use datafusion_expr::{DescribeTable, DmlStatement, StringifiedPlan, WriteOp}; use datafusion_expr::{WindowFrame, WindowFrameBound}; -use datafusion_physical_expr::equivalence::PartitionSearchMode; use datafusion_physical_expr::expressions::Literal; +use datafusion_physical_plan::windows::PartitionSearchMode; use datafusion_sql::utils::window_expr_common_partition_keys; use futures::future::BoxFuture; use futures::{FutureExt, StreamExt, TryStreamExt}; diff --git a/datafusion/core/tests/fuzz_cases/window_fuzz.rs b/datafusion/core/tests/fuzz_cases/window_fuzz.rs index 278ea37acb36..0365228fd0fb 100644 --- a/datafusion/core/tests/fuzz_cases/window_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/window_fuzz.rs @@ -40,15 +40,15 @@ use datafusion_expr::{ use datafusion::prelude::{SessionConfig, SessionContext}; use datafusion_common::{Result, ScalarValue}; use datafusion_expr::type_coercion::aggregates::coerce_types; -use datafusion_physical_expr::equivalence::PartitionSearchMode; use datafusion_physical_expr::expressions::{cast, col, lit}; use datafusion_physical_expr::{PhysicalExpr, PhysicalSortExpr}; +use datafusion_physical_plan::windows::PartitionSearchMode; use test_utils::add_empty_batches; #[cfg(test)] mod tests { use super::*; - use datafusion_physical_expr::equivalence::PartitionSearchMode::{ + use datafusion_physical_plan::windows::PartitionSearchMode::{ Linear, PartiallySorted, Sorted, }; diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index f49700ad1421..65f6c9cb00c2 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -18,8 +18,8 @@ use crate::expressions::Column; use crate::utils::get_indices_of_exprs_strict; use crate::{ - physical_exprs_contains, reverse_order_bys, LexOrdering, LexOrderingRef, - LexOrderingReq, PhysicalExpr, PhysicalSortExpr, PhysicalSortRequirement, + physical_exprs_contains, LexOrdering, LexOrderingRef, LexOrderingReq, PhysicalExpr, + PhysicalSortExpr, PhysicalSortRequirement, }; use arrow::datatypes::SchemaRef; @@ -710,6 +710,7 @@ impl OrderingEquivalenceProperties { /// Add physical expression that have constant value to the `self.constants` pub fn with_constants(mut self, constants: Vec>) -> Self { + let constants = self.eq_groups.normalize_exprs(&constants); constants.into_iter().for_each(|constant| { if !physical_exprs_contains(&self.constants, &constant) { self.constants.push(constant); @@ -1107,72 +1108,6 @@ impl OrderingEquivalenceProperties { .all(|(req, given)| given.compatible(&req)) } - /// Compares physical ordering (output ordering of the `input` operator) with - /// `partitionby_exprs` and `orderby_keys` to decide whether existing ordering - /// is sufficient to run the current window operator. - /// - A `None` return value indicates that we can not remove the sort in question - /// (input ordering is not sufficient to run current window executor). - /// - A `Some((bool, PartitionSearchMode))` value indicates that the window operator - /// can run with existing input ordering, so we can remove `SortExec` before it. - /// The `bool` field in the return value represents whether we should reverse window - /// operator to remove `SortExec` before it. The `PartitionSearchMode` field represents - /// the mode this window operator should work in to accomodate the existing ordering. - pub fn get_window_mode( - &self, - partitionby_exprs: &[Arc], - orderby_keys: &[PhysicalSortExpr], - ) -> Result> { - let partitionby_exprs = self.eq_groups.normalize_exprs(partitionby_exprs); - let mut orderby_keys = self.eq_groups.normalize_sort_exprs(orderby_keys); - // Keep the order by expressions that are not inside partition by expressions. - orderby_keys.retain(|sort_expr| { - !physical_exprs_contains(&partitionby_exprs, &sort_expr.expr) - }); - let mut partition_search_mode = PartitionSearchMode::Linear; - let mut partition_by_reqs: Vec = vec![]; - if partitionby_exprs.is_empty() { - partition_search_mode = PartitionSearchMode::Sorted; - } else if let Some(indices) = self.set_satisfy(&partitionby_exprs) { - let elem = indices - .iter() - .map(|&idx| PhysicalSortRequirement { - expr: partitionby_exprs[idx].clone(), - options: None, - }) - .collect::>(); - partition_by_reqs.extend(elem); - if indices.len() == partitionby_exprs.len() { - partition_search_mode = PartitionSearchMode::Sorted; - } else if !indices.is_empty() { - partition_search_mode = PartitionSearchMode::PartiallySorted(indices); - } - } - - let order_by_reqs = PhysicalSortRequirement::from_sort_exprs(&orderby_keys); - let req = [partition_by_reqs.clone(), order_by_reqs].concat(); - let req = collapse_lex_req(req); - if req.is_empty() { - // When requirement is empty, - // prefer None. Instead of Linear. - return Ok(None); - } - // Treat partition by exprs as constant. During analysis of requirements are satisfied. - let partition_by_oeq = self.clone().with_constants(partitionby_exprs.clone()); - if partition_by_oeq.ordering_satisfy_requirement_concrete(&req) { - // Window can be run with existing ordering - return Ok(Some((false, partition_search_mode))); - } - let reverse_order_by_reqs = - PhysicalSortRequirement::from_sort_exprs(&reverse_order_bys(&orderby_keys)); - let req = [partition_by_reqs, reverse_order_by_reqs].concat(); - let req = collapse_lex_req(req); - if partition_by_oeq.ordering_satisfy_requirement_concrete(&req) { - // Window can be run with existing ordering, if the ordering requirements would be reversed - return Ok(Some((true, partition_search_mode))); - } - Ok(None) - } - /// Calculate ordering equivalence properties for the given join operation. pub fn join( &self, @@ -1255,17 +1190,6 @@ impl OrderingEquivalenceProperties { } } -#[derive(Debug, Clone, PartialEq)] -/// Specifies partition column properties in terms of input ordering -pub enum PartitionSearchMode { - /// None of the columns among the partition columns is ordered. - Linear, - /// Some columns of the partition columns are ordered but not all - PartiallySorted(Vec), - /// All Partition columns are ordered (Also empty case) - Sorted, -} - /// Retrieves the ordering equivalence properties for a given schema and output ordering. pub fn ordering_equivalence_properties_helper( schema: SchemaRef, diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 22586c2d910c..4c4f5175f212 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -51,10 +51,12 @@ mod topk_stream; use crate::aggregates::topk_stream::GroupedTopKAggregateStream; use crate::common::calculate_projection_mapping; -use crate::windows::get_ordered_partition_by_indices; +use crate::windows::{ + get_ordered_partition_by_indices, get_window_mode, PartitionSearchMode, +}; pub use datafusion_expr::AggregateFunction; use datafusion_physical_expr::aggregate::is_order_sensitive; -use datafusion_physical_expr::equivalence::{collapse_lex_req, PartitionSearchMode}; +use datafusion_physical_expr::equivalence::collapse_lex_req; pub use datafusion_physical_expr::expressions::create_aggregate_expr; use datafusion_physical_expr::expressions::{Max, Min}; @@ -386,13 +388,12 @@ fn get_aggregate_search_mode( .map(|(item, _)| item.clone()) .collect::>(); let mut partition_search_mode = PartitionSearchMode::Linear; - let input_oeq = input.ordering_equivalence_properties(); if !group_by.is_single() || groupby_exprs.is_empty() { return Ok(partition_search_mode); } if let Some((should_reverse, mode)) = - input_oeq.get_window_mode(&groupby_exprs, ordering_req)? + get_window_mode(&groupby_exprs, ordering_req, input)? { let all_reversible = aggr_expr .iter() diff --git a/datafusion/physical-plan/src/aggregates/order/mod.rs b/datafusion/physical-plan/src/aggregates/order/mod.rs index 7fc5f5dbb9bb..d905a3118af6 100644 --- a/datafusion/physical-plan/src/aggregates/order/mod.rs +++ b/datafusion/physical-plan/src/aggregates/order/mod.rs @@ -18,12 +18,12 @@ use arrow_array::ArrayRef; use arrow_schema::Schema; use datafusion_common::Result; -use datafusion_physical_expr::equivalence::PartitionSearchMode; use datafusion_physical_expr::{EmitTo, PhysicalSortExpr}; mod full; mod partial; +use crate::windows::PartitionSearchMode; pub(crate) use full::GroupOrderingFull; pub(crate) use partial::GroupOrderingPartial; diff --git a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs index 75b66b175d93..026442cfead3 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -31,6 +31,7 @@ use crate::expressions::PhysicalSortExpr; use crate::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; use crate::windows::{ calc_requirements, get_ordered_partition_by_indices, window_ordering_equivalence, + PartitionSearchMode, }; use crate::{ ColumnStatistics, DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, @@ -61,7 +62,6 @@ use datafusion_physical_expr::{ }; use ahash::RandomState; -use datafusion_physical_expr::equivalence::PartitionSearchMode; use futures::stream::Stream; use futures::{ready, StreamExt}; use hashbrown::raw::RawTable; diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index b388240789bf..a701700f7e9e 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -38,6 +38,7 @@ use datafusion_expr::{ PartitionEvaluator, WindowFrame, WindowUDF, }; use datafusion_physical_expr::{ + reverse_order_bys, window::{BuiltInWindowFunctionExpr, SlidingAggregateWindowExpr}, AggregateExpr, OrderingEquivalenceProperties, PhysicalSortRequirement, }; @@ -46,13 +47,24 @@ mod bounded_window_agg_exec; mod window_agg_exec; pub use bounded_window_agg_exec::BoundedWindowAggExec; -use datafusion_physical_expr::equivalence::PartitionSearchMode; +use datafusion_physical_expr::equivalence::collapse_lex_req; pub use window_agg_exec::WindowAggExec; pub use datafusion_physical_expr::window::{ BuiltInWindowExpr, PlainAggregateWindowExpr, WindowExpr, }; +#[derive(Debug, Clone, PartialEq)] +/// Specifies partition column properties in terms of input ordering +pub enum PartitionSearchMode { + /// None of the columns among the partition columns is ordered. + Linear, + /// Some columns of the partition columns are ordered but not all + PartiallySorted(Vec), + /// All Partition columns are ordered (Also empty case) + Sorted, +} + /// Create a physical expression for window function pub fn create_window_expr( fun: &WindowFunction, @@ -366,9 +378,8 @@ pub fn get_best_fitting_window( let partitionby_exprs = window_exprs[0].partition_by(); let orderby_keys = window_exprs[0].order_by(); let (should_reverse, partition_search_mode) = - if let Some((should_reverse, partition_search_mode)) = input - .ordering_equivalence_properties() - .get_window_mode(partitionby_exprs, orderby_keys)? + if let Some((should_reverse, partition_search_mode)) = + get_window_mode(partitionby_exprs, orderby_keys, input)? { (should_reverse, partition_search_mode) } else { @@ -423,6 +434,64 @@ pub fn get_best_fitting_window( } } +/// Compares physical ordering (output ordering of the `input` operator) with +/// `partitionby_exprs` and `orderby_keys` to decide whether existing ordering +/// is sufficient to run the current window operator. +/// - A `None` return value indicates that we can not remove the sort in question +/// (input ordering is not sufficient to run current window executor). +/// - A `Some((bool, PartitionSearchMode))` value indicates that the window operator +/// can run with existing input ordering, so we can remove `SortExec` before it. +/// The `bool` field in the return value represents whether we should reverse window +/// operator to remove `SortExec` before it. The `PartitionSearchMode` field represents +/// the mode this window operator should work in to accomodate the existing ordering. +pub fn get_window_mode( + partitionby_exprs: &[Arc], + orderby_keys: &[PhysicalSortExpr], + input: &Arc, +) -> Result> { + let input_oeq = input.ordering_equivalence_properties(); + let mut partition_search_mode = PartitionSearchMode::Linear; + let mut partition_by_reqs: Vec = vec![]; + if partitionby_exprs.is_empty() { + partition_search_mode = PartitionSearchMode::Sorted; + } else if let Some(indices) = input_oeq.set_satisfy(partitionby_exprs) { + let elem = indices + .iter() + .map(|&idx| PhysicalSortRequirement { + expr: partitionby_exprs[idx].clone(), + options: None, + }) + .collect::>(); + partition_by_reqs.extend(elem); + if indices.len() == partitionby_exprs.len() { + partition_search_mode = PartitionSearchMode::Sorted; + } else if !indices.is_empty() { + partition_search_mode = PartitionSearchMode::PartiallySorted(indices); + } + } + + // Treat partition by exprs as constant. During analysis of requirements are satisfied. + let partition_by_oeq = input_oeq.with_constants(partitionby_exprs.to_vec()); + let order_by_reqs = PhysicalSortRequirement::from_sort_exprs(orderby_keys); + let reverse_order_by_reqs = + PhysicalSortRequirement::from_sort_exprs(&reverse_order_bys(orderby_keys)); + for (should_swap, order_by_reqs) in + [(false, order_by_reqs), (true, reverse_order_by_reqs)] + { + let req = [partition_by_reqs.clone(), order_by_reqs].concat(); + let req = collapse_lex_req(req); + if req.is_empty() { + // When requirement is empty, + // prefer None. Instead of Linear. + return Ok(None); + } else if partition_by_oeq.ordering_satisfy_requirement_concrete(&req) { + // Window can be run with existing ordering + return Ok(Some((should_swap, partition_search_mode))); + } + } + Ok(None) +} + #[cfg(test)] mod tests { use super::*; @@ -437,10 +506,8 @@ mod tests { use arrow::datatypes::{DataType, Field, SchemaRef}; use datafusion_execution::TaskContext; - use datafusion_physical_expr::equivalence::PartitionSearchMode::{ - Linear, PartiallySorted, Sorted, - }; use futures::FutureExt; + use PartitionSearchMode::{Linear, PartiallySorted, Sorted}; fn create_test_schema() -> Result { let nullable_column = Field::new("nullable_col", DataType::Int32, true); @@ -717,7 +784,7 @@ mod tests { (vec!["a"], vec!["a", "c"], None), (vec!["a"], vec!["a", "b", "c"], Some(Sorted)), (vec!["b"], vec!["a"], Some(Linear)), - (vec!["b"], vec!["b"], None), + (vec!["b"], vec!["b"], Some(Linear)), (vec!["b"], vec!["c"], None), (vec!["b"], vec!["a", "b"], Some(Linear)), (vec!["b"], vec!["b", "c"], None), @@ -725,7 +792,7 @@ mod tests { (vec!["b"], vec!["a", "b", "c"], Some(Linear)), (vec!["c"], vec!["a"], Some(Linear)), (vec!["c"], vec!["b"], None), - (vec!["c"], vec!["c"], None), + (vec!["c"], vec!["c"], Some(Linear)), (vec!["c"], vec!["a", "b"], Some(Linear)), (vec!["c"], vec!["b", "c"], None), (vec!["c"], vec!["a", "c"], Some(Linear)), @@ -738,10 +805,10 @@ mod tests { (vec!["b", "a"], vec!["a", "c"], Some(Sorted)), (vec!["b", "a"], vec!["a", "b", "c"], Some(Sorted)), (vec!["c", "b"], vec!["a"], Some(Linear)), - (vec!["c", "b"], vec!["b"], None), - (vec!["c", "b"], vec!["c"], None), + (vec!["c", "b"], vec!["b"], Some(Linear)), + (vec!["c", "b"], vec!["c"], Some(Linear)), (vec!["c", "b"], vec!["a", "b"], Some(Linear)), - (vec!["c", "b"], vec!["b", "c"], None), + (vec!["c", "b"], vec!["b", "c"], Some(Linear)), (vec!["c", "b"], vec!["a", "c"], Some(Linear)), (vec!["c", "b"], vec!["a", "b", "c"], Some(Linear)), (vec!["c", "a"], vec!["a"], Some(PartiallySorted(vec![1]))), @@ -790,9 +857,8 @@ mod tests { let options = SortOptions::default(); order_by_exprs.push(PhysicalSortExpr { expr, options }); } - let res = exec_unbounded - .ordering_equivalence_properties() - .get_window_mode(&partition_by_exprs, &order_by_exprs)?; + let res = + get_window_mode(&partition_by_exprs, &order_by_exprs, &exec_unbounded)?; // Since reversibility is not important in this test. Convert Option<(bool, PartitionSearchMode)> to Option let res = res.map(|(_, mode)| mode); assert_eq!( @@ -956,7 +1022,7 @@ mod tests { } assert_eq!( - exec_unbounded.ordering_equivalence_properties().get_window_mode(&partition_by_exprs, &order_by_exprs)?, + get_window_mode(&partition_by_exprs, &order_by_exprs, &exec_unbounded)?, *expected, "Unexpected result for in unbounded test case#: {case_idx:?}, case: {test_case:?}" ); diff --git a/datafusion/proto/src/physical_plan/mod.rs b/datafusion/proto/src/physical_plan/mod.rs index 708f8872fda2..8257f9aa3458 100644 --- a/datafusion/proto/src/physical_plan/mod.rs +++ b/datafusion/proto/src/physical_plan/mod.rs @@ -25,7 +25,6 @@ use datafusion::datasource::file_format::file_compression_type::FileCompressionT use datafusion::datasource::physical_plan::{AvroExec, CsvExec, ParquetExec}; use datafusion::execution::runtime_env::RuntimeEnv; use datafusion::execution::FunctionRegistry; -use datafusion::physical_expr::equivalence::PartitionSearchMode; use datafusion::physical_plan::aggregates::{create_aggregate_expr, AggregateMode}; use datafusion::physical_plan::aggregates::{AggregateExec, PhysicalGroupBy}; use datafusion::physical_plan::analyze::AnalyzeExec; @@ -44,7 +43,9 @@ use datafusion::physical_plan::repartition::RepartitionExec; use datafusion::physical_plan::sorts::sort::SortExec; use datafusion::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; use datafusion::physical_plan::union::UnionExec; -use datafusion::physical_plan::windows::{BoundedWindowAggExec, WindowAggExec}; +use datafusion::physical_plan::windows::{ + BoundedWindowAggExec, PartitionSearchMode, WindowAggExec, +}; use datafusion::physical_plan::{ udaf, AggregateExpr, ExecutionPlan, Partitioning, PhysicalExpr, WindowExpr, }; From 4a6ecd9f745dc86cbcfc625cdf0c410c98d34e13 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 5 Oct 2023 10:28:10 +0300 Subject: [PATCH 037/122] Simplifications --- .../src/physical_optimizer/enforce_sorting.rs | 1 - .../src/physical_optimizer/sort_pushdown.rs | 27 ++++++++++--------- 2 files changed, 15 insertions(+), 13 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index d35805224c2b..b912c5da4098 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -2204,7 +2204,6 @@ mod tests { } #[tokio::test] - #[ignore] async fn test_pushdown_through_spm() -> Result<()> { let schema = create_test_schema3()?; let sort_exprs = vec![sort_expr("a", &schema), sort_expr("b", &schema)]; diff --git a/datafusion/core/src/physical_optimizer/sort_pushdown.rs b/datafusion/core/src/physical_optimizer/sort_pushdown.rs index 77faee84d0ab..a5328f41d419 100644 --- a/datafusion/core/src/physical_optimizer/sort_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/sort_pushdown.rs @@ -251,23 +251,26 @@ fn pushdown_requirement_to_children( || plan.as_any().is::() || is_limit(plan) || plan.as_any().is::() - // Do not push-down through SortPreservingMergeExec when - // ordering requirement invalidates requirement of sort preserving merge exec. - || is_sort_preserving_merge(plan) - // !ordering_satisfy( - // parent_required - // .map(|req| PhysicalSortRequirement::to_sort_exprs(req.to_vec())) - // .as_deref(), - // plan.output_ordering(), - // || plan.ordering_equivalence_properties(), - // ) - // && plan.ordering_equivalence_properties().get_finer_ordering() - // ) { // If the current plan is a leaf node or can not maintain any of the input ordering, can not pushed down requirements. // For RepartitionExec, we always choose to not push down the sort requirements even the RepartitionExec(input_partition=1) could maintain input ordering. // Pushing down is not beneficial Ok(None) + } else if is_sort_preserving_merge(plan) { + let ordering_req = parent_required.unwrap_or(&[]); + let new_ordering = PhysicalSortRequirement::to_sort_exprs(ordering_req.to_vec()); + let mut spm_oeq = plan.ordering_equivalence_properties(); + // Sort preserving merge will have new ordering, one requirement above is pushed down to its below. + spm_oeq = spm_oeq.with_reorder(new_ordering); + // Do not push-down through SortPreservingMergeExec when + // ordering requirement invalidates requirement of sort preserving merge exec. + if !spm_oeq.ordering_satisfy(plan.output_ordering()) { + Ok(None) + } else { + // Can push-down through SortPreservingMergeExec, because parent requirement is finer + // than SortPreservingMergeExec output ordering. + Ok(Some(vec![parent_required.map(|elem| elem.to_vec())])) + } } else { Ok(Some( maintains_input_order From 795bf5b7530c60fe2adedf13254f4446e11d3704 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 5 Oct 2023 11:37:32 +0300 Subject: [PATCH 038/122] Propagate meet in the union --- .../src/physical_optimizer/enforce_sorting.rs | 1 - .../src/physical_optimizer/sort_pushdown.rs | 14 +++------ datafusion/physical-expr/src/equivalence.rs | 20 ++++++++++++ datafusion/physical-plan/src/union.rs | 31 +++++++++++++++---- 4 files changed, 49 insertions(+), 17 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index b912c5da4098..29104fc87047 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -1600,7 +1600,6 @@ mod tests { } #[tokio::test] - #[ignore] async fn test_window_multi_path_sort() -> Result<()> { let schema = create_test_schema()?; diff --git a/datafusion/core/src/physical_optimizer/sort_pushdown.rs b/datafusion/core/src/physical_optimizer/sort_pushdown.rs index a5328f41d419..35920540d724 100644 --- a/datafusion/core/src/physical_optimizer/sort_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/sort_pushdown.rs @@ -335,16 +335,10 @@ fn try_pushdown_requirements_to_join( &smj.maintains_input_order(), Some(SortMergeJoinExec::probe_side(&smj.join_type())), )?; - // TODO: use hypothetical new ordering. Add support for this API. - let new_output_ordering = - PhysicalSortRequirement::from_sort_exprs(&new_output_ordering.unwrap_or(vec![])); - let finer_req = smj - .ordering_equivalence_properties() - .get_finer_requirement(&new_output_ordering, parent_required.unwrap_or(&[])); - let mut should_pushdown = false; - if let Some(finer_req) = finer_req { - should_pushdown = finer_req.eq(&new_output_ordering); - } + let mut smj_oeq = smj.ordering_equivalence_properties(); + // smj will have this ordering when its input changes. + smj_oeq = smj_oeq.with_reorder(new_output_ordering.unwrap_or(vec![])); + let should_pushdown = smj_oeq.ordering_satisfy_requirement(parent_required); Ok(should_pushdown.then(|| { let required_input_ordering = smj.required_input_ordering(); let new_req = Some(PhysicalSortRequirement::from_sort_exprs(&sort_expr)); diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 65f6c9cb00c2..ea51509fd4fc 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -1007,6 +1007,26 @@ impl OrderingEquivalenceProperties { None } + /// Find the coarser requirement among `req1` and `req2` + /// If `None`, this means that `req1` and `req2` are not compatible + pub fn get_meet_ordering<'a>( + &self, + req1: &'a [PhysicalSortExpr], + req2: &'a [PhysicalSortExpr], + ) -> Option<&'a [PhysicalSortExpr]> { + let lhs = self.normalize_sort_exprs(req1); + let rhs = self.normalize_sort_exprs(req2); + if izip!(lhs.iter(), rhs.iter()).all(|(lhs, rhs)| lhs.eq(rhs)) { + if lhs.len() < rhs.len() { + return Some(req1); + } else { + return Some(req2); + } + } + // Neither `provided` nor `req` satisfies one another, they are incompatible. + None + } + /// Find the finer requirement among `req1` and `req2` /// If `None`, this means that `req1` and `req2` are not compatible /// e.g there is no requirement that satisfies both diff --git a/datafusion/physical-plan/src/union.rs b/datafusion/physical-plan/src/union.rs index b6bc5d6d4ff5..10d4aeeb91ed 100644 --- a/datafusion/physical-plan/src/union.rs +++ b/datafusion/physical-plan/src/union.rs @@ -229,14 +229,33 @@ impl ExecutionPlan for UnionExec { let child_oeqs = self .inputs .iter() - .map(|child| child.ordering_equivalence_properties().oeq_group().clone()) + .map(|child| child.ordering_equivalence_properties()) .collect::>(); - let first_oeq = child_oeqs[0].clone(); let mut union_oeq = OrderingEquivalenceProperties::new(self.schema()); - for elem in first_oeq.iter() { - if child_oeqs.iter().all(|child_oeq| child_oeq.contains(elem)) { - // TODO: Search meet instead of exact - union_oeq.add_new_orderings(&[elem.clone()]) + // Iterate ordering equivalent group of first child + for elem in child_oeqs[0].oeq_group().iter() { + let mut meet = None; + for child_oeq in &child_oeqs { + let mut meet_found = false; + for ordering in child_oeq.oeq_group().iter() { + if let Some(coarse_ordering) = + child_oeq.get_meet_ordering(ordering, elem) + { + meet = Some(coarse_ordering.to_vec()); + meet_found = true; + } + } + // If either of the child doesn't have a meet + // There is no meet. + if !meet_found { + meet = None; + break; + } + } + // All of the children have a common meet ordering. + // This ordering can be propagated in Union. + if let Some(meet) = meet { + union_oeq.add_new_orderings(&[meet]) } } union_oeq From 7d3d4f2c79453bc35d7f30a0cc8f0553e94499c0 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 5 Oct 2023 12:28:46 +0300 Subject: [PATCH 039/122] Simplifications --- datafusion/physical-plan/src/union.rs | 35 +++++++++++++++------------ 1 file changed, 19 insertions(+), 16 deletions(-) diff --git a/datafusion/physical-plan/src/union.rs b/datafusion/physical-plan/src/union.rs index 10d4aeeb91ed..b6000c9f4859 100644 --- a/datafusion/physical-plan/src/union.rs +++ b/datafusion/physical-plan/src/union.rs @@ -234,24 +234,27 @@ impl ExecutionPlan for UnionExec { let mut union_oeq = OrderingEquivalenceProperties::new(self.schema()); // Iterate ordering equivalent group of first child for elem in child_oeqs[0].oeq_group().iter() { - let mut meet = None; - for child_oeq in &child_oeqs { - let mut meet_found = false; - for ordering in child_oeq.oeq_group().iter() { - if let Some(coarse_ordering) = - child_oeq.get_meet_ordering(ordering, elem) - { - meet = Some(coarse_ordering.to_vec()); - meet_found = true; + // Seed for the meet. + let mut meet = Some(elem.clone()); + child_oeqs.iter().for_each(|child_oeq| { + if let Some(meet_vec) = &meet { + // let mut meet_found = false; + let res = child_oeq + .oeq_group() + .iter() + .filter_map(|ordering| { + child_oeq.get_meet_ordering(ordering, meet_vec) + }) + .collect::>(); + if let Some(new_meet) = res.first() { + meet = Some(new_meet.to_vec()); + } else { + // If none of the child doesn't have a meet + // There is no meet. + meet = None; } } - // If either of the child doesn't have a meet - // There is no meet. - if !meet_found { - meet = None; - break; - } - } + }); // All of the children have a common meet ordering. // This ordering can be propagated in Union. if let Some(meet) = meet { From a2f989af00ce56779053ffb8d2e09d21df68831d Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 5 Oct 2023 17:31:28 +0300 Subject: [PATCH 040/122] Minor changes, rename --- .../datasource/physical_plan/arrow_file.rs | 6 +- .../core/src/datasource/physical_plan/avro.rs | 4 +- .../core/src/datasource/physical_plan/csv.rs | 4 +- .../core/src/datasource/physical_plan/json.rs | 4 +- .../src/datasource/physical_plan/parquet.rs | 6 +- .../enforce_distribution.rs | 22 +- .../src/physical_optimizer/enforce_sorting.rs | 4 +- .../replace_with_order_preserving_variants.rs | 2 +- .../src/physical_optimizer/sort_pushdown.rs | 12 +- .../core/src/physical_optimizer/utils.rs | 2 +- datafusion/physical-expr/src/equivalence.rs | 234 +++++++++--------- datafusion/physical-expr/src/lib.rs | 3 +- datafusion/physical-expr/src/partitioning.rs | 14 +- datafusion/physical-expr/src/utils.rs | 26 +- .../physical-expr/src/window/built_in.rs | 7 +- .../physical-plan/src/aggregates/mod.rs | 18 +- .../physical-plan/src/coalesce_batches.rs | 6 +- datafusion/physical-plan/src/filter.rs | 8 +- .../physical-plan/src/joins/cross_join.rs | 8 +- .../physical-plan/src/joins/hash_join.rs | 8 +- .../src/joins/sort_merge_join.rs | 8 +- .../src/joins/symmetric_hash_join.rs | 8 +- datafusion/physical-plan/src/lib.rs | 8 +- datafusion/physical-plan/src/limit.rs | 10 +- datafusion/physical-plan/src/memory.rs | 14 +- datafusion/physical-plan/src/projection.rs | 8 +- .../physical-plan/src/repartition/mod.rs | 8 +- datafusion/physical-plan/src/sorts/sort.rs | 6 +- .../src/sorts/sort_preserving_merge.rs | 6 +- datafusion/physical-plan/src/streaming.rs | 8 +- datafusion/physical-plan/src/union.rs | 8 +- .../src/windows/bounded_window_agg_exec.rs | 8 +- datafusion/physical-plan/src/windows/mod.rs | 17 +- .../src/windows/window_agg_exec.rs | 6 +- 34 files changed, 254 insertions(+), 267 deletions(-) diff --git a/datafusion/core/src/datasource/physical_plan/arrow_file.rs b/datafusion/core/src/datasource/physical_plan/arrow_file.rs index a47376248ed3..0d10556d223c 100644 --- a/datafusion/core/src/datasource/physical_plan/arrow_file.rs +++ b/datafusion/core/src/datasource/physical_plan/arrow_file.rs @@ -28,8 +28,8 @@ use arrow_schema::SchemaRef; use datafusion_common::Statistics; use datafusion_execution::TaskContext; use datafusion_physical_expr::{ - ordering_equivalence_properties_helper, LexOrdering, OrderingEquivalenceProperties, - PhysicalSortExpr, + ordering_equivalence_properties_helper, LexOrdering, PhysicalSortExpr, + SchemaProperties, }; use futures::StreamExt; use object_store::{GetResultPayload, ObjectStore}; @@ -102,7 +102,7 @@ impl ExecutionPlan for ArrowExec { .map(|ordering| ordering.as_slice()) } - fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { + fn schema_properties(&self) -> SchemaProperties { ordering_equivalence_properties_helper( self.schema(), &self.projected_output_ordering, diff --git a/datafusion/core/src/datasource/physical_plan/avro.rs b/datafusion/core/src/datasource/physical_plan/avro.rs index 93655e8665f0..4c73fe55b6a0 100644 --- a/datafusion/core/src/datasource/physical_plan/avro.rs +++ b/datafusion/core/src/datasource/physical_plan/avro.rs @@ -27,7 +27,7 @@ use datafusion_execution::TaskContext; use arrow::datatypes::SchemaRef; use datafusion_physical_expr::{ - ordering_equivalence_properties_helper, LexOrdering, OrderingEquivalenceProperties, + ordering_equivalence_properties_helper, LexOrdering, SchemaProperties, }; use std::any::Any; @@ -101,7 +101,7 @@ impl ExecutionPlan for AvroExec { .map(|ordering| ordering.as_slice()) } - fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { + fn schema_properties(&self) -> SchemaProperties { ordering_equivalence_properties_helper( self.schema(), &self.projected_output_ordering, diff --git a/datafusion/core/src/datasource/physical_plan/csv.rs b/datafusion/core/src/datasource/physical_plan/csv.rs index dfc6acdde073..1c4b0f9160d1 100644 --- a/datafusion/core/src/datasource/physical_plan/csv.rs +++ b/datafusion/core/src/datasource/physical_plan/csv.rs @@ -34,7 +34,7 @@ use arrow::csv; use arrow::datatypes::SchemaRef; use datafusion_execution::TaskContext; use datafusion_physical_expr::{ - ordering_equivalence_properties_helper, LexOrdering, OrderingEquivalenceProperties, + ordering_equivalence_properties_helper, LexOrdering, SchemaProperties, }; use tokio::io::AsyncWriteExt; @@ -186,7 +186,7 @@ impl ExecutionPlan for CsvExec { .map(|ordering| ordering.as_slice()) } - fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { + fn schema_properties(&self) -> SchemaProperties { ordering_equivalence_properties_helper( self.schema(), &self.projected_output_ordering, diff --git a/datafusion/core/src/datasource/physical_plan/json.rs b/datafusion/core/src/datasource/physical_plan/json.rs index 537855704a76..0e14ca188456 100644 --- a/datafusion/core/src/datasource/physical_plan/json.rs +++ b/datafusion/core/src/datasource/physical_plan/json.rs @@ -34,7 +34,7 @@ use datafusion_execution::TaskContext; use arrow::json::ReaderBuilder; use arrow::{datatypes::SchemaRef, json}; use datafusion_physical_expr::{ - ordering_equivalence_properties_helper, LexOrdering, OrderingEquivalenceProperties, + ordering_equivalence_properties_helper, LexOrdering, SchemaProperties, }; use bytes::{Buf, Bytes}; @@ -121,7 +121,7 @@ impl ExecutionPlan for NdJsonExec { .map(|ordering| ordering.as_slice()) } - fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { + fn schema_properties(&self) -> SchemaProperties { ordering_equivalence_properties_helper( self.schema(), &self.projected_output_ordering, diff --git a/datafusion/core/src/datasource/physical_plan/parquet.rs b/datafusion/core/src/datasource/physical_plan/parquet.rs index d16c79a9692c..2b8c05c9d386 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet.rs @@ -49,9 +49,7 @@ use tokio::task::JoinSet; use arrow::datatypes::{DataType, SchemaRef}; use arrow::error::ArrowError; -use datafusion_physical_expr::{ - LexOrdering, OrderingEquivalenceProperties, PhysicalExpr, -}; +use datafusion_physical_expr::{LexOrdering, PhysicalExpr, SchemaProperties}; use bytes::Bytes; use futures::future::BoxFuture; @@ -321,7 +319,7 @@ impl ExecutionPlan for ParquetExec { .map(|ordering| ordering.as_slice()) } - fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { + fn schema_properties(&self) -> SchemaProperties { ordering_equivalence_properties_helper( self.schema(), &self.projected_output_ordering, diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index 42ac0e69dd9d..340685b411ba 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -52,7 +52,7 @@ use datafusion_expr::logical_plan::JoinType; use datafusion_physical_expr::expressions::{Column, NoOp}; use datafusion_physical_expr::utils::map_columns_before_projection; use datafusion_physical_expr::{ - expr_list_eq_strict_order, OrderingEquivalenceProperties, PhysicalExpr, + expr_list_eq_strict_order, PhysicalExpr, SchemaProperties, }; use datafusion_physical_plan::unbounded_output; @@ -444,7 +444,7 @@ where )) = try_reorder( join_key_pairs.clone(), parent_required, - &join_plan.ordering_equivalence_properties(), + &join_plan.schema_properties(), ) { if !new_positions.is_empty() { let new_join_on = new_join_conditions(&left_keys, &right_keys); @@ -685,8 +685,8 @@ pub(crate) fn reorder_join_keys_to_inputs( join_key_pairs, Some(left.output_partitioning()), Some(right.output_partitioning()), - &left.ordering_equivalence_properties(), - &right.ordering_equivalence_properties(), + &left.schema_properties(), + &right.schema_properties(), ) { if !new_positions.is_empty() { let new_join_on = new_join_conditions(&left_keys, &right_keys); @@ -729,8 +729,8 @@ pub(crate) fn reorder_join_keys_to_inputs( join_key_pairs, Some(left.output_partitioning()), Some(right.output_partitioning()), - &left.ordering_equivalence_properties(), - &right.ordering_equivalence_properties(), + &left.schema_properties(), + &right.schema_properties(), ) { if !new_positions.is_empty() { let new_join_on = new_join_conditions(&left_keys, &right_keys); @@ -762,8 +762,8 @@ fn reorder_current_join_keys( join_keys: JoinKeyPairs, left_partition: Option, right_partition: Option, - left_equivalence_properties: &OrderingEquivalenceProperties, - right_equivalence_properties: &OrderingEquivalenceProperties, + left_equivalence_properties: &SchemaProperties, + right_equivalence_properties: &SchemaProperties, ) -> Option<(JoinKeyPairs, Vec)> { match (left_partition, right_partition.clone()) { (Some(Partitioning::Hash(left_exprs, _)), _) => { @@ -788,7 +788,7 @@ fn reorder_current_join_keys( fn try_reorder( join_keys: JoinKeyPairs, expected: &[Arc], - equivalence_properties: &OrderingEquivalenceProperties, + equivalence_properties: &SchemaProperties, ) -> Option<(JoinKeyPairs, Vec)> { let eq_groups = equivalence_properties.eq_groups(); let mut normalized_expected = vec![]; @@ -1028,7 +1028,7 @@ fn add_hash_on_top( let satisfied = input .output_partitioning() .satisfy(Distribution::HashPartitioned(hash_exprs.clone()), || { - input.ordering_equivalence_properties() + input.schema_properties() }); // Add hash repartitioning when: // - The hash distribution requirement is not satisfied, or @@ -1349,7 +1349,7 @@ fn ensure_distribution( // - Ordering requirement cannot be satisfied by preserving ordering through repartitions, or // - using order preserving variant is not desirable. let ordering_satisfied = child - .ordering_equivalence_properties() + .schema_properties() .ordering_satisfy_requirement_concrete(required_input_ordering); if !ordering_satisfied || !order_preserving_variants_desirable { replace_order_preserving_variants(&mut child, dist_onward)?; diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index 29104fc87047..55aa08715124 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -450,7 +450,7 @@ fn ensure_sorting( match (required_ordering, physical_ordering) { (Some(required_ordering), Some(_)) => { if !child - .ordering_equivalence_properties() + .schema_properties() .ordering_satisfy_requirement_concrete(&required_ordering) { // Make sure we preserve the ordering requirements: @@ -513,7 +513,7 @@ fn analyze_immediate_sort_removal( // If this sort is unnecessary, we should remove it: if sort_input - .ordering_equivalence_properties() + .schema_properties() .ordering_satisfy(sort_exec.output_ordering()) { // Since we know that a `SortExec` has exactly one child, diff --git a/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs b/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs index f1c3ee265948..ee143f85dba6 100644 --- a/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs +++ b/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs @@ -258,7 +258,7 @@ pub(crate) fn replace_with_order_preserving_variants( )?; // If this sort is unnecessary, we should remove it and update the plan: if updated_sort_input - .ordering_equivalence_properties() + .schema_properties() .ordering_satisfy(plan.output_ordering()) { return Ok(Transformed::Yes(OrderPreservationContext { diff --git a/datafusion/core/src/physical_optimizer/sort_pushdown.rs b/datafusion/core/src/physical_optimizer/sort_pushdown.rs index 35920540d724..eecffc484a19 100644 --- a/datafusion/core/src/physical_optimizer/sort_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/sort_pushdown.rs @@ -128,7 +128,7 @@ pub(crate) fn pushdown_sorts( if let Some(sort_exec) = plan.as_any().downcast_ref::() { let mut new_plan = plan.clone(); if !plan - .ordering_equivalence_properties() + .schema_properties() .ordering_satisfy_requirement(parent_required) { // If the current plan is a SortExec, modify it to satisfy parent requirements: @@ -157,7 +157,7 @@ pub(crate) fn pushdown_sorts( } else { // Executors other than SortExec if plan - .ordering_equivalence_properties() + .schema_properties() .ordering_satisfy_requirement(parent_required) { // Satisfies parent requirements, immediately return. @@ -259,7 +259,7 @@ fn pushdown_requirement_to_children( } else if is_sort_preserving_merge(plan) { let ordering_req = parent_required.unwrap_or(&[]); let new_ordering = PhysicalSortRequirement::to_sort_exprs(ordering_req.to_vec()); - let mut spm_oeq = plan.ordering_equivalence_properties(); + let mut spm_oeq = plan.schema_properties(); // Sort preserving merge will have new ordering, one requirement above is pushed down to its below. spm_oeq = spm_oeq.with_reorder(new_ordering); // Do not push-down through SortPreservingMergeExec when @@ -298,13 +298,13 @@ fn determine_children_requirement( child_plan: Arc, ) -> RequirementsCompatibility { if child_plan - .ordering_equivalence_properties() + .schema_properties() .requirements_compatible(request_child, parent_required) { // request child requirements are more specific, no need to push down the parent requirements RequirementsCompatibility::Satisfy } else if child_plan - .ordering_equivalence_properties() + .schema_properties() .requirements_compatible(parent_required, request_child) { // parent requirements are more specific, adjust the request child requirements and push down the new requirements @@ -335,7 +335,7 @@ fn try_pushdown_requirements_to_join( &smj.maintains_input_order(), Some(SortMergeJoinExec::probe_side(&smj.join_type())), )?; - let mut smj_oeq = smj.ordering_equivalence_properties(); + let mut smj_oeq = smj.schema_properties(); // smj will have this ordering when its input changes. smj_oeq = smj_oeq.with_reorder(new_output_ordering.unwrap_or(vec![])); let should_pushdown = smj_oeq.ordering_satisfy_requirement(parent_required); diff --git a/datafusion/core/src/physical_optimizer/utils.rs b/datafusion/core/src/physical_optimizer/utils.rs index 9c70a208b510..0e180449112e 100644 --- a/datafusion/core/src/physical_optimizer/utils.rs +++ b/datafusion/core/src/physical_optimizer/utils.rs @@ -105,7 +105,7 @@ pub fn add_sort_above( ) -> Result<()> { // If the ordering requirement is already satisfied, do not add a sort. if !node - .ordering_equivalence_properties() + .schema_properties() .ordering_satisfy_requirement_concrete(sort_requirement) { let sort_expr = PhysicalSortRequirement::to_sort_exprs(sort_requirement.to_vec()); diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index ea51509fd4fc..11dcfcb5ece2 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -67,15 +67,11 @@ impl EquivalentGroups { self.len() == 0 } + /// Iterate over inner vector. fn iter(&self) -> impl Iterator>> { self.inner.iter() } - #[allow(dead_code)] - fn into_iter(self) -> impl Iterator>> { - self.inner.into_iter() - } - /// Adds tuple argument to the equivalent groups /// It is known that first and second entry in the tuple will have same values in the table. /// This can arise after filter(a=b), alias(a, a as b), etc. @@ -317,6 +313,31 @@ impl EquivalentGroups { } } + /// Projects EquivalentGroups according to projection mapping described in `source_to_target_mapping`. + pub fn project( + &self, + source_to_target_mapping: &ProjectionMapping, + ) -> EquivalentGroups { + let mut new_eq_classes = vec![]; + for eq_class in self.iter() { + let new_eq_class = eq_class + .iter() + .filter_map(|expr| self.project_expr(source_to_target_mapping, expr)) + .collect::>(); + if new_eq_class.len() > 1 { + new_eq_classes.push(new_eq_class.clone()); + } + } + let new_classes = + Self::calculate_new_projection_equivalent_groups(source_to_target_mapping); + new_eq_classes.extend(new_classes); + + let mut projection_eq_groups = EquivalentGroups::new(new_eq_classes); + // Make sure there is no redundant entry after projection. + projection_eq_groups.remove_redundant_entries(); + projection_eq_groups + } + /// Construct equivalent groups according to projection mapping. /// In the result, each inner vector contains equivalents sets. Outer vector corresponds to /// distinct equivalent groups @@ -345,31 +366,6 @@ impl EquivalentGroups { .collect() } - /// Projects EquivalentGroups according to projection mapping described in `source_to_target_mapping`. - pub fn project( - &self, - source_to_target_mapping: &ProjectionMapping, - ) -> EquivalentGroups { - let mut new_eq_classes = vec![]; - for eq_class in self.iter() { - let new_eq_class = eq_class - .iter() - .filter_map(|expr| self.project_expr(source_to_target_mapping, expr)) - .collect::>(); - if new_eq_class.len() > 1 { - new_eq_classes.push(new_eq_class.clone()); - } - } - let new_classes = - Self::calculate_new_projection_equivalent_groups(source_to_target_mapping); - new_eq_classes.extend(new_classes); - - let mut projection_eq_groups = EquivalentGroups::new(new_eq_classes); - // Make sure there is no redundant entry after projection. - projection_eq_groups.remove_redundant_entries(); - projection_eq_groups - } - /// Returns the equivalent group that contains `expr` /// If none of the groups contains `expr`, returns None. fn get_equivalent_group( @@ -475,6 +471,34 @@ impl OrderingEquivalentGroup { self.remove_redundant_entries(); } + /// Check whether ordering equivalent group is empty + pub fn is_empty(&self) -> bool { + self.len() == 0 + } + + pub fn iter(&self) -> impl Iterator { + self.inner.iter() + } + + fn into_iter(self) -> impl Iterator { + self.inner.into_iter() + } + + /// Get length of the entries in the ordering equivalent group + fn len(&self) -> usize { + self.inner.len() + } + + /// Extend ordering equivalent group with other group + pub fn extend(&mut self, other: OrderingEquivalentGroup) { + for ordering in other.iter() { + if !self.contains(ordering) { + self.inner.push(ordering.clone()) + } + } + self.remove_redundant_entries(); + } + /// Adds new ordering into the ordering equivalent group. pub fn add_new_orderings(&mut self, orderings: &[LexOrdering]) { for ordering in orderings.iter() { @@ -505,34 +529,6 @@ impl OrderingEquivalentGroup { self.inner = res; } - /// Check whether ordering equivalent group is empty - pub fn is_empty(&self) -> bool { - self.len() == 0 - } - - pub fn iter(&self) -> impl Iterator { - self.inner.iter() - } - - fn into_iter(self) -> impl Iterator { - self.inner.into_iter() - } - - /// Get length of the entries in the ordering equivalent group - fn len(&self) -> usize { - self.inner.len() - } - - /// Extend ordering equivalent group with other group - pub fn extend(&mut self, other: OrderingEquivalentGroup) { - for ordering in other.iter() { - if !self.contains(ordering) { - self.inner.push(ordering.clone()) - } - } - self.remove_redundant_entries(); - } - /// Get first ordering entry in the ordering equivalences /// This is one of the many valid orderings (if available) pub fn output_ordering(&self) -> Option> { @@ -596,8 +592,11 @@ impl OrderingEquivalentGroup { } } -/// `OrderingEquivalenceProperties` keeps track of columns that describe the -/// global ordering of the schema. These columns are not necessarily same; e.g. +/// `SchemaProperties` keeps track of useful information related to schema. +/// Currently, it keeps track of +/// - Equivalent columns, e.g columns that have same value. +/// - Valid ordering sort expressions for the schema. +/// Consider table below /// ```text /// ┌-------┐ /// | a | b | @@ -609,10 +608,24 @@ impl OrderingEquivalentGroup { /// └---┴---┘ /// ``` /// where both `a ASC` and `b DESC` can describe the table ordering. With -/// `OrderingEquivalenceProperties`, we can keep track of these equivalences +/// `SchemaProperties`, we can keep track of these different valid ordering expressions /// and treat `a ASC` and `b DESC` as the same ordering requirement. +/// Similarly, as in the table below if we know that Column a and b have always same value. +/// ```text +/// ┌-------┐ +/// | a | b | +/// |---|---| +/// | 1 | 1 | +/// | 2 | 2 | +/// | 3 | 3 | +/// | 5 | 5 | +/// └---┴---┘ +/// ``` +/// We keep track of their equivalence inside schema properties. With this information +/// if partition requirement is Hash(a), and output partitioning is Hash(b). We can deduce that +/// existing partitioning satisfies the requirement. #[derive(Debug, Clone)] -pub struct OrderingEquivalenceProperties { +pub struct SchemaProperties { /// Keeps track of expressions that have equivalent value. eq_groups: EquivalentGroups, /// Keeps track of valid ordering that satisfied table. @@ -624,8 +637,8 @@ pub struct OrderingEquivalenceProperties { schema: SchemaRef, } -impl OrderingEquivalenceProperties { - /// Create an empty `OrderingEquivalenceProperties` +impl SchemaProperties { + /// Create an empty `SchemaProperties` pub fn new(schema: SchemaRef) -> Self { Self { eq_groups: EquivalentGroups::empty(), @@ -635,14 +648,35 @@ impl OrderingEquivalenceProperties { } } - /// Add OrderingEquivalenceProperties of the other to the state. - pub fn extend(mut self, other: OrderingEquivalenceProperties) -> Self { + /// Get schema. + pub fn schema(&self) -> SchemaRef { + self.schema.clone() + } + + /// Return a reference to the ordering equivalent group + pub fn oeq_group(&self) -> &OrderingEquivalentGroup { + &self.oeq_group + } + + /// Return a reference to the equivalent groups + pub fn eq_groups(&self) -> &EquivalentGroups { + &self.eq_groups + } + + /// Add SchemaProperties of the other to the state. + pub fn extend(mut self, other: SchemaProperties) -> Self { self.eq_groups.extend(other.eq_groups); self.oeq_group.extend(other.oeq_group); self.with_constants(other.constants) } - /// Extends `OrderingEquivalenceProperties` by adding ordering inside the `other` + /// Empties the `oeq_group` inside self, When existing orderings are invalidated. + pub fn with_empty_ordering_equivalence(mut self) -> Self { + self.oeq_group = OrderingEquivalentGroup::empty(); + self + } + + /// Extends `SchemaProperties` by adding ordering inside the `other` /// to the `self.oeq_class`. pub fn add_ordering_equivalent_group(&mut self, other: OrderingEquivalentGroup) { for ordering in other.into_iter() { @@ -653,16 +687,6 @@ impl OrderingEquivalenceProperties { self.normalize_state(); } - /// Return a reference to the ordering equivalent group - pub fn oeq_group(&self) -> &OrderingEquivalentGroup { - &self.oeq_group - } - - /// Return a reference to the equivalent groups - pub fn eq_groups(&self) -> &EquivalentGroups { - &self.eq_groups - } - /// Adds new ordering into the ordering equivalent group. pub fn add_new_orderings(&mut self, orderings: &[LexOrdering]) { self.oeq_group.add_new_orderings(orderings); @@ -720,11 +744,6 @@ impl OrderingEquivalenceProperties { self } - /// Get schema. - pub fn schema(&self) -> SchemaRef { - self.schema.clone() - } - /// Transform `sort_exprs` vector, to standardized version using `eq_properties` and `ordering_eq_properties` /// Assume `eq_properties` states that `Column a` and `Column b` are aliases. /// Also assume `ordering_eq_properties` states that ordering `vec![d ASC]` and `vec![a ASC, c ASC]` are @@ -811,13 +830,13 @@ impl OrderingEquivalenceProperties { .any(|ordering| ordering[0].eq(leading_ordering)) } - /// Projects `OrderingEquivalenceProperties` according to mapping given in `source_to_target_mapping`. + /// Projects `SchemaProperties` according to mapping given in `source_to_target_mapping`. pub fn project( &self, source_to_target_mapping: &ProjectionMapping, output_schema: SchemaRef, - ) -> OrderingEquivalenceProperties { - let mut projected_properties = OrderingEquivalenceProperties::new(output_schema); + ) -> SchemaProperties { + let mut projected_properties = SchemaProperties::new(output_schema); let projected_eq_groups = self.eq_groups.project(source_to_target_mapping); projected_properties.eq_groups = projected_eq_groups; @@ -856,12 +875,9 @@ impl OrderingEquivalenceProperties { projected_properties } - /// Re-creates `OrderingEquivalenceProperties` given that + /// Re-creates `SchemaProperties` given that /// schema is re-ordered by `sort_expr` in the argument. - pub fn with_reorder( - mut self, - sort_expr: Vec, - ) -> OrderingEquivalenceProperties { + pub fn with_reorder(mut self, sort_expr: Vec) -> SchemaProperties { // TODO: In some cases, existing ordering equivalences may still be valid add this analysis. // Normalize sort_expr according to equivalences @@ -957,12 +973,6 @@ impl OrderingEquivalenceProperties { None } - /// Empties the `oeq_group` inside self, When existing orderings are invalidated. - pub fn with_empty_ordering_equivalence(mut self) -> OrderingEquivalenceProperties { - self.oeq_group = OrderingEquivalentGroup::empty(); - self - } - /// Checks whether given ordering requirements are satisfied by provided [PhysicalSortExpr]s. pub fn ordering_satisfy(&self, required: Option<&[PhysicalSortExpr]>) -> bool { match required { @@ -1132,14 +1142,14 @@ impl OrderingEquivalenceProperties { pub fn join( &self, join_type: &JoinType, - right: &OrderingEquivalenceProperties, + right: &SchemaProperties, join_schema: SchemaRef, maintains_input_order: &[bool], probe_side: Option, on: &[(Column, Column)], - ) -> Result { + ) -> Result { let left_columns_len = self.schema.fields.len(); - let mut new_properties = OrderingEquivalenceProperties::new(join_schema); + let mut new_properties = SchemaProperties::new(join_schema); let join_eq_groups = self.eq_groups() @@ -1214,10 +1224,10 @@ impl OrderingEquivalenceProperties { pub fn ordering_equivalence_properties_helper( schema: SchemaRef, eq_orderings: &[LexOrdering], -) -> OrderingEquivalenceProperties { - let mut oep = OrderingEquivalenceProperties::new(schema); +) -> SchemaProperties { + let mut oep = SchemaProperties::new(schema); if eq_orderings.is_empty() { - // Return an empty OrderingEquivalenceProperties: + // Return an empty `SchemaProperties`: oep } else { oep.add_ordering_equivalent_group(OrderingEquivalentGroup::new( @@ -1430,7 +1440,7 @@ pub fn add_offset_to_lex_ordering( /// the order coming from the children. pub fn update_ordering( mut node: ExprOrdering, - ordering_equal_properties: &OrderingEquivalenceProperties, + ordering_equal_properties: &SchemaProperties, ) -> Result> { let eq_groups = &ordering_equal_properties.eq_groups; let oeq_group = &ordering_equal_properties.oeq_group; @@ -1514,7 +1524,7 @@ mod tests { /// [a ASC], [d ASC, b ASC], [e DESC, f ASC, g ASC] /// and /// Column [a=c] (e.g they are aliases). - fn create_test_params() -> Result<(SchemaRef, OrderingEquivalenceProperties)> { + fn create_test_params() -> Result<(SchemaRef, SchemaProperties)> { let col_a = &Column::new("a", 0); let col_b = &Column::new("b", 1); let col_c = &Column::new("c", 2); @@ -1533,8 +1543,7 @@ mod tests { let test_schema = create_test_schema()?; let col_a_expr = Arc::new(col_a.clone()) as _; let col_c_expr = Arc::new(col_c.clone()) as _; - let mut ordering_eq_properties = - OrderingEquivalenceProperties::new(test_schema.clone()); + let mut ordering_eq_properties = SchemaProperties::new(test_schema.clone()); ordering_eq_properties.add_equal_conditions((&col_a_expr, &col_c_expr)); ordering_eq_properties.add_new_orderings(&[ vec![PhysicalSortExpr { @@ -1590,7 +1599,7 @@ mod tests { Field::new("y", DataType::Int64, true), ])); - let mut eq_properties = OrderingEquivalenceProperties::new(schema); + let mut eq_properties = SchemaProperties::new(schema); let col_a_expr = Arc::new(Column::new("a", 0)) as Arc; let col_b_expr = Arc::new(Column::new("b", 1)) as Arc; let col_c_expr = Arc::new(Column::new("c", 2)) as Arc; @@ -1647,7 +1656,7 @@ mod tests { Field::new("c", DataType::Int64, true), ])); - let mut input_properties = OrderingEquivalenceProperties::new(input_schema); + let mut input_properties = SchemaProperties::new(input_schema); let col_a_expr = Arc::new(Column::new("a", 0)) as Arc; let col_b_expr = Arc::new(Column::new("b", 1)) as Arc; let col_c_expr = Arc::new(Column::new("c", 2)) as Arc; @@ -1674,7 +1683,6 @@ mod tests { (col_a_expr.clone(), col_a3_expr.clone()), (col_a_expr.clone(), col_a4_expr.clone()), ]; - // let mut out_properties = OrderingEquivalenceProperties::new(out_schema); let out_properties = input_properties.project(&source_to_target_mapping, out_schema); @@ -1748,12 +1756,12 @@ mod tests { ]; // finer ordering satisfies, crude ordering shoul return true let empty_schema = &Arc::new(Schema::empty()); - let mut oeq_properties = OrderingEquivalenceProperties::new(empty_schema.clone()); + let mut oeq_properties = SchemaProperties::new(empty_schema.clone()); oeq_properties.oeq_group.push(finer.clone()); assert!(oeq_properties.ordering_satisfy(Some(&crude))); // Crude ordering doesn't satisfy finer ordering. should return false - let mut oeq_properties = OrderingEquivalenceProperties::new(empty_schema.clone()); + let mut oeq_properties = SchemaProperties::new(empty_schema.clone()); oeq_properties.oeq_group.push(crude.clone()); assert!(!oeq_properties.ordering_satisfy(Some(&finer))); Ok(()) @@ -1889,7 +1897,7 @@ mod tests { nulls_first: false, }; // Column a and c are aliases. - let mut ordering_eq_properties = OrderingEquivalenceProperties::new(test_schema); + let mut ordering_eq_properties = SchemaProperties::new(test_schema); ordering_eq_properties.add_equal_conditions((&col_a_expr, &col_c_expr)); // Column a and e are ordering equivalent (e.g global ordering of the table can be described both as a ASC and e ASC.) @@ -2074,7 +2082,7 @@ mod tests { let col_z_expr = col("z", &schema)?; let col_w_expr = col("w", &schema)?; - let mut join_eq_properties = OrderingEquivalenceProperties::new(Arc::new(schema)); + let mut join_eq_properties = SchemaProperties::new(Arc::new(schema)); join_eq_properties.add_equal_conditions((&col_a_expr, &col_x_expr)); join_eq_properties.add_equal_conditions((&col_d_expr, &col_w_expr)); diff --git a/datafusion/physical-expr/src/lib.rs b/datafusion/physical-expr/src/lib.rs index 104a740cb237..6e7964cc64de 100644 --- a/datafusion/physical-expr/src/lib.rs +++ b/datafusion/physical-expr/src/lib.rs @@ -57,8 +57,7 @@ pub use aggregate::groups_accumulator::{ pub use aggregate::AggregateExpr; pub use analysis::{analyze, AnalysisContext, ExprBoundaries}; pub use equivalence::{ - add_offset_to_lex_ordering, ordering_equivalence_properties_helper, - OrderingEquivalenceProperties, + add_offset_to_lex_ordering, ordering_equivalence_properties_helper, SchemaProperties, }; pub use partitioning::{Distribution, Partitioning}; diff --git a/datafusion/physical-expr/src/partitioning.rs b/datafusion/physical-expr/src/partitioning.rs index dc0f3656431f..7d8427b83ccf 100644 --- a/datafusion/physical-expr/src/partitioning.rs +++ b/datafusion/physical-expr/src/partitioning.rs @@ -20,7 +20,7 @@ use std::fmt; use std::sync::Arc; -use crate::{expr_list_eq_strict_order, OrderingEquivalenceProperties, PhysicalExpr}; +use crate::{expr_list_eq_strict_order, PhysicalExpr, SchemaProperties}; /// Partitioning schemes supported by operators. #[derive(Debug, Clone)] @@ -63,7 +63,7 @@ impl Partitioning { /// Returns true when the guarantees made by this [[Partitioning]] are sufficient to /// satisfy the partitioning scheme mandated by the `required` [[Distribution]] - pub fn satisfy OrderingEquivalenceProperties>( + pub fn satisfy SchemaProperties>( &self, required: Distribution, equal_properties: F, @@ -200,19 +200,19 @@ mod tests { for distribution in distribution_types { let result = ( single_partition.satisfy(distribution.clone(), || { - OrderingEquivalenceProperties::new(schema.clone()) + SchemaProperties::new(schema.clone()) }), unspecified_partition.satisfy(distribution.clone(), || { - OrderingEquivalenceProperties::new(schema.clone()) + SchemaProperties::new(schema.clone()) }), round_robin_partition.satisfy(distribution.clone(), || { - OrderingEquivalenceProperties::new(schema.clone()) + SchemaProperties::new(schema.clone()) }), hash_partition1.satisfy(distribution.clone(), || { - OrderingEquivalenceProperties::new(schema.clone()) + SchemaProperties::new(schema.clone()) }), hash_partition2.satisfy(distribution.clone(), || { - OrderingEquivalenceProperties::new(schema.clone()) + SchemaProperties::new(schema.clone()) }), ); diff --git a/datafusion/physical-expr/src/utils.rs b/datafusion/physical-expr/src/utils.rs index d31d78c885bf..c59508683e6d 100644 --- a/datafusion/physical-expr/src/utils.rs +++ b/datafusion/physical-expr/src/utils.rs @@ -428,7 +428,7 @@ mod tests { use std::sync::Arc; use super::*; - use crate::equivalence::{OrderingEquivalenceProperties, OrderingEquivalentGroup}; + use crate::equivalence::{OrderingEquivalentGroup, SchemaProperties}; use crate::expressions::{binary, cast, col, in_list, lit, Column, Literal}; use crate::{PhysicalSortExpr, PhysicalSortRequirement}; @@ -492,7 +492,7 @@ mod tests { Ok(schema) } - fn create_test_params() -> Result<(SchemaRef, OrderingEquivalenceProperties)> { + fn create_test_params() -> Result<(SchemaRef, SchemaProperties)> { // Assume schema satisfies ordering a ASC NULLS LAST // and d ASC NULLS LAST, b ASC NULLS LAST and e DESC NULLS FIRST, f ASC NULLS LAST, g ASC NULLS LAST // Assume that column a and c are aliases. @@ -514,8 +514,7 @@ mod tests { let test_schema = create_test_schema()?; let col_a_expr = Arc::new(col_a.clone()) as _; let col_c_expr = Arc::new(col_c.clone()) as _; - let mut ordering_eq_properties = - OrderingEquivalenceProperties::new(test_schema.clone()); + let mut ordering_eq_properties = SchemaProperties::new(test_schema.clone()); ordering_eq_properties.add_equal_conditions((&col_a_expr, &col_c_expr)); ordering_eq_properties.add_new_orderings(&[ vec![PhysicalSortExpr { @@ -1020,8 +1019,7 @@ mod tests { Field::new("a", DataType::Int32, true), Field::new("b", DataType::Int32, true), ]); - let mut ordering_equal_properties = - OrderingEquivalenceProperties::new(Arc::new(schema)); + let mut ordering_equal_properties = SchemaProperties::new(Arc::new(schema)); ordering_equal_properties.add_new_orderings(&[vec![ PhysicalSortExpr { expr: Arc::new(Column::new("b", 1)), @@ -1051,8 +1049,7 @@ mod tests { Field::new("b", DataType::Int32, true), Field::new("c", DataType::Int32, true), ]); - let mut ordering_equal_properties = - OrderingEquivalenceProperties::new(Arc::new(schema)); + let mut ordering_equal_properties = SchemaProperties::new(Arc::new(schema)); ordering_equal_properties.add_new_orderings(&[ vec![PhysicalSortExpr { expr: Arc::new(Column::new("c", 2)), @@ -1088,8 +1085,7 @@ mod tests { Field::new("b", DataType::Int32, true), Field::new("c", DataType::Int32, true), ]); - let mut ordering_equal_properties = - OrderingEquivalenceProperties::new(Arc::new(schema)); + let mut ordering_equal_properties = SchemaProperties::new(Arc::new(schema)); // not satisfied orders ordering_equal_properties.add_new_orderings(&[vec![ @@ -1126,8 +1122,7 @@ mod tests { let col_a_expr = col("a", &schema)?; let col_b_expr = col("b", &schema)?; let col_c_expr = col("c", &schema)?; - let mut equal_properties = - OrderingEquivalenceProperties::new(Arc::new(schema.clone())); + let mut equal_properties = SchemaProperties::new(Arc::new(schema.clone())); equal_properties.add_equal_conditions((&col_a_expr, &col_c_expr)); let others = vec![ @@ -1143,7 +1138,7 @@ mod tests { equal_properties .add_ordering_equivalent_group(OrderingEquivalentGroup::new(others)); - let mut expected_oeq = OrderingEquivalenceProperties::new(Arc::new(schema)); + let mut expected_oeq = SchemaProperties::new(Arc::new(schema)); expected_oeq.add_new_orderings(&[ vec![PhysicalSortExpr { expr: col_b_expr.clone(), @@ -1169,8 +1164,7 @@ mod tests { Field::new("b", DataType::Int32, true), Field::new("c", DataType::Int32, true), ]); - let mut oeq_properties = - OrderingEquivalenceProperties::new(Arc::new(schema.clone())); + let mut oeq_properties = SchemaProperties::new(Arc::new(schema.clone())); let ordering = vec![PhysicalSortExpr { expr: Arc::new(Column::new("b", 1)), options: SortOptions::default(), @@ -1210,7 +1204,7 @@ mod tests { Field::new("b", DataType::Int32, true), Field::new("c", DataType::Int32, true), ]); - let oeq_properties = OrderingEquivalenceProperties::new(Arc::new(schema.clone())); + let oeq_properties = SchemaProperties::new(Arc::new(schema.clone())); let source_to_target_mapping = vec![ ( Arc::new(Column::new("c", 2)) as _, diff --git a/datafusion/physical-expr/src/window/built_in.rs b/datafusion/physical-expr/src/window/built_in.rs index 133493452913..85262ced28fe 100644 --- a/datafusion/physical-expr/src/window/built_in.rs +++ b/datafusion/physical-expr/src/window/built_in.rs @@ -26,7 +26,7 @@ use super::WindowExpr; use crate::expressions::PhysicalSortExpr; use crate::window::window_expr::{get_orderby_values, WindowFn}; use crate::window::{PartitionBatches, PartitionWindowAggStates, WindowState}; -use crate::{reverse_order_bys, OrderingEquivalenceProperties, PhysicalExpr}; +use crate::{reverse_order_bys, PhysicalExpr, SchemaProperties}; use arrow::array::{new_empty_array, ArrayRef}; use arrow::compute::SortOptions; use arrow::datatypes::Field; @@ -74,10 +74,7 @@ impl BuiltInWindowExpr { /// If `self.expr` doesn't have an ordering, ordering equivalence properties /// are not updated. Otherwise, ordering equivalence properties are updated /// by the ordering of `self.expr`. - pub fn add_equal_orderings( - &self, - oeq_properties: &mut OrderingEquivalenceProperties, - ) { + pub fn add_equal_orderings(&self, oeq_properties: &mut SchemaProperties) { let schema = oeq_properties.schema(); if let Some(fn_res_ordering) = self.expr.get_result_ordering(&schema) { if self.partition_by.is_empty() { diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 4c4f5175f212..67d9b85c7dd9 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -34,8 +34,8 @@ use datafusion_execution::TaskContext; use datafusion_expr::Accumulator; use datafusion_physical_expr::{ expressions::Column, physical_exprs_contains, project_out_expr, reverse_order_bys, - AggregateExpr, LexOrdering, LexOrderingReq, OrderingEquivalenceProperties, - PhysicalExpr, PhysicalSortExpr, PhysicalSortRequirement, + AggregateExpr, LexOrdering, LexOrderingReq, PhysicalExpr, PhysicalSortExpr, + PhysicalSortRequirement, SchemaProperties, }; use itertools::{izip, Itertools}; @@ -327,7 +327,7 @@ fn get_init_req( /// This function gets the finest ordering requirement among all the aggregation /// functions. If requirements are conflicting, (i.e. we can not compute the /// aggregations in a single [`AggregateExec`]), the function returns an error. -fn get_finest_requirement OrderingEquivalenceProperties>( +fn get_finest_requirement SchemaProperties>( aggr_expr: &mut [Arc], order_by_expr: &mut [Option], ordering_eq_properties: F2, @@ -479,7 +479,7 @@ impl AggregateExec { .collect::>(); let requirement = get_finest_requirement(&mut aggr_expr, &mut order_by_expr, || { - input.ordering_equivalence_properties() + input.schema_properties() })?; let mut ordering_req = requirement.unwrap_or(vec![]); let partition_search_mode = get_aggregate_search_mode( @@ -518,7 +518,7 @@ impl AggregateExec { }; let aggregate_oeq = input - .ordering_equivalence_properties() + .schema_properties() .project(&source_to_target_mapping, schema.clone()); let output_ordering = aggregate_oeq.oeq_group().output_ordering(); @@ -778,9 +778,9 @@ impl ExecutionPlan for AggregateExec { vec![self.required_input_ordering.clone()] } - fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { + fn schema_properties(&self) -> SchemaProperties { self.input - .ordering_equivalence_properties() + .schema_properties() .project(&self.source_to_target_mapping, self.schema()) } @@ -1114,7 +1114,7 @@ mod tests { lit, ApproxDistinct, Column, Count, FirstValue, LastValue, Median, }; use datafusion_physical_expr::{ - AggregateExpr, OrderingEquivalenceProperties, PhysicalExpr, PhysicalSortExpr, + AggregateExpr, PhysicalExpr, PhysicalSortExpr, SchemaProperties, }; use std::any::Any; @@ -1992,7 +1992,7 @@ mod tests { let col_d = Column::new("d", 3); let col_a_expr = Arc::new(col_a.clone()) as Arc; let col_b_expr = Arc::new(col_b.clone()) as Arc; - let mut ordering_eq_properties = OrderingEquivalenceProperties::new(test_schema); + let mut ordering_eq_properties = SchemaProperties::new(test_schema); // Columns a and b are equal. ordering_eq_properties.add_equal_conditions((&col_a_expr, &col_b_expr)); // [a ASC], [c DESC] describes ordering of the schema. diff --git a/datafusion/physical-plan/src/coalesce_batches.rs b/datafusion/physical-plan/src/coalesce_batches.rs index a49de9eb5f02..3cef193efd5e 100644 --- a/datafusion/physical-plan/src/coalesce_batches.rs +++ b/datafusion/physical-plan/src/coalesce_batches.rs @@ -33,7 +33,7 @@ use arrow::error::Result as ArrowResult; use arrow::record_batch::RecordBatch; use datafusion_common::Result; use datafusion_execution::TaskContext; -use datafusion_physical_expr::OrderingEquivalenceProperties; +use datafusion_physical_expr::SchemaProperties; use super::expressions::PhysicalSortExpr; use super::metrics::{BaselineMetrics, MetricsSet}; @@ -136,8 +136,8 @@ impl ExecutionPlan for CoalesceBatchesExec { vec![false] } - fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { - self.input.ordering_equivalence_properties() + fn schema_properties(&self) -> SchemaProperties { + self.input.schema_properties() } fn with_new_children( diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index 5f5a6e1700ec..274f17f50652 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -42,8 +42,8 @@ use datafusion_execution::TaskContext; use datafusion_expr::Operator; use datafusion_physical_expr::expressions::BinaryExpr; use datafusion_physical_expr::{ - analyze, split_conjunction, AnalysisContext, ExprBoundaries, - OrderingEquivalenceProperties, PhysicalExpr, + analyze, split_conjunction, AnalysisContext, ExprBoundaries, PhysicalExpr, + SchemaProperties, }; use datafusion_physical_expr::intervals::utils::check_support; @@ -143,10 +143,10 @@ impl ExecutionPlan for FilterExec { vec![true] } - fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { + fn schema_properties(&self) -> SchemaProperties { let stats = self.statistics(); // Combine the equal predicates with the input equivalence properties - let mut filter_oeq = self.input.ordering_equivalence_properties(); + let mut filter_oeq = self.input.schema_properties(); let (equal_pairs, _ne_pairs) = collect_columns_from_predicate(&self.predicate); for (lhs, rhs) in equal_pairs { let lhs_expr = Arc::new(lhs.clone()) as _; diff --git a/datafusion/physical-plan/src/joins/cross_join.rs b/datafusion/physical-plan/src/joins/cross_join.rs index 7a79cd27d952..466299d6062e 100644 --- a/datafusion/physical-plan/src/joins/cross_join.rs +++ b/datafusion/physical-plan/src/joins/cross_join.rs @@ -37,7 +37,7 @@ use datafusion_common::{plan_err, DataFusionError, JoinType}; use datafusion_common::{Result, ScalarValue}; use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; use datafusion_execution::TaskContext; -use datafusion_physical_expr::OrderingEquivalenceProperties; +use datafusion_physical_expr::SchemaProperties; use super::utils::{ adjust_right_output_partitioning, BuildProbeJoinMetrics, OnceAsync, OnceFut, @@ -214,9 +214,9 @@ impl ExecutionPlan for CrossJoinExec { None } - fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { - let left = self.left.ordering_equivalence_properties(); - let right = self.right.ordering_equivalence_properties(); + fn schema_properties(&self) -> SchemaProperties { + let left = self.left.schema_properties(); + let right = self.right.schema_properties(); left.join( &JoinType::Full, &right, diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index 7dc86755f06b..3d5409547bce 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -67,7 +67,7 @@ use datafusion_common::{ }; use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; use datafusion_execution::TaskContext; -use datafusion_physical_expr::OrderingEquivalenceProperties; +use datafusion_physical_expr::SchemaProperties; use ahash::RandomState; use arrow::compute::kernels::cmp::{eq, not_distinct}; @@ -364,9 +364,9 @@ impl ExecutionPlan for HashJoinExec { Self::maintains_input_order(self.join_type) } - fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { - let left = self.left.ordering_equivalence_properties(); - let right = self.right.ordering_equivalence_properties(); + fn schema_properties(&self) -> SchemaProperties { + let left = self.left.schema_properties(); + let right = self.right.schema_properties(); left.join( &self.join_type, &right, diff --git a/datafusion/physical-plan/src/joins/sort_merge_join.rs b/datafusion/physical-plan/src/joins/sort_merge_join.rs index b9b1724dd8cf..e187ced569d4 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join.rs @@ -52,7 +52,7 @@ use datafusion_common::{ }; use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; use datafusion_execution::TaskContext; -use datafusion_physical_expr::{OrderingEquivalenceProperties, PhysicalSortRequirement}; +use datafusion_physical_expr::{PhysicalSortRequirement, SchemaProperties}; use futures::{Stream, StreamExt}; @@ -281,9 +281,9 @@ impl ExecutionPlan for SortMergeJoinExec { Self::maintains_input_order(self.join_type) } - fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { - let left = self.left.ordering_equivalence_properties(); - let right = self.right.ordering_equivalence_properties(); + fn schema_properties(&self) -> SchemaProperties { + let left = self.left.schema_properties(); + let right = self.right.schema_properties(); left.join( &self.join_type, &right, diff --git a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs index 98e2dd036ef2..94e385242487 100644 --- a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs +++ b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs @@ -69,7 +69,7 @@ use datafusion_physical_expr::intervals::ExprIntervalGraph; use crate::joins::utils::prepare_sorted_exprs; use ahash::RandomState; -use datafusion_physical_expr::OrderingEquivalenceProperties; +use datafusion_physical_expr::SchemaProperties; use futures::stream::{select, BoxStream}; use futures::{Stream, StreamExt}; use hashbrown::HashSet; @@ -432,9 +432,9 @@ impl ExecutionPlan for SymmetricHashJoinExec { None } - fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { - let left = self.left.ordering_equivalence_properties(); - let right = self.right.ordering_equivalence_properties(); + fn schema_properties(&self) -> SchemaProperties { + let left = self.left.schema_properties(); + let right = self.right.schema_properties(); left.join( &self.join_type, &right, diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs index d07ad839a0f6..940771e040d2 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -34,7 +34,7 @@ use arrow::record_batch::RecordBatch; use datafusion_common::utils::DataPtr; pub use datafusion_expr::Accumulator; pub use datafusion_expr::ColumnarValue; -use datafusion_physical_expr::equivalence::OrderingEquivalenceProperties; +use datafusion_physical_expr::equivalence::SchemaProperties; pub use display::{DefaultDisplay, DisplayAs, DisplayFormatType, VerboseDisplay}; use futures::stream::TryStreamExt; use std::fmt::Debug; @@ -147,9 +147,9 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { .collect() } - /// Get the OrderingEquivalenceProperties within the plan - fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { - OrderingEquivalenceProperties::new(self.schema()) + /// Get the `SchemaProperties` within the plan + fn schema_properties(&self) -> SchemaProperties { + SchemaProperties::new(self.schema()) } /// Get a list of child execution plans that provide the input for this plan. The returned list diff --git a/datafusion/physical-plan/src/limit.rs b/datafusion/physical-plan/src/limit.rs index b2d98097caf7..d15930438ead 100644 --- a/datafusion/physical-plan/src/limit.rs +++ b/datafusion/physical-plan/src/limit.rs @@ -33,7 +33,7 @@ use arrow::datatypes::SchemaRef; use arrow::record_batch::{RecordBatch, RecordBatchOptions}; use datafusion_common::{internal_err, DataFusionError, Result}; use datafusion_execution::TaskContext; -use datafusion_physical_expr::OrderingEquivalenceProperties; +use datafusion_physical_expr::SchemaProperties; use futures::stream::Stream; use futures::stream::StreamExt; @@ -133,8 +133,8 @@ impl ExecutionPlan for GlobalLimitExec { self.input.output_ordering() } - fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { - self.input.ordering_equivalence_properties() + fn schema_properties(&self) -> SchemaProperties { + self.input.schema_properties() } fn with_new_children( @@ -310,8 +310,8 @@ impl ExecutionPlan for LocalLimitExec { vec![true] } - fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { - self.input.ordering_equivalence_properties() + fn schema_properties(&self) -> SchemaProperties { + self.input.schema_properties() } fn unbounded_output(&self, _children: &[bool]) -> Result { diff --git a/datafusion/physical-plan/src/memory.rs b/datafusion/physical-plan/src/memory.rs index d07e525dd9c2..20024835f3f2 100644 --- a/datafusion/physical-plan/src/memory.rs +++ b/datafusion/physical-plan/src/memory.rs @@ -33,7 +33,7 @@ use std::task::{Context, Poll}; use crate::ordering_equivalence_properties_helper; use datafusion_common::DataFusionError; use datafusion_execution::TaskContext; -use datafusion_physical_expr::{LexOrdering, OrderingEquivalenceProperties}; +use datafusion_physical_expr::{LexOrdering, SchemaProperties}; use futures::Stream; /// Execution plan for reading in-memory batches of data @@ -120,7 +120,7 @@ impl ExecutionPlan for MemoryExec { .map(|ordering| ordering.as_slice()) } - fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { + fn schema_properties(&self) -> SchemaProperties { ordering_equivalence_properties_helper(self.schema(), &self.sort_information) } @@ -177,7 +177,7 @@ impl MemoryExec { } /// A memory table can be ordered by multiple expressions simultaneously. - /// `OrderingEquivalenceProperties` keeps track of expressions that describe the + /// `SchemaProperties` keeps track of expressions that describe the /// global ordering of the schema. These columns are not necessarily same; e.g. /// ```text /// ┌-------┐ @@ -190,10 +190,8 @@ impl MemoryExec { /// └---┴---┘ /// ``` /// where both `a ASC` and `b DESC` can describe the table ordering. With - /// `OrderingEquivalenceProperties`, we can keep track of these equivalences - /// and treat `a ASC` and `b DESC` as the same ordering requirement - /// by outputting the `a ASC` from output_ordering API - /// and add `b DESC` into `OrderingEquivalenceProperties` + /// `SchemaProperties`, we can keep track of these equivalences + /// and treat `a ASC` and `b DESC` as the same ordering requirement. pub fn with_sort_information(mut self, sort_information: Vec) -> Self { self.sort_information = sort_information; self @@ -299,7 +297,7 @@ mod tests { .with_sort_information(sort_information); assert_eq!(mem_exec.output_ordering().unwrap(), expected_output_order); - let order_eq = mem_exec.ordering_equivalence_properties(); + let order_eq = mem_exec.schema_properties(); assert!(order_eq.oeq_group().contains(&expected_order_eq)); Ok(()) } diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index 5ce17f829658..b5c924570269 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -38,7 +38,7 @@ use arrow::record_batch::{RecordBatch, RecordBatchOptions}; use datafusion_common::Result; use datafusion_execution::TaskContext; use datafusion_physical_expr::expressions::Literal; -use datafusion_physical_expr::{project_out_expr, OrderingEquivalenceProperties}; +use datafusion_physical_expr::{project_out_expr, SchemaProperties}; use crate::common::calculate_projection_mapping; use futures::stream::{Stream, StreamExt}; @@ -95,7 +95,7 @@ impl ProjectionExec { let source_to_target_mapping = calculate_projection_mapping(&expr, &input_schema)?; - let input_oeq = input.ordering_equivalence_properties(); + let input_oeq = input.schema_properties(); let project_oeq = input_oeq.project(&source_to_target_mapping, schema.clone()); let project_orderings = project_oeq.oeq_group(); let output_ordering = project_orderings.output_ordering(); @@ -196,9 +196,9 @@ impl ExecutionPlan for ProjectionExec { vec![true] } - fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { + fn schema_properties(&self) -> SchemaProperties { self.input - .ordering_equivalence_properties() + .schema_properties() .project(&self.source_to_target_mapping, self.schema()) } diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 3b1a699dcaf1..368e3b17e6ce 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -44,7 +44,7 @@ use arrow::record_batch::RecordBatch; use datafusion_common::{not_impl_err, DataFusionError, Result}; use datafusion_execution::memory_pool::MemoryConsumer; use datafusion_execution::TaskContext; -use datafusion_physical_expr::{OrderingEquivalenceProperties, PhysicalExpr}; +use datafusion_physical_expr::{PhysicalExpr, SchemaProperties}; use futures::stream::Stream; use futures::{FutureExt, StreamExt}; @@ -455,14 +455,14 @@ impl ExecutionPlan for RepartitionExec { } } - fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { + fn schema_properties(&self) -> SchemaProperties { // If ordering is not preserved, reset ordering equivalent group. if !self.maintains_input_order()[0] { self.input - .ordering_equivalence_properties() + .schema_properties() .with_empty_ordering_equivalence() } else { - self.input.ordering_equivalence_properties() + self.input.schema_properties() } } diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index 0893281545d3..66ab2da99db9 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -42,7 +42,7 @@ use datafusion_execution::memory_pool::{ }; use datafusion_execution::runtime_env::RuntimeEnv; use datafusion_execution::TaskContext; -use datafusion_physical_expr::OrderingEquivalenceProperties; +use datafusion_physical_expr::SchemaProperties; use futures::{StreamExt, TryStreamExt}; use log::{debug, error, trace}; use std::any::Any; @@ -825,8 +825,8 @@ impl ExecutionPlan for SortExec { Some(&self.expr) } - fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { - let input_oeq = self.input.ordering_equivalence_properties(); + fn schema_properties(&self) -> SchemaProperties { + let input_oeq = self.input.schema_properties(); // reset ordering equivalent group with new ordering. input_oeq.with_reorder(self.expr.to_vec()) } diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index 10b95de413e0..8d5ecd1377dc 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -33,7 +33,7 @@ use datafusion_execution::memory_pool::MemoryConsumer; use arrow::datatypes::SchemaRef; use datafusion_common::{internal_err, DataFusionError, Result}; use datafusion_execution::TaskContext; -use datafusion_physical_expr::{OrderingEquivalenceProperties, PhysicalSortRequirement}; +use datafusion_physical_expr::{PhysicalSortRequirement, SchemaProperties}; use log::{debug, trace}; @@ -170,8 +170,8 @@ impl ExecutionPlan for SortPreservingMergeExec { vec![true] } - fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { - self.input.ordering_equivalence_properties() + fn schema_properties(&self) -> SchemaProperties { + self.input.schema_properties() } fn children(&self) -> Vec> { diff --git a/datafusion/physical-plan/src/streaming.rs b/datafusion/physical-plan/src/streaming.rs index ad813cf54c8d..8555b802f3de 100644 --- a/datafusion/physical-plan/src/streaming.rs +++ b/datafusion/physical-plan/src/streaming.rs @@ -25,9 +25,7 @@ use async_trait::async_trait; use futures::stream::StreamExt; use datafusion_common::{internal_err, plan_err, DataFusionError, Result, Statistics}; -use datafusion_physical_expr::{ - LexOrdering, OrderingEquivalenceProperties, PhysicalSortExpr, -}; +use datafusion_physical_expr::{LexOrdering, PhysicalSortExpr, SchemaProperties}; use log::debug; use crate::display::{OutputOrderingDisplay, ProjectSchemaDisplay}; @@ -159,8 +157,8 @@ impl ExecutionPlan for StreamingTableExec { self.projected_output_ordering.as_deref() } - fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { - let mut oeq = OrderingEquivalenceProperties::new(self.schema()); + fn schema_properties(&self) -> SchemaProperties { + let mut oeq = SchemaProperties::new(self.schema()); if let Some(ordering) = &self.projected_output_ordering { oeq.add_new_orderings(&[ordering.clone()]) } diff --git a/datafusion/physical-plan/src/union.rs b/datafusion/physical-plan/src/union.rs index b6000c9f4859..81bf8d5b6656 100644 --- a/datafusion/physical-plan/src/union.rs +++ b/datafusion/physical-plan/src/union.rs @@ -46,7 +46,7 @@ use crate::stream::ObservedStream; use crate::{expressions, metrics::BaselineMetrics}; use datafusion_common::Result; use datafusion_execution::TaskContext; -use datafusion_physical_expr::OrderingEquivalenceProperties; +use datafusion_physical_expr::SchemaProperties; use tokio::macros::support::thread_rng_n; /// `UnionExec`: `UNION ALL` execution plan. @@ -225,13 +225,13 @@ impl ExecutionPlan for UnionExec { } } - fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { + fn schema_properties(&self) -> SchemaProperties { let child_oeqs = self .inputs .iter() - .map(|child| child.ordering_equivalence_properties()) + .map(|child| child.schema_properties()) .collect::>(); - let mut union_oeq = OrderingEquivalenceProperties::new(self.schema()); + let mut union_oeq = SchemaProperties::new(self.schema()); // Iterate ordering equivalent group of first child for elem in child_oeqs[0].oeq_group().iter() { // Seed for the meet. diff --git a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs index 026442cfead3..efe967a80d49 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -57,9 +57,7 @@ use datafusion_expr::ColumnarValue; use datafusion_physical_expr::window::{ PartitionBatches, PartitionKey, PartitionWindowAggStates, WindowState, }; -use datafusion_physical_expr::{ - OrderingEquivalenceProperties, PhysicalExpr, PhysicalSortRequirement, -}; +use datafusion_physical_expr::{PhysicalExpr, PhysicalSortRequirement, SchemaProperties}; use ahash::RandomState; use futures::stream::Stream; @@ -272,8 +270,8 @@ impl ExecutionPlan for BoundedWindowAggExec { } } - /// Get the OrderingEquivalenceProperties within the plan - fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { + /// Get the `SchemaProperties` within the plan + fn schema_properties(&self) -> SchemaProperties { window_ordering_equivalence(&self.schema, &self.input, &self.window_expr) } diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index a701700f7e9e..5080d6c8ece1 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -40,7 +40,7 @@ use datafusion_expr::{ use datafusion_physical_expr::{ reverse_order_bys, window::{BuiltInWindowFunctionExpr, SlidingAggregateWindowExpr}, - AggregateExpr, OrderingEquivalenceProperties, PhysicalSortRequirement, + AggregateExpr, PhysicalSortRequirement, SchemaProperties, }; mod bounded_window_agg_exec; @@ -325,10 +325,7 @@ pub(crate) fn get_ordered_partition_by_indices( partition_by_exprs: &[Arc], input: &Arc, ) -> Vec { - if let Some(indices) = input - .ordering_equivalence_properties() - .set_satisfy(partition_by_exprs) - { + if let Some(indices) = input.schema_properties().set_satisfy(partition_by_exprs) { indices } else { vec![] @@ -339,11 +336,11 @@ pub(crate) fn window_ordering_equivalence( schema: &SchemaRef, input: &Arc, window_expr: &[Arc], -) -> OrderingEquivalenceProperties { +) -> SchemaProperties { // We need to update the schema, so we can not directly use - // `input.ordering_equivalence_properties()`. - let mut window_oeq_properties = OrderingEquivalenceProperties::new(schema.clone()) - .extend(input.ordering_equivalence_properties()); + // `input.schema_properties()`. + let mut window_oeq_properties = + SchemaProperties::new(schema.clone()).extend(input.schema_properties()); for expr in window_expr { if let Some(builtin_window_expr) = @@ -449,7 +446,7 @@ pub fn get_window_mode( orderby_keys: &[PhysicalSortExpr], input: &Arc, ) -> Result> { - let input_oeq = input.ordering_equivalence_properties(); + let input_oeq = input.schema_properties(); let mut partition_search_mode = PartitionSearchMode::Linear; let mut partition_by_reqs: Vec = vec![]; if partitionby_exprs.is_empty() { diff --git a/datafusion/physical-plan/src/windows/window_agg_exec.rs b/datafusion/physical-plan/src/windows/window_agg_exec.rs index 492166e58c97..c8809dd67726 100644 --- a/datafusion/physical-plan/src/windows/window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/window_agg_exec.rs @@ -45,7 +45,7 @@ use arrow::{ use datafusion_common::utils::{evaluate_partition_ranges, get_at_indices}; use datafusion_common::{internal_err, plan_err, DataFusionError, Result}; use datafusion_execution::TaskContext; -use datafusion_physical_expr::{OrderingEquivalenceProperties, PhysicalSortRequirement}; +use datafusion_physical_expr::{PhysicalSortRequirement, SchemaProperties}; use futures::stream::Stream; use futures::{ready, StreamExt}; @@ -214,8 +214,8 @@ impl ExecutionPlan for WindowAggExec { } } - /// Get the OrderingEquivalenceProperties within the plan - fn ordering_equivalence_properties(&self) -> OrderingEquivalenceProperties { + /// Get the `SchemaProperties` within the plan + fn schema_properties(&self) -> SchemaProperties { window_ordering_equivalence(&self.schema, &self.input, &self.window_expr) } From 597ce82ff0c65f32c34ee628b79974c840824d45 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 6 Oct 2023 15:56:23 +0300 Subject: [PATCH 041/122] Address berkay reviews --- .../datasource/physical_plan/arrow_file.rs | 8 +- .../core/src/datasource/physical_plan/avro.rs | 9 +- .../core/src/datasource/physical_plan/csv.rs | 9 +- .../core/src/datasource/physical_plan/json.rs | 9 +- .../src/datasource/physical_plan/parquet.rs | 9 +- datafusion/physical-expr/src/equivalence.rs | 424 ++++++------------ datafusion/physical-expr/src/lib.rs | 2 +- datafusion/physical-expr/src/partitioning.rs | 16 +- datafusion/physical-expr/src/utils.rs | 47 +- .../physical-expr/src/window/built_in.rs | 12 +- .../physical-plan/src/aggregates/mod.rs | 17 +- datafusion/physical-plan/src/lib.rs | 2 +- datafusion/physical-plan/src/memory.rs | 4 +- datafusion/physical-plan/src/windows/mod.rs | 10 +- datafusion/sqllogictest/test_files/window.slt | 9 +- 15 files changed, 220 insertions(+), 367 deletions(-) diff --git a/datafusion/core/src/datasource/physical_plan/arrow_file.rs b/datafusion/core/src/datasource/physical_plan/arrow_file.rs index 0d10556d223c..2021dd93a53f 100644 --- a/datafusion/core/src/datasource/physical_plan/arrow_file.rs +++ b/datafusion/core/src/datasource/physical_plan/arrow_file.rs @@ -28,8 +28,7 @@ use arrow_schema::SchemaRef; use datafusion_common::Statistics; use datafusion_execution::TaskContext; use datafusion_physical_expr::{ - ordering_equivalence_properties_helper, LexOrdering, PhysicalSortExpr, - SchemaProperties, + schema_properties_helper, LexOrdering, PhysicalSortExpr, SchemaProperties, }; use futures::StreamExt; use object_store::{GetResultPayload, ObjectStore}; @@ -103,10 +102,7 @@ impl ExecutionPlan for ArrowExec { } fn schema_properties(&self) -> SchemaProperties { - ordering_equivalence_properties_helper( - self.schema(), - &self.projected_output_ordering, - ) + schema_properties_helper(self.schema(), &self.projected_output_ordering) } fn children(&self) -> Vec> { diff --git a/datafusion/core/src/datasource/physical_plan/avro.rs b/datafusion/core/src/datasource/physical_plan/avro.rs index 4c73fe55b6a0..6989b0ae7640 100644 --- a/datafusion/core/src/datasource/physical_plan/avro.rs +++ b/datafusion/core/src/datasource/physical_plan/avro.rs @@ -26,9 +26,7 @@ use crate::physical_plan::{ use datafusion_execution::TaskContext; use arrow::datatypes::SchemaRef; -use datafusion_physical_expr::{ - ordering_equivalence_properties_helper, LexOrdering, SchemaProperties, -}; +use datafusion_physical_expr::{schema_properties_helper, LexOrdering, SchemaProperties}; use std::any::Any; use std::sync::Arc; @@ -102,10 +100,7 @@ impl ExecutionPlan for AvroExec { } fn schema_properties(&self) -> SchemaProperties { - ordering_equivalence_properties_helper( - self.schema(), - &self.projected_output_ordering, - ) + schema_properties_helper(self.schema(), &self.projected_output_ordering) } fn children(&self) -> Vec> { diff --git a/datafusion/core/src/datasource/physical_plan/csv.rs b/datafusion/core/src/datasource/physical_plan/csv.rs index 1c4b0f9160d1..3f3651e71b72 100644 --- a/datafusion/core/src/datasource/physical_plan/csv.rs +++ b/datafusion/core/src/datasource/physical_plan/csv.rs @@ -33,9 +33,7 @@ use crate::physical_plan::{ use arrow::csv; use arrow::datatypes::SchemaRef; use datafusion_execution::TaskContext; -use datafusion_physical_expr::{ - ordering_equivalence_properties_helper, LexOrdering, SchemaProperties, -}; +use datafusion_physical_expr::{schema_properties_helper, LexOrdering, SchemaProperties}; use tokio::io::AsyncWriteExt; use super::FileScanConfig; @@ -187,10 +185,7 @@ impl ExecutionPlan for CsvExec { } fn schema_properties(&self) -> SchemaProperties { - ordering_equivalence_properties_helper( - self.schema(), - &self.projected_output_ordering, - ) + schema_properties_helper(self.schema(), &self.projected_output_ordering) } fn children(&self) -> Vec> { diff --git a/datafusion/core/src/datasource/physical_plan/json.rs b/datafusion/core/src/datasource/physical_plan/json.rs index 0e14ca188456..6ff92a7ffd56 100644 --- a/datafusion/core/src/datasource/physical_plan/json.rs +++ b/datafusion/core/src/datasource/physical_plan/json.rs @@ -33,9 +33,7 @@ use datafusion_execution::TaskContext; use arrow::json::ReaderBuilder; use arrow::{datatypes::SchemaRef, json}; -use datafusion_physical_expr::{ - ordering_equivalence_properties_helper, LexOrdering, SchemaProperties, -}; +use datafusion_physical_expr::{schema_properties_helper, LexOrdering, SchemaProperties}; use bytes::{Buf, Bytes}; use futures::{ready, stream, StreamExt, TryStreamExt}; @@ -122,10 +120,7 @@ impl ExecutionPlan for NdJsonExec { } fn schema_properties(&self) -> SchemaProperties { - ordering_equivalence_properties_helper( - self.schema(), - &self.projected_output_ordering, - ) + schema_properties_helper(self.schema(), &self.projected_output_ordering) } fn children(&self) -> Vec> { diff --git a/datafusion/core/src/datasource/physical_plan/parquet.rs b/datafusion/core/src/datasource/physical_plan/parquet.rs index 2b8c05c9d386..ce52fcf8c7a5 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet.rs @@ -36,9 +36,7 @@ use crate::{ Statistics, }, }; -use datafusion_physical_expr::{ - ordering_equivalence_properties_helper, PhysicalSortExpr, -}; +use datafusion_physical_expr::{schema_properties_helper, PhysicalSortExpr}; use fmt::Debug; use object_store::path::Path; use std::any::Any; @@ -320,10 +318,7 @@ impl ExecutionPlan for ParquetExec { } fn schema_properties(&self) -> SchemaProperties { - ordering_equivalence_properties_helper( - self.schema(), - &self.projected_output_ordering, - ) + schema_properties_helper(self.schema(), &self.projected_output_ordering) } fn with_new_children( diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 11dcfcb5ece2..9ce0e3472fc4 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -26,13 +26,12 @@ use arrow::datatypes::SchemaRef; use crate::physical_expr::{deduplicate_physical_exprs, have_common_entries}; use crate::sort_properties::{ExprOrdering, SortProperties}; -use arrow_schema::SortOptions; +use arrow_schema::{Schema, SortOptions}; use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::utils::longest_consecutive_prefix; use datafusion_common::{DataFusionError, JoinSide, JoinType, Result}; use itertools::izip; use std::hash::Hash; -use std::ops::Range; use std::sync::Arc; /// EquivalentClass is a set of [`Arc`]s that are known @@ -115,7 +114,6 @@ impl EquivalentGroups { self.inner.push(vec![first.clone(), second.clone()]); } } - self.remove_redundant_entries(); } /// Remove redundant entries from the state. @@ -318,24 +316,22 @@ impl EquivalentGroups { &self, source_to_target_mapping: &ProjectionMapping, ) -> EquivalentGroups { - let mut new_eq_classes = vec![]; + let mut projected_eq_groups = vec![]; for eq_class in self.iter() { let new_eq_class = eq_class .iter() .filter_map(|expr| self.project_expr(source_to_target_mapping, expr)) .collect::>(); if new_eq_class.len() > 1 { - new_eq_classes.push(new_eq_class.clone()); + projected_eq_groups.push(new_eq_class.clone()); } } - let new_classes = + let new_eq_groups = Self::calculate_new_projection_equivalent_groups(source_to_target_mapping); - new_eq_classes.extend(new_classes); + projected_eq_groups.extend(new_eq_groups); - let mut projection_eq_groups = EquivalentGroups::new(new_eq_classes); - // Make sure there is no redundant entry after projection. - projection_eq_groups.remove_redundant_entries(); - projection_eq_groups + // Return projected equivalent groups + EquivalentGroups::new(projected_eq_groups) } /// Construct equivalent groups according to projection mapping. @@ -684,19 +680,16 @@ impl SchemaProperties { self.oeq_group.push(ordering); } } - self.normalize_state(); } /// Adds new ordering into the ordering equivalent group. pub fn add_new_orderings(&mut self, orderings: &[LexOrdering]) { self.oeq_group.add_new_orderings(orderings); - self.normalize_state(); } /// Add new equivalent group to state. pub fn add_equivalent_groups(&mut self, other_eq_group: EquivalentGroups) { self.eq_groups.extend(other_eq_group); - self.normalize_state(); } /// Adds new equality group into the equivalent groups. @@ -706,30 +699,6 @@ impl SchemaProperties { new_conditions: (&Arc, &Arc), ) { self.eq_groups.add_equal_conditions(new_conditions); - self.normalize_state(); - } - - /// Normalizes state according to equivalent classes - /// This util makes sure that all of the entries that have have equivalent groups among the ordering equivalent group - /// uses representative expression of corresponding equivalent group (first entry in the group). - fn normalize_state(&mut self) { - let normalized_ordering = self - .oeq_group - .inner - .iter() - .map(|ordering| { - // Use a representative version of the each equivalent group inside ordering expressions. - let ordering = self.eq_groups.normalize_sort_exprs(ordering); - // Prune with constants - let req = prune_sort_reqs_with_constants( - &PhysicalSortRequirement::from_sort_exprs(&ordering), - &self.constants, - ); - PhysicalSortRequirement::to_sort_exprs(req) - }) - .collect::>>(); - // Create new oeq group normalized according to equivalent groups. - self.oeq_group = OrderingEquivalentGroup::new(normalized_ordering); } /// Add physical expression that have constant value to the `self.constants` @@ -740,17 +709,16 @@ impl SchemaProperties { self.constants.push(constant); } }); - self.normalize_state(); self } - /// Transform `sort_exprs` vector, to standardized version using `eq_properties` and `ordering_eq_properties` - /// Assume `eq_properties` states that `Column a` and `Column b` are aliases. - /// Also assume `ordering_eq_properties` states that ordering `vec![d ASC]` and `vec![a ASC, c ASC]` are + /// Transform `sort_exprs` vector, to standardized version using `eq_groups` and `oeq_group` + /// Assume `eq_groups` states that `Column a` and `Column b` are aliases. + /// Also assume `oeq_group` states that ordering `vec![d ASC]` and `vec![a ASC, c ASC]` are /// ordering equivalent (in the sense that both describe the ordering of the table). /// If the `sort_exprs` input to this function were `vec![b ASC, c ASC]`, - /// This function converts `sort_exprs` `vec![b ASC, c ASC]` to first `vec![a ASC, c ASC]` after considering `eq_properties` - /// Then converts `vec![a ASC, c ASC]` to `vec![d ASC]` after considering `ordering_eq_properties`. + /// This function converts `sort_exprs` `vec![b ASC, c ASC]` to first `vec![a ASC, c ASC]` after considering `eq_groups` + /// Then converts `vec![a ASC, c ASC]` to `vec![d ASC]` after considering `oeq_group`. /// Standardized version `vec![d ASC]` is used in subsequent operations. pub fn normalize_sort_exprs( &self, @@ -766,16 +734,17 @@ impl SchemaProperties { /// This function normalizes `sort_reqs` by /// - removing expressions that have constant value from requirement - /// - replacing sections that are in the `self.oeq_class.others` with `self.oeq_class.head` + /// - replacing sections that are in the `self.oeq_group` with `oeq_group[0]` (e.g standard representative + /// version of the group) /// - removing sections that satisfies global ordering that are in the post fix of requirement /// - /// Transform `sort_reqs` vector, to standardized version using `eq_properties` and `ordering_eq_properties` - /// Assume `eq_properties` states that `Column a` and `Column b` are aliases. - /// Also assume `ordering_eq_properties` states that ordering `vec![d ASC]` and `vec![a ASC, c ASC]` are + /// Transform `sort_reqs` vector, to standardized version using `eq_groups` and `oeq_group` + /// Assume `eq_groups` states that `Column a` and `Column b` are aliases. + /// Also assume `oeq_group` states that ordering `vec![d ASC]` and `vec![a ASC, c ASC]` are /// ordering equivalent (in the sense that both describe the ordering of the table). /// If the `sort_reqs` input to this function were `vec![b Some(ASC), c None]`, - /// This function converts `sort_exprs` `vec![b Some(ASC), c None]` to first `vec![a Some(ASC), c None]` after considering `eq_properties` - /// Then converts `vec![a Some(ASC), c None]` to `vec![d Some(ASC)]` after considering `ordering_eq_properties`. + /// This function converts `sort_exprs` `vec![b Some(ASC), c None]` to first `vec![a Some(ASC), c None]` after considering `eq_groups` + /// Then converts `vec![a Some(ASC), c None]` to `vec![d Some(ASC)]` after considering `oeq_group`. /// Standardized version `vec![d Some(ASC)]` is used in subsequent operations. pub fn normalize_sort_requirements( &self, @@ -786,37 +755,70 @@ impl SchemaProperties { let normalized_sort_reqs = prune_sort_reqs_with_constants(&normalized_sort_reqs, &self.constants); let mut normalized_sort_reqs = collapse_lex_req(normalized_sort_reqs); - if self.oeq_group.is_empty() { - return normalized_sort_reqs; + + // Prune redundant sections in the requirement. + normalized_sort_reqs = self.prune_lex_req(normalized_sort_reqs); + + // Remove duplicates from expression. + let normalized_sort_reqs = collapse_lex_req(normalized_sort_reqs); + + let oeq_group = self.oeq_group(); + if normalized_sort_reqs.is_empty() && !oeq_group.is_empty() { + // By convention use first entry + PhysicalSortRequirement::from_sort_exprs(&oeq_group.inner[0]) + } else { + normalized_sort_reqs } - let first_entry = - PhysicalSortRequirement::from_sort_exprs(&self.oeq_group.inner[0]); - let first_entry = self.eq_groups.normalize_sort_requirements(&first_entry); - for item in self.oeq_group.iter() { - let item = PhysicalSortRequirement::from_sort_exprs(item); - let item = self.eq_groups.normalize_sort_requirements(&item); - let ranges = get_compatible_ranges(&normalized_sort_reqs, &item); - let mut offset: i64 = 0; - for Range { start, end } in ranges { - let mut head = first_entry.clone(); - let updated_start = (start as i64 + offset) as usize; - let updated_end = (end as i64 + offset) as usize; - let range = end - start; - offset += head.len() as i64 - range as i64; - let all_none = normalized_sort_reqs[updated_start..updated_end] - .iter() - .all(|req| req.options.is_none()); - if all_none { - for req in head.iter_mut() { - req.options = None; - } - } - normalized_sort_reqs.splice(updated_start..updated_end, head); + } + + /// This function simplifies lexicographical ordering requirement + /// inside `sort_req` by removing postfix lexicographical requirements + /// that satisfy global ordering (occurs inside the ordering equivalent class) + fn prune_lex_req(&self, sort_req: LexOrderingReq) -> LexOrderingReq { + let mut section = &sort_req[..]; + // Eat up from the end of the sort_req until no section can be removed + // from the ending. + loop { + let n_prune = self.prune_last_n_that_is_in_oeq(section); + // Cannot prune entries from the end of requirement + if n_prune == 0 { + break; } + section = §ion[0..section.len() - n_prune]; } - normalized_sort_reqs = simplify_lex_req(normalized_sort_reqs, &self.oeq_group); + section.to_vec() + } - collapse_lex_req(normalized_sort_reqs) + /// Determines how many entries from the end can be deleted. + /// Last n entry satisfies global ordering, hence having them + /// as postfix in the lexicographical requirement is unnecessary. + /// Assume requirement is [a ASC, b ASC, c ASC], also assume that + /// existing ordering is [c ASC, d ASC]. In this case, since [c ASC] + /// is satisfied by the existing ordering (e.g corresponding section is global ordering), + /// [c ASC] can be pruned from the requirement: [a ASC, b ASC, c ASC]. In this case, + /// this function will return 1, to indicate last element can be removed from the requirement + fn prune_last_n_that_is_in_oeq(&self, sort_req: &[PhysicalSortRequirement]) -> usize { + let sort_req_len = sort_req.len(); + let oeq_group = self.oeq_group(); + let eq_groups = self.eq_groups(); + for ordering in oeq_group.iter() { + let ordering = eq_groups.normalize_sort_exprs(ordering); + let req = prune_sort_reqs_with_constants( + &PhysicalSortRequirement::from_sort_exprs(&ordering), + &self.constants, + ); + let ordering = PhysicalSortRequirement::to_sort_exprs(req); + let mut search_range = std::cmp::min(ordering.len(), sort_req_len); + while search_range > 0 { + let req_section = &sort_req[sort_req_len - search_range..]; + if req_satisfied(&ordering, req_section, &self.schema) { + return search_range; + } else { + search_range -= 1; + } + } + } + 0 } /// Checks whether `leading_ordering` is contained in any of the ordering @@ -985,7 +987,8 @@ impl SchemaProperties { /// any of the existing orderings. pub fn ordering_satisfy_concrete(&self, required: &[PhysicalSortExpr]) -> bool { let required_normalized = self.normalize_sort_exprs(required); - let provided_normalized = self.oeq_group().output_ordering().unwrap_or(vec![]); + let provided_normalized = self.oeq_group().output_ordering().unwrap_or_default(); + if required_normalized.len() > provided_normalized.len() { return false; } @@ -1093,7 +1096,7 @@ impl SchemaProperties { &self, required: &[PhysicalSortRequirement], ) -> bool { - let provided_normalized = self.oeq_group().output_ordering().unwrap_or(vec![]); + let provided_normalized = self.oeq_group().output_ordering().unwrap_or_default(); let required_normalized = self.normalize_sort_requirements(required); if required_normalized.len() > provided_normalized.len() { return false; @@ -1220,18 +1223,18 @@ impl SchemaProperties { } } -/// Retrieves the ordering equivalence properties for a given schema and output ordering. -pub fn ordering_equivalence_properties_helper( +/// Constructs a `SchemaProperties` struct from the given `orderings`. +pub fn schema_properties_helper( schema: SchemaRef, - eq_orderings: &[LexOrdering], + orderings: &[LexOrdering], ) -> SchemaProperties { let mut oep = SchemaProperties::new(schema); - if eq_orderings.is_empty() { + if orderings.is_empty() { // Return an empty `SchemaProperties`: oep } else { oep.add_ordering_equivalent_group(OrderingEquivalentGroup::new( - eq_orderings.to_vec(), + orderings.to_vec(), )); oep } @@ -1271,98 +1274,21 @@ pub fn collapse_lex_ordering(input: LexOrdering) -> LexOrdering { output } -/// This function simplifies lexicographical ordering requirement -/// inside `input` by removing postfix lexicographical requirements -/// that satisfy global ordering (occurs inside the ordering equivalent class) -fn simplify_lex_req( - input: LexOrderingReq, - oeq_class: &OrderingEquivalentGroup, -) -> LexOrderingReq { - let mut section = &input[..]; - loop { - let n_prune = prune_last_n_that_is_in_oeq(section, oeq_class); - // Cannot prune entries from the end of requirement - if n_prune == 0 { - break; - } - section = §ion[0..section.len() - n_prune]; - } - if section.is_empty() { - // By convention use first entry - PhysicalSortRequirement::from_sort_exprs(&oeq_class.inner[0]) - } else { - section.to_vec() - } -} - -/// Determines how many entries from the end can be deleted. -/// Last n entry satisfies global ordering, hence having them -/// as postfix in the lexicographical requirement is unnecessary. -/// Assume requirement is [a ASC, b ASC, c ASC], also assume that -/// existing ordering is [c ASC, d ASC]. In this case, since [c ASC] -/// is satisfied by the existing ordering (e.g corresponding section is global ordering), -/// [c ASC] can be pruned from the requirement: [a ASC, b ASC, c ASC]. In this case, -/// this function will return 1, to indicate last element can be removed from the requirement -fn prune_last_n_that_is_in_oeq( - input: &[PhysicalSortRequirement], - oeq_class: &OrderingEquivalentGroup, -) -> usize { - let input_len = input.len(); - for ordering in oeq_class.iter() { - let mut search_range = std::cmp::min(ordering.len(), input_len); - while search_range > 0 { - let req_section = &input[input_len - search_range..]; - // let given_section = &ordering[0..search_range]; - if req_satisfied(ordering, req_section) { - return search_range; - } else { - search_range -= 1; - } - } - } - 0 -} - /// Checks whether given section satisfies req. -fn req_satisfied(given: LexOrderingRef, req: &[PhysicalSortRequirement]) -> bool { +fn req_satisfied( + given: LexOrderingRef, + req: &[PhysicalSortRequirement], + schema: &Arc, +) -> bool { + // Write below code as any/all for (given, req) in izip!(given.iter(), req.iter()) { - let PhysicalSortRequirement { expr, options } = req; - if let Some(options) = options { - if options != &given.options || !expr.eq(&given.expr) { - return false; - } - } else if !expr.eq(&given.expr) { + if !given.satisfy_with_schema(req, schema) { return false; } } true } -/// This function searches for the slice `section` inside the slice `given`. -/// It returns each range where `section` is compatible with the corresponding -/// slice in `given`. -fn get_compatible_ranges( - given: &[PhysicalSortRequirement], - section: &[PhysicalSortRequirement], -) -> Vec> { - let n_section = section.len(); - let n_end = if given.len() >= n_section { - given.len() - n_section + 1 - } else { - 0 - }; - (0..n_end) - .filter_map(|idx| { - let end = idx + n_section; - given[idx..end] - .iter() - .zip(section) - .all(|(req, given)| given.compatible(req)) - .then_some(Range { start: idx, end }) - }) - .collect() -} - /// Remove ordering requirements that have constant value fn prune_sort_reqs_with_constants( ordering: &[PhysicalSortRequirement], @@ -1543,9 +1469,9 @@ mod tests { let test_schema = create_test_schema()?; let col_a_expr = Arc::new(col_a.clone()) as _; let col_c_expr = Arc::new(col_c.clone()) as _; - let mut ordering_eq_properties = SchemaProperties::new(test_schema.clone()); - ordering_eq_properties.add_equal_conditions((&col_a_expr, &col_c_expr)); - ordering_eq_properties.add_new_orderings(&[ + let mut schema_properties = SchemaProperties::new(test_schema.clone()); + schema_properties.add_equal_conditions((&col_a_expr, &col_c_expr)); + schema_properties.add_new_orderings(&[ vec![PhysicalSortExpr { expr: Arc::new(col_a.clone()), options: option1, @@ -1575,18 +1501,7 @@ mod tests { }, ], ]); - Ok((test_schema, ordering_eq_properties)) - } - - fn convert_to_requirement( - in_data: &[(&Column, Option)], - ) -> Vec { - in_data - .iter() - .map(|(col, options)| { - PhysicalSortRequirement::new(Arc::new((*col).clone()) as _, *options) - }) - .collect::>() + Ok((test_schema, schema_properties)) } #[test] @@ -1599,7 +1514,7 @@ mod tests { Field::new("y", DataType::Int64, true), ])); - let mut eq_properties = SchemaProperties::new(schema); + let mut schema_properties = SchemaProperties::new(schema); let col_a_expr = Arc::new(Column::new("a", 0)) as Arc; let col_b_expr = Arc::new(Column::new("b", 1)) as Arc; let col_c_expr = Arc::new(Column::new("c", 2)) as Arc; @@ -1607,43 +1522,43 @@ mod tests { let col_y_expr = Arc::new(Column::new("y", 4)) as Arc; let new_condition = (&col_a_expr, &col_b_expr); - eq_properties.add_equal_conditions(new_condition); - assert_eq!(eq_properties.eq_groups().len(), 1); + schema_properties.add_equal_conditions(new_condition); + assert_eq!(schema_properties.eq_groups().len(), 1); let new_condition = (&col_b_expr, &col_a_expr); - eq_properties.add_equal_conditions(new_condition); - assert_eq!(eq_properties.eq_groups().len(), 1); - let eq_class = &eq_properties.eq_groups().inner[0]; - assert_eq!(eq_class.len(), 2); - assert!(physical_exprs_contains(eq_class, &col_a_expr)); - assert!(physical_exprs_contains(eq_class, &col_b_expr)); + schema_properties.add_equal_conditions(new_condition); + assert_eq!(schema_properties.eq_groups().len(), 1); + let eq_groups = &schema_properties.eq_groups().inner[0]; + assert_eq!(eq_groups.len(), 2); + assert!(physical_exprs_contains(eq_groups, &col_a_expr)); + assert!(physical_exprs_contains(eq_groups, &col_b_expr)); let new_condition = (&col_b_expr, &col_c_expr); - eq_properties.add_equal_conditions(new_condition); - assert_eq!(eq_properties.eq_groups().len(), 1); - let eq_class = &eq_properties.eq_groups().inner[0]; - assert_eq!(eq_class.len(), 3); - assert!(physical_exprs_contains(eq_class, &col_a_expr)); - assert!(physical_exprs_contains(eq_class, &col_b_expr)); - assert!(physical_exprs_contains(eq_class, &col_c_expr)); + schema_properties.add_equal_conditions(new_condition); + assert_eq!(schema_properties.eq_groups().len(), 1); + let eq_groups = &schema_properties.eq_groups().inner[0]; + assert_eq!(eq_groups.len(), 3); + assert!(physical_exprs_contains(eq_groups, &col_a_expr)); + assert!(physical_exprs_contains(eq_groups, &col_b_expr)); + assert!(physical_exprs_contains(eq_groups, &col_c_expr)); // This is a new set of equality. Hence equivalent class count should be 2. let new_condition = (&col_x_expr, &col_y_expr); - eq_properties.add_equal_conditions(new_condition); - assert_eq!(eq_properties.eq_groups().len(), 2); + schema_properties.add_equal_conditions(new_condition); + assert_eq!(schema_properties.eq_groups().len(), 2); // This equality bridges distinct equality sets. // Hence equivalent class count should decrease from 2 to 1. let new_condition = (&col_x_expr, &col_a_expr); - eq_properties.add_equal_conditions(new_condition); - assert_eq!(eq_properties.eq_groups().len(), 1); - let eq_class = &eq_properties.eq_groups().inner[0]; - assert_eq!(eq_class.len(), 5); - assert!(physical_exprs_contains(eq_class, &col_a_expr)); - assert!(physical_exprs_contains(eq_class, &col_b_expr)); - assert!(physical_exprs_contains(eq_class, &col_c_expr)); - assert!(physical_exprs_contains(eq_class, &col_x_expr)); - assert!(physical_exprs_contains(eq_class, &col_y_expr)); + schema_properties.add_equal_conditions(new_condition); + assert_eq!(schema_properties.eq_groups().len(), 1); + let eq_groups = &schema_properties.eq_groups().inner[0]; + assert_eq!(eq_groups.len(), 5); + assert!(physical_exprs_contains(eq_groups, &col_a_expr)); + assert!(physical_exprs_contains(eq_groups, &col_b_expr)); + assert!(physical_exprs_contains(eq_groups, &col_c_expr)); + assert!(physical_exprs_contains(eq_groups, &col_x_expr)); + assert!(physical_exprs_contains(eq_groups, &col_y_expr)); Ok(()) } @@ -1697,47 +1612,6 @@ mod tests { Ok(()) } - #[test] - fn test_get_compatible_ranges() -> Result<()> { - let col_a = &Column::new("a", 0); - let col_b = &Column::new("b", 1); - let option1 = SortOptions { - descending: false, - nulls_first: false, - }; - let test_data = vec![ - ( - vec![(col_a, Some(option1)), (col_b, Some(option1))], - vec![(col_a, Some(option1))], - vec![(0, 1)], - ), - ( - vec![(col_a, None), (col_b, Some(option1))], - vec![(col_a, Some(option1))], - vec![(0, 1)], - ), - ( - vec![ - (col_a, None), - (col_b, Some(option1)), - (col_a, Some(option1)), - ], - vec![(col_a, Some(option1))], - vec![(0, 1), (2, 3)], - ), - ]; - for (searched, to_search, expected) in test_data { - let searched = convert_to_requirement(&searched); - let to_search = convert_to_requirement(&to_search); - let expected = expected - .into_iter() - .map(|(start, end)| Range { start, end }) - .collect::>(); - assert_eq!(get_compatible_ranges(&searched, &to_search), expected); - } - Ok(()) - } - #[test] fn test_ordering_satisfy() -> Result<()> { let crude = vec![PhysicalSortExpr { @@ -1756,14 +1630,14 @@ mod tests { ]; // finer ordering satisfies, crude ordering shoul return true let empty_schema = &Arc::new(Schema::empty()); - let mut oeq_properties = SchemaProperties::new(empty_schema.clone()); - oeq_properties.oeq_group.push(finer.clone()); - assert!(oeq_properties.ordering_satisfy(Some(&crude))); + let mut schema_properties = SchemaProperties::new(empty_schema.clone()); + schema_properties.oeq_group.push(finer.clone()); + assert!(schema_properties.ordering_satisfy(Some(&crude))); // Crude ordering doesn't satisfy finer ordering. should return false - let mut oeq_properties = SchemaProperties::new(empty_schema.clone()); - oeq_properties.oeq_group.push(crude.clone()); - assert!(!oeq_properties.ordering_satisfy(Some(&finer))); + let mut schema_properties = SchemaProperties::new(empty_schema.clone()); + schema_properties.oeq_group.push(crude.clone()); + assert!(!schema_properties.ordering_satisfy(Some(&finer))); Ok(()) } @@ -1785,7 +1659,7 @@ mod tests { nulls_first: true, }; // The schema is ordered by a ASC NULLS LAST, b ASC NULLS LAST - let (_test_schema, ordering_eq_properties) = create_test_params()?; + let (_test_schema, schema_properties) = create_test_params()?; // First element in the tuple stores vector of requirement, second element is the expected return value for ordering_satisfy function let requirements = vec![ // `a ASC NULLS LAST`, expects `ordering_satisfy` to be `true`, since existing ordering `a ASC NULLS LAST, b ASC NULLS LAST` satisfies it @@ -1875,7 +1749,7 @@ mod tests { let required = Some(&required[..]); assert_eq!( - ordering_eq_properties.ordering_satisfy(required), + schema_properties.ordering_satisfy(required), expected, "{err_msg}" ); @@ -1897,11 +1771,11 @@ mod tests { nulls_first: false, }; // Column a and c are aliases. - let mut ordering_eq_properties = SchemaProperties::new(test_schema); - ordering_eq_properties.add_equal_conditions((&col_a_expr, &col_c_expr)); + let mut schema_properties = SchemaProperties::new(test_schema); + schema_properties.add_equal_conditions((&col_a_expr, &col_c_expr)); // Column a and e are ordering equivalent (e.g global ordering of the table can be described both as a ASC and e ASC.) - ordering_eq_properties.add_new_orderings(&[ + schema_properties.add_new_orderings(&[ vec![PhysicalSortExpr { expr: col_a_expr.clone(), options: option1, @@ -1913,7 +1787,7 @@ mod tests { ]); // Column a and d,f are ordering equivalent (e.g global ordering of the table can be described both as [a ASC] and [d ASC, f ASC].) - ordering_eq_properties.add_new_orderings(&[ + schema_properties.add_new_orderings(&[ vec![PhysicalSortExpr { expr: col_a_expr.clone(), options: option1, @@ -1958,28 +1832,28 @@ mod tests { options: option1, }; - assert!(ordering_eq_properties.ordering_satisfy_concrete( + assert!(schema_properties.ordering_satisfy_concrete( // After normalization would be a ASC &[sort_req_c.clone(), sort_req_a.clone(), sort_req_e.clone(),], )); - assert!(!ordering_eq_properties.ordering_satisfy_concrete( + assert!(!schema_properties.ordering_satisfy_concrete( // After normalization would be a ASC, b ASC // which is not satisfied &[sort_req_c.clone(), sort_req_b.clone(),], )); - assert!(ordering_eq_properties.ordering_satisfy_concrete( + assert!(schema_properties.ordering_satisfy_concrete( // After normalization would be a ASC &[sort_req_c.clone(), sort_req_d.clone(),], )); - assert!(!ordering_eq_properties.ordering_satisfy_concrete( + assert!(!schema_properties.ordering_satisfy_concrete( // After normalization would be a ASC, b ASC // which is not satisfied &[sort_req_d.clone(), sort_req_f.clone(), sort_req_b.clone(),], )); - assert!(ordering_eq_properties.ordering_satisfy_concrete( + assert!(schema_properties.ordering_satisfy_concrete( // After normalization would be a ASC // which is satisfied &[sort_req_d.clone(), sort_req_f.clone()], @@ -2082,22 +1956,22 @@ mod tests { let col_z_expr = col("z", &schema)?; let col_w_expr = col("w", &schema)?; - let mut join_eq_properties = SchemaProperties::new(Arc::new(schema)); - join_eq_properties.add_equal_conditions((&col_a_expr, &col_x_expr)); - join_eq_properties.add_equal_conditions((&col_d_expr, &col_w_expr)); + let mut join_schema_properties = SchemaProperties::new(Arc::new(schema)); + join_schema_properties.add_equal_conditions((&col_a_expr, &col_x_expr)); + join_schema_properties.add_equal_conditions((&col_d_expr, &col_w_expr)); let result = get_updated_right_ordering_equivalent_group( &join_type, &right_oeq_class, left_columns_len, )?; - join_eq_properties.add_ordering_equivalent_group(result); - let result = join_eq_properties.oeq_group().clone(); + join_schema_properties.add_ordering_equivalent_group(result); + let result = join_schema_properties.oeq_group().clone(); let expected = OrderingEquivalentGroup::new(vec![ vec![ PhysicalSortExpr { - expr: col_a_expr, + expr: col_x_expr, options, }, PhysicalSortExpr { @@ -2111,7 +1985,7 @@ mod tests { options, }, PhysicalSortExpr { - expr: col_d_expr, + expr: col_w_expr, options, }, ], diff --git a/datafusion/physical-expr/src/lib.rs b/datafusion/physical-expr/src/lib.rs index 6e7964cc64de..7b37728871d2 100644 --- a/datafusion/physical-expr/src/lib.rs +++ b/datafusion/physical-expr/src/lib.rs @@ -57,7 +57,7 @@ pub use aggregate::groups_accumulator::{ pub use aggregate::AggregateExpr; pub use analysis::{analyze, AnalysisContext, ExprBoundaries}; pub use equivalence::{ - add_offset_to_lex_ordering, ordering_equivalence_properties_helper, SchemaProperties, + add_offset_to_lex_ordering, schema_properties_helper, SchemaProperties, }; pub use partitioning::{Distribution, Partitioning}; diff --git a/datafusion/physical-expr/src/partitioning.rs b/datafusion/physical-expr/src/partitioning.rs index 7d8427b83ccf..4e1f728d01fc 100644 --- a/datafusion/physical-expr/src/partitioning.rs +++ b/datafusion/physical-expr/src/partitioning.rs @@ -66,7 +66,7 @@ impl Partitioning { pub fn satisfy SchemaProperties>( &self, required: Distribution, - equal_properties: F, + schema_properties: F, ) -> bool { match required { Distribution::UnspecifiedDistribution => true, @@ -79,19 +79,19 @@ impl Partitioning { Partitioning::Hash(partition_exprs, _) => { let fast_match = expr_list_eq_strict_order(&required_exprs, partition_exprs); - // If the required exprs do not match, need to leverage the eq_properties provided by the child - // and normalize both exprs based on the eq_properties + // If the required exprs do not match, need to leverage the schema_properties provided by the child + // and normalize both exprs based on the equivalent groups. if !fast_match { - let eq_properties = equal_properties(); - let eq_classes = eq_properties.eq_groups(); - if !eq_classes.is_empty() { + let schema_properties = schema_properties(); + let eq_groups = schema_properties.eq_groups(); + if !eq_groups.is_empty() { let normalized_required_exprs = required_exprs .iter() - .map(|e| eq_classes.normalize_expr(e.clone())) + .map(|e| eq_groups.normalize_expr(e.clone())) .collect::>(); let normalized_partition_exprs = partition_exprs .iter() - .map(|e| eq_classes.normalize_expr(e.clone())) + .map(|e| eq_groups.normalize_expr(e.clone())) .collect::>(); expr_list_eq_strict_order( &normalized_required_exprs, diff --git a/datafusion/physical-expr/src/utils.rs b/datafusion/physical-expr/src/utils.rs index c59508683e6d..cd5b3a68536c 100644 --- a/datafusion/physical-expr/src/utils.rs +++ b/datafusion/physical-expr/src/utils.rs @@ -428,7 +428,7 @@ mod tests { use std::sync::Arc; use super::*; - use crate::equivalence::{OrderingEquivalentGroup, SchemaProperties}; + use crate::equivalence::SchemaProperties; use crate::expressions::{binary, cast, col, in_list, lit, Column, Literal}; use crate::{PhysicalSortExpr, PhysicalSortRequirement}; @@ -514,9 +514,9 @@ mod tests { let test_schema = create_test_schema()?; let col_a_expr = Arc::new(col_a.clone()) as _; let col_c_expr = Arc::new(col_c.clone()) as _; - let mut ordering_eq_properties = SchemaProperties::new(test_schema.clone()); - ordering_eq_properties.add_equal_conditions((&col_a_expr, &col_c_expr)); - ordering_eq_properties.add_new_orderings(&[ + let mut schema_properties = SchemaProperties::new(test_schema.clone()); + schema_properties.add_equal_conditions((&col_a_expr, &col_c_expr)); + schema_properties.add_new_orderings(&[ vec![PhysicalSortExpr { expr: Arc::new(col_a.clone()), options: option1, @@ -532,7 +532,7 @@ mod tests { }, ], ]); - ordering_eq_properties.add_new_orderings(&[ + schema_properties.add_new_orderings(&[ vec![PhysicalSortExpr { expr: Arc::new(col_a.clone()), options: option1, @@ -552,7 +552,7 @@ mod tests { }, ], ]); - Ok((test_schema, ordering_eq_properties)) + Ok((test_schema, schema_properties)) } #[test] @@ -765,13 +765,13 @@ mod tests { ), ]; - let (_test_schema, ordering_eq_properties) = create_test_params()?; + let (_test_schema, schema_properties) = create_test_params()?; for (reqs, expected_normalized) in requirements.into_iter() { let req = convert_to_requirement(&reqs); let expected_normalized = convert_to_requirement(&expected_normalized); assert_eq!( - ordering_eq_properties.normalize_sort_requirements(&req), + schema_properties.normalize_sort_requirements(&req), expected_normalized ); } @@ -823,7 +823,7 @@ mod tests { let _col_d = &Column::new("d", 3); let _col_e = &Column::new("e", 4); // Assume that column a and c are aliases. - let (_test_schema, ordering_eq_properties) = create_test_params()?; + let (_test_schema, schema_properties) = create_test_params()?; let col_a_expr = Arc::new(col_a.clone()) as Arc; let col_b_expr = Arc::new(col_b.clone()) as Arc; @@ -837,7 +837,7 @@ mod tests { // Cannot normalize column b (&col_b_expr, &col_b_expr), ]; - let eq_groups = ordering_eq_properties.eq_groups(); + let eq_groups = schema_properties.eq_groups(); for (expr, expected_eq) in expressions { assert!( expected_eq.eq(&eq_groups.normalize_expr(expr.clone())), @@ -855,7 +855,7 @@ mod tests { nulls_first: false, }; // Assume that column a and c are aliases. - let (test_schema, ordering_eq_properties) = create_test_params()?; + let (test_schema, schema_properties) = create_test_params()?; let col_a_expr = col("a", &test_schema)?; let _col_b_expr = col("b", &test_schema)?; let col_c_expr = col("c", &test_schema)?; @@ -910,7 +910,7 @@ mod tests { ), ]; for (arg, expected) in expressions.into_iter() { - let normalized = ordering_eq_properties.normalize_sort_requirements(&arg); + let normalized = schema_properties.normalize_sort_requirements(&arg); assert!( expected.eq(&normalized), "error in test: arg: {arg:?}, expected: {expected:?}, normalized: {normalized:?}" @@ -1122,9 +1122,9 @@ mod tests { let col_a_expr = col("a", &schema)?; let col_b_expr = col("b", &schema)?; let col_c_expr = col("c", &schema)?; - let mut equal_properties = SchemaProperties::new(Arc::new(schema.clone())); + let mut schema_properties = SchemaProperties::new(Arc::new(schema.clone())); - equal_properties.add_equal_conditions((&col_a_expr, &col_c_expr)); + schema_properties.add_equal_conditions((&col_a_expr, &col_c_expr)); let others = vec![ vec![PhysicalSortExpr { expr: col_b_expr.clone(), @@ -1135,8 +1135,7 @@ mod tests { options: sort_options, }], ]; - equal_properties - .add_ordering_equivalent_group(OrderingEquivalentGroup::new(others)); + schema_properties.add_new_orderings(&others); let mut expected_oeq = SchemaProperties::new(Arc::new(schema)); expected_oeq.add_new_orderings(&[ @@ -1145,12 +1144,12 @@ mod tests { options: sort_options, }], vec![PhysicalSortExpr { - expr: col_a_expr.clone(), + expr: col_c_expr.clone(), options: sort_options, }], ]); - let oeq_class = equal_properties.oeq_group().clone(); + let oeq_class = schema_properties.oeq_group().clone(); let expected = expected_oeq.oeq_group(); assert!(oeq_class.eq(expected)); @@ -1164,12 +1163,12 @@ mod tests { Field::new("b", DataType::Int32, true), Field::new("c", DataType::Int32, true), ]); - let mut oeq_properties = SchemaProperties::new(Arc::new(schema.clone())); + let mut schema_properties = SchemaProperties::new(Arc::new(schema.clone())); let ordering = vec![PhysicalSortExpr { expr: Arc::new(Column::new("b", 1)), options: SortOptions::default(), }]; - oeq_properties.add_new_orderings(&[ordering]); + schema_properties.add_new_orderings(&[ordering]); let source_to_target_mapping = vec![ ( Arc::new(Column::new("b", 1)) as _, @@ -1185,11 +1184,11 @@ mod tests { Field::new("a_new", DataType::Int32, true), ])); let projected_oeq = - oeq_properties.project(&source_to_target_mapping, projection_schema); + schema_properties.project(&source_to_target_mapping, projection_schema); let orderings = projected_oeq .oeq_group() .output_ordering() - .unwrap_or(vec![]); + .unwrap_or_default(); assert_eq!( vec![PhysicalSortExpr { @@ -1204,7 +1203,7 @@ mod tests { Field::new("b", DataType::Int32, true), Field::new("c", DataType::Int32, true), ]); - let oeq_properties = SchemaProperties::new(Arc::new(schema.clone())); + let schema_properties = SchemaProperties::new(Arc::new(schema.clone())); let source_to_target_mapping = vec![ ( Arc::new(Column::new("c", 2)) as _, @@ -1220,7 +1219,7 @@ mod tests { Field::new("b_new", DataType::Int32, true), ])); let projected_oeq = - oeq_properties.project(&source_to_target_mapping, projection_schema); + schema_properties.project(&source_to_target_mapping, projection_schema); // After projection there is no ordering. assert!(projected_oeq.oeq_group().output_ordering().is_none()); diff --git a/datafusion/physical-expr/src/window/built_in.rs b/datafusion/physical-expr/src/window/built_in.rs index 85262ced28fe..dab6f6c50290 100644 --- a/datafusion/physical-expr/src/window/built_in.rs +++ b/datafusion/physical-expr/src/window/built_in.rs @@ -74,12 +74,12 @@ impl BuiltInWindowExpr { /// If `self.expr` doesn't have an ordering, ordering equivalence properties /// are not updated. Otherwise, ordering equivalence properties are updated /// by the ordering of `self.expr`. - pub fn add_equal_orderings(&self, oeq_properties: &mut SchemaProperties) { - let schema = oeq_properties.schema(); + pub fn add_equal_orderings(&self, schema_properties: &mut SchemaProperties) { + let schema = schema_properties.schema(); if let Some(fn_res_ordering) = self.expr.get_result_ordering(&schema) { if self.partition_by.is_empty() { // In the absence of a PARTITION BY, ordering of `self.expr` is global: - oeq_properties.add_new_orderings(&[vec![fn_res_ordering]]); + schema_properties.add_new_orderings(&[vec![fn_res_ordering]]); } else { // If we have a PARTITION BY, built-in functions can not introduce // a global ordering unless the existing ordering is compatible @@ -88,14 +88,14 @@ impl BuiltInWindowExpr { // set equality), we can prefix the ordering of `self.expr` with // the existing ordering. if let Some(indices) = - oeq_properties.set_exactly_satisfy(&self.partition_by) + schema_properties.set_exactly_satisfy(&self.partition_by) { let lex_partition_by = indices .into_iter() .map(|idx| self.partition_by[idx].clone()) .collect::>(); if let Some(ordering_options) = - oeq_properties.get_lex_ordering(&lex_partition_by) + schema_properties.get_lex_ordering(&lex_partition_by) { let mut ordering = izip!( lex_partition_by.into_iter(), @@ -104,7 +104,7 @@ impl BuiltInWindowExpr { .map(|(expr, options)| PhysicalSortExpr { expr, options }) .collect::>(); ordering.push(fn_res_ordering); - oeq_properties.add_new_orderings(&[ordering]); + schema_properties.add_new_orderings(&[ordering]); } } } diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 67d9b85c7dd9..84651258f4e4 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -330,10 +330,10 @@ fn get_init_req( fn get_finest_requirement SchemaProperties>( aggr_expr: &mut [Arc], order_by_expr: &mut [Option], - ordering_eq_properties: F2, + schema_properties: F2, ) -> Result> { let mut finest_req = get_init_req(aggr_expr, order_by_expr); - let oeq_properties = ordering_eq_properties(); + let properties = schema_properties(); for (aggr_expr, fn_req) in aggr_expr.iter_mut().zip(order_by_expr.iter_mut()) { let fn_req = if let Some(fn_req) = fn_req { fn_req @@ -342,7 +342,7 @@ fn get_finest_requirement SchemaProperties>( }; if let Some(finest_req) = &mut finest_req { - if let Some(finer) = oeq_properties.get_finer_ordering(finest_req, fn_req) { + if let Some(finer) = properties.get_finer_ordering(finest_req, fn_req) { *finest_req = finer.to_vec(); continue; } @@ -351,7 +351,7 @@ fn get_finest_requirement SchemaProperties>( if let Some(reverse) = aggr_expr.reverse_expr() { let fn_req_reverse = reverse_order_bys(fn_req); if let Some(finer) = - oeq_properties.get_finer_ordering(finest_req, &fn_req_reverse) + properties.get_finer_ordering(finest_req, &fn_req_reverse) { // We need to update `aggr_expr` with its reverse, since only its // reverse requirement is compatible with existing requirements: @@ -1985,18 +1985,17 @@ mod tests { descending: true, nulls_first: true, }; - // let mut eq_properties = EquivalenceProperties::new(test_schema.clone()); let col_a = Column::new("a", 0); let col_b = Column::new("b", 1); let col_c = Column::new("c", 2); let col_d = Column::new("d", 3); let col_a_expr = Arc::new(col_a.clone()) as Arc; let col_b_expr = Arc::new(col_b.clone()) as Arc; - let mut ordering_eq_properties = SchemaProperties::new(test_schema); + let mut schema_properties = SchemaProperties::new(test_schema); // Columns a and b are equal. - ordering_eq_properties.add_equal_conditions((&col_a_expr, &col_b_expr)); + schema_properties.add_equal_conditions((&col_a_expr, &col_b_expr)); // [a ASC], [c DESC] describes ordering of the schema. - ordering_eq_properties.add_new_orderings(&[ + schema_properties.add_new_orderings(&[ vec![PhysicalSortExpr { expr: Arc::new(col_a.clone()) as _, options: options1, @@ -2048,7 +2047,7 @@ mod tests { )) as _; let mut aggr_exprs = vec![aggr_expr; order_by_exprs.len()]; let res = get_finest_requirement(&mut aggr_exprs, &mut order_by_exprs, || { - ordering_eq_properties.clone() + schema_properties.clone() })?; assert_eq!(res, order_by_exprs[4]); Ok(()) diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs index 940771e040d2..356b108419f1 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -374,7 +374,7 @@ pub use datafusion_common::hash_utils; pub use datafusion_common::utils::project_schema; use datafusion_execution::TaskContext; pub use datafusion_physical_expr::{ - expressions, functions, ordering_equivalence_properties_helper, udf, + expressions, functions, schema_properties_helper, udf, }; #[cfg(test)] diff --git a/datafusion/physical-plan/src/memory.rs b/datafusion/physical-plan/src/memory.rs index 20024835f3f2..a2074b7976fa 100644 --- a/datafusion/physical-plan/src/memory.rs +++ b/datafusion/physical-plan/src/memory.rs @@ -30,7 +30,7 @@ use std::any::Any; use std::sync::Arc; use std::task::{Context, Poll}; -use crate::ordering_equivalence_properties_helper; +use crate::schema_properties_helper; use datafusion_common::DataFusionError; use datafusion_execution::TaskContext; use datafusion_physical_expr::{LexOrdering, SchemaProperties}; @@ -121,7 +121,7 @@ impl ExecutionPlan for MemoryExec { } fn schema_properties(&self) -> SchemaProperties { - ordering_equivalence_properties_helper(self.schema(), &self.sort_information) + schema_properties_helper(self.schema(), &self.sort_information) } fn with_new_children( diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index 5080d6c8ece1..89c2d6dc99c8 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -339,17 +339,17 @@ pub(crate) fn window_ordering_equivalence( ) -> SchemaProperties { // We need to update the schema, so we can not directly use // `input.schema_properties()`. - let mut window_oeq_properties = + let mut window_schema_properties = SchemaProperties::new(schema.clone()).extend(input.schema_properties()); for expr in window_expr { if let Some(builtin_window_expr) = expr.as_any().downcast_ref::() { - builtin_window_expr.add_equal_orderings(&mut window_oeq_properties); + builtin_window_expr.add_equal_orderings(&mut window_schema_properties); } } - window_oeq_properties + window_schema_properties } /// Constructs the best-fitting windowing operator (a `WindowAggExec` or a @@ -1001,6 +1001,10 @@ mod tests { // ORDER BY b, a ASC NULLS FIRST (vec![], vec![("b", false, true), ("a", false, true)], None), ]; + // let test_cases = vec![ + // // PARTITION BY a, ORDER BY b ASC NULLS LAST + // (vec!["a"], vec![("b", false, false)], Some((false, Sorted))), + // ]; for (case_idx, test_case) in test_cases.iter().enumerate() { let (partition_by_columns, order_by_params, expected) = &test_case; let mut partition_by_exprs = vec![]; diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index b23cf391c82b..2d90f38b3058 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -2339,10 +2339,11 @@ Limit: skip=0, fetch=5 ----------TableScan: aggregate_test_100 projection=[c9] physical_plan GlobalLimitExec: skip=0, fetch=5 ---ProjectionExec: expr=[c9@0 as c9, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as rn1] -----BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow }], mode=[Sorted] -------SortExec: expr=[c9@0 DESC] ---------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], has_header=true +--SortExec: fetch=5, expr=[rn1@1 ASC NULLS LAST,c9@0 ASC NULLS LAST] +----ProjectionExec: expr=[c9@0 as c9, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as rn1] +------BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow }], mode=[Sorted] +--------SortExec: expr=[c9@0 DESC] +----------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/testing/data/csv/aggregate_test_100.csv]]}, projection=[c9], has_header=true query II SELECT c9, rn1 FROM (SELECT c9, From 1cb14c5e2ac300fdeb4ef86082209659c1e1b190 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 11 Oct 2023 12:58:36 +0300 Subject: [PATCH 042/122] Simplifications --- datafusion/physical-expr/src/equivalence.rs | 82 ++++++++++++++++----- 1 file changed, 65 insertions(+), 17 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 9ce0e3472fc4..15b22ea962db 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -750,18 +750,12 @@ impl SchemaProperties { &self, sort_reqs: &[PhysicalSortRequirement], ) -> Vec { - let normalized_sort_reqs = self.eq_groups.normalize_sort_requirements(sort_reqs); - // Remove entries that are known to be constant from requirement expression. - let normalized_sort_reqs = - prune_sort_reqs_with_constants(&normalized_sort_reqs, &self.constants); - let mut normalized_sort_reqs = collapse_lex_req(normalized_sort_reqs); + let mut normalized_sort_reqs = + self.eq_groups.normalize_sort_requirements(sort_reqs); // Prune redundant sections in the requirement. normalized_sort_reqs = self.prune_lex_req(normalized_sort_reqs); - // Remove duplicates from expression. - let normalized_sort_reqs = collapse_lex_req(normalized_sort_reqs); - let oeq_group = self.oeq_group(); if normalized_sort_reqs.is_empty() && !oeq_group.is_empty() { // By convention use first entry @@ -771,11 +765,44 @@ impl SchemaProperties { } } + fn satisfy_prefix(&self, sort_req: &[PhysicalSortRequirement]) -> usize { + let mut prefix_length = 0; + while prefix_length < sort_req.len() { + if self.ordering_satisfy_requirement_concrete(&sort_req[0..prefix_length + 1]) + { + prefix_length += 1; + } else { + break; + } + } + prefix_length + } + /// This function simplifies lexicographical ordering requirement /// inside `sort_req` by removing postfix lexicographical requirements /// that satisfy global ordering (occurs inside the ordering equivalent class) fn prune_lex_req(&self, sort_req: LexOrderingReq) -> LexOrderingReq { - let mut section = &sort_req[..]; + // Remove entries that are known to be constant from requirement expression. + let sort_req = prune_sort_reqs_with_constants(&sort_req, &self.constants); + let sort_req = collapse_lex_req(sort_req); + + // If empty immediately return + if sort_req.is_empty() { + return sort_req; + } + let mut new_sort_req = vec![sort_req[0].clone()]; + let mut idx = 1; + while idx < sort_req.len() { + let n_remove = self.satisfy_prefix(&sort_req[idx..]); + if n_remove == 0 { + new_sort_req.push(sort_req[idx].clone()); + idx += 1; + } else { + idx += n_remove; + } + } + + let mut section = &new_sort_req[..]; // Eat up from the end of the sort_req until no section can be removed // from the ending. loop { @@ -786,7 +813,8 @@ impl SchemaProperties { } section = §ion[0..section.len() - n_prune]; } - section.to_vec() + // Remove duplicates from expression. + collapse_lex_req(section.to_vec()) } /// Determines how many entries from the end can be deleted. @@ -821,15 +849,19 @@ impl SchemaProperties { 0 } - /// Checks whether `leading_ordering` is contained in any of the ordering + /// Checks whether `leading_requirement` is contained in any of the ordering /// equivalence classes. - pub fn satisfies_leading_ordering( + pub fn satisfies_leading_requirement( &self, - leading_ordering: &PhysicalSortExpr, + leading_requirement: &PhysicalSortRequirement, ) -> bool { - self.oeq_group() - .iter() - .any(|ordering| ordering[0].eq(leading_ordering)) + let leading_requirement = self + .eq_groups + .normalize_sort_requirement(leading_requirement.clone()); + self.oeq_group().iter().any(|ordering| { + let ordering = self.eq_groups.normalize_sort_exprs(ordering); + ordering[0].satisfy_with_schema(&leading_requirement, &self.schema) + }) } /// Projects `SchemaProperties` according to mapping given in `source_to_target_mapping`. @@ -1658,7 +1690,10 @@ mod tests { descending: true, nulls_first: true, }; - // The schema is ordered by a ASC NULLS LAST, b ASC NULLS LAST + // Schema satisfies following orderings: + // [a ASC], [d ASC, b ASC], [e DESC, f ASC, g ASC] + // and + // Column [a=c] (e.g they are aliases). let (_test_schema, schema_properties) = create_test_params()?; // First element in the tuple stores vector of requirement, second element is the expected return value for ordering_satisfy function let requirements = vec![ @@ -1735,6 +1770,19 @@ mod tests { false, ), (vec![(col_d, option1), (col_e, option2)], true), + ( + vec![(col_d, option1), (col_c, option1), (col_b, option1)], + true, + ), + ( + vec![ + (col_d, option1), + (col_e, option2), + (col_f, option1), + (col_b, option1), + ], + true, + ), ]; for (cols, expected) in requirements { From 8dafda2be73fc10c97a9885afacbd59c765bef26 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 11 Oct 2023 13:37:54 +0300 Subject: [PATCH 043/122] Add new buggy test --- datafusion/physical-expr/src/equivalence.rs | 23 +++++++++++++++++++-- 1 file changed, 21 insertions(+), 2 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 15b22ea962db..1fdd849fbbfd 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -765,7 +765,8 @@ impl SchemaProperties { } } - fn satisfy_prefix(&self, sort_req: &[PhysicalSortRequirement]) -> usize { + // Find the prefix length that is satisfied in the sort requirement + fn prefix_len_satisfied(&self, sort_req: &[PhysicalSortRequirement]) -> usize { let mut prefix_length = 0; while prefix_length < sort_req.len() { if self.ordering_satisfy_requirement_concrete(&sort_req[0..prefix_length + 1]) @@ -793,7 +794,7 @@ impl SchemaProperties { let mut new_sort_req = vec![sort_req[0].clone()]; let mut idx = 1; while idx < sort_req.len() { - let n_remove = self.satisfy_prefix(&sort_req[idx..]); + let n_remove = self.prefix_len_satisfied(&sort_req[idx..]); if n_remove == 0 { new_sort_req.push(sort_req[idx].clone()); idx += 1; @@ -1783,6 +1784,24 @@ mod tests { ], true, ), + ( + vec![ + (col_d, option1), + (col_e, option2), + (col_c, option1), + (col_b, option1), + ], + true, + ), + ( + vec![ + (col_d, option1), + (col_e, option2), + (col_b, option1), + (col_f, option1), + ], + true, + ), ]; for (cols, expected) in requirements { From e8089c20252d348ab366f012c64810e210dbb74c Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 11 Oct 2023 15:14:12 +0300 Subject: [PATCH 044/122] Add data test for sort requirement --- datafusion/physical-expr/src/equivalence.rs | 175 ++++++++++++++++++++ 1 file changed, 175 insertions(+) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 1fdd849fbbfd..e93502e607bc 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -1462,7 +1462,11 @@ mod tests { use datafusion_common::Result; use crate::physical_expr::physical_exprs_equal; + use arrow::compute::{lexsort_to_indices, SortColumn}; + use arrow_array::{ArrayRef, RecordBatch, UInt32Array, UInt64Array}; use arrow_schema::{Fields, SortOptions}; + use rand::rngs::StdRng; + use rand::{Rng, SeedableRng}; use std::sync::Arc; // Generate a schema which consists of 5 columns (a, b, c, d, e) @@ -2062,4 +2066,175 @@ mod tests { Ok(()) } + + #[test] + fn check_expected_ordering_with_data() -> Result<()> { + let input_schema = Arc::new(Schema::new(vec![ + Field::new("a", DataType::UInt64, true), + Field::new("b", DataType::UInt64, true), + Field::new("c", DataType::UInt64, true), + Field::new("d", DataType::UInt64, true), + Field::new("e", DataType::UInt64, true), + Field::new("f", DataType::UInt64, true), + ])); + let col_a_expr = Arc::new(Column::new("a", 0)) as Arc; + let col_b_expr = Arc::new(Column::new("b", 1)) as Arc; + let col_c_expr = Arc::new(Column::new("c", 2)) as Arc; + let col_d_expr = Arc::new(Column::new("d", 3)) as Arc; + let col_e_expr = Arc::new(Column::new("e", 4)) as Arc; + let col_f_expr = Arc::new(Column::new("f", 5)) as Arc; + + let col_a_expr = &col_a_expr; + let col_b_expr = &col_b_expr; + let col_c_expr = &col_c_expr; + let col_d_expr = &col_d_expr; + let col_e_expr = &col_e_expr; + let col_f_expr = &col_f_expr; + let mut schema_properties = SchemaProperties::new(input_schema.clone()); + schema_properties.add_new_orderings(&[ + vec![ + PhysicalSortExpr { + expr: col_a_expr.clone(), + options: Default::default(), + }, + PhysicalSortExpr { + expr: col_b_expr.clone(), + options: Default::default(), + }, + ], + vec![PhysicalSortExpr { + expr: col_c_expr.clone(), + options: Default::default(), + }], + vec![ + PhysicalSortExpr { + expr: col_d_expr.clone(), + options: Default::default(), + }, + PhysicalSortExpr { + expr: col_e_expr.clone(), + options: Default::default(), + }, + PhysicalSortExpr { + expr: col_f_expr.clone(), + options: Default::default(), + }, + ], + ]); + let record_batch = + generate_table_for_schema_properties(&schema_properties, 625, 5)?; + + // true means table is same after sorting, (sort was unnecessary) + // false means table changes after sorting, (sort was necessary) + let test_cases = vec![ + // a ASC + (vec![col_a_expr], true), + // a ASC, d ASC, b ASC, e ASC + (vec![col_a_expr, col_d_expr, col_b_expr, col_e_expr], true), + // a ASC, e ASC + (vec![col_a_expr, col_e_expr], false), + ]; + for (required, expected) in test_cases { + let required_ordering = required + .into_iter() + .map(|expr| PhysicalSortExpr { + expr: expr.clone(), + options: Default::default(), + }) + .collect(); + + assert_eq!( + is_table_same_after_sort(required_ordering, record_batch.clone())?, + expected + ); + } + + Ok(()) + } + + // Check whether table will stay the same after ordered according to requirement + // given. If so it means that required ordering is already satisfied (according to + // random data). + fn is_table_same_after_sort( + mut required_ordering: Vec, + batch: RecordBatch, + ) -> Result { + let schema = batch.schema(); + let n_row = batch.num_rows() as u64; + let mut sort_columns = vec![]; + let new_arr = Arc::new(UInt64Array::from_iter_values(0..n_row)) as ArrayRef; + let mut cols = batch.columns().to_vec(); + cols.push(new_arr); + let mut fields = schema.fields.to_vec(); + let new_col_expr = + Arc::new(Column::new("unique", fields.len())) as Arc; + fields.push(Arc::new(Field::new("unique", DataType::UInt64, false))); + let schema = Arc::new(Schema::new(fields)); + let batch = RecordBatch::try_new(schema.clone(), cols)?; + + // Add a unique ordering to the requirement to make resulting indices deterministic + required_ordering.push(PhysicalSortExpr { + expr: new_col_expr, + options: Default::default(), + }); + + for elem in required_ordering.into_iter() { + let (idx, _field) = schema + .column_with_name( + elem.expr.as_any().downcast_ref::().unwrap().name(), + ) + .unwrap(); + let arr = batch.column(idx); + sort_columns.push(SortColumn { + values: arr.clone(), + options: Some(elem.options), + }) + } + let indices = lexsort_to_indices(&sort_columns, None)?; + let no_change = UInt32Array::from_iter_values(0..n_row as u32); + Ok(indices == no_change) + } + + // Generate a table that satisfies schema properties, in terms of ordering equivalences. + fn generate_table_for_schema_properties( + schema_properties: &SchemaProperties, + n_elem: usize, + n_distinct: usize, + ) -> Result { + // use a random number for values + let mut rng = StdRng::seed_from_u64(23); + + let schema = schema_properties.schema(); + + let mut schema_vec = vec![None; schema.fields.len()]; + for ordering in schema_properties.oeq_group.iter() { + let mut sort_columns = vec![]; + let mut indices = vec![]; + for PhysicalSortExpr { expr, options } in ordering { + let col = expr.as_any().downcast_ref::().unwrap(); + let (idx, _field) = schema.column_with_name(col.name()).unwrap(); + let mut arr: Vec = vec![0; n_elem]; + arr.iter_mut().for_each(|v| { + *v = rng.gen_range(0..n_distinct) as u64; + }); + let arr = Arc::new(UInt64Array::from_iter_values(arr)) as ArrayRef; + sort_columns.push(SortColumn { + values: arr, + options: Some(*options), + }); + indices.push(idx); + } + let sort_arrs = arrow::compute::lexsort(&sort_columns, None)?; + for (idx, arr) in izip!(indices, sort_arrs) { + schema_vec[idx] = Some(arr); + } + } + let res = schema_vec + .into_iter() + .zip(schema.fields.iter()) + .map(|(elem, field)| (field.name(), elem.unwrap())) + .collect::>(); + let res = RecordBatch::try_from_iter(res)?; + Ok(res) + } } From 091bf80a6ac3c0d6ea441849d83a2eda1ab96619 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 11 Oct 2023 15:41:36 +0300 Subject: [PATCH 045/122] Add experimental check --- datafusion/physical-expr/src/equivalence.rs | 182 ++++++++------------ 1 file changed, 71 insertions(+), 111 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index e93502e607bc..2d784a9ea02a 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -1469,7 +1469,7 @@ mod tests { use rand::{Rng, SeedableRng}; use std::sync::Arc; - // Generate a schema which consists of 5 columns (a, b, c, d, e) + // Generate a schema which consists of 7 columns (a, b, c, d, e, f, g) fn create_test_schema() -> Result { let a = Field::new("a", DataType::Int32, true); let b = Field::new("b", DataType::Int32, true); @@ -1477,7 +1477,8 @@ mod tests { let d = Field::new("d", DataType::Int32, true); let e = Field::new("e", DataType::Int32, true); let f = Field::new("f", DataType::Int32, true); - let schema = Arc::new(Schema::new(vec![a, b, c, d, e, f])); + let g = Field::new("g", DataType::Int32, true); + let schema = Arc::new(Schema::new(vec![a, b, c, d, e, f, g])); Ok(schema) } @@ -1488,13 +1489,6 @@ mod tests { /// and /// Column [a=c] (e.g they are aliases). fn create_test_params() -> Result<(SchemaRef, SchemaProperties)> { - let col_a = &Column::new("a", 0); - let col_b = &Column::new("b", 1); - let col_c = &Column::new("c", 2); - let col_d = &Column::new("d", 3); - let col_e = &Column::new("e", 4); - let col_f = &Column::new("f", 5); - let col_g = &Column::new("g", 6); let option1 = SortOptions { descending: false, nulls_first: false, @@ -1504,36 +1498,41 @@ mod tests { nulls_first: true, }; let test_schema = create_test_schema()?; - let col_a_expr = Arc::new(col_a.clone()) as _; - let col_c_expr = Arc::new(col_c.clone()) as _; + let col_a_expr = &col("a", &test_schema)?; + let col_b_expr = &col("b", &test_schema)?; + let col_c_expr = &col("c", &test_schema)?; + let col_d_expr = &col("d", &test_schema)?; + let col_e_expr = &col("e", &test_schema)?; + let col_f_expr = &col("f", &test_schema)?; + let col_g_expr = &col("g", &test_schema)?; let mut schema_properties = SchemaProperties::new(test_schema.clone()); schema_properties.add_equal_conditions((&col_a_expr, &col_c_expr)); schema_properties.add_new_orderings(&[ vec![PhysicalSortExpr { - expr: Arc::new(col_a.clone()), + expr: col_a_expr.clone(), options: option1, }], vec![ PhysicalSortExpr { - expr: Arc::new(col_d.clone()), + expr: col_d_expr.clone(), options: option1, }, PhysicalSortExpr { - expr: Arc::new(col_b.clone()), + expr: col_b_expr.clone(), options: option1, }, ], vec![ PhysicalSortExpr { - expr: Arc::new(col_e.clone()), + expr: col_e_expr.clone(), options: option2, }, PhysicalSortExpr { - expr: Arc::new(col_f.clone()), + expr: col_f_expr.clone(), options: option1, }, PhysicalSortExpr { - expr: Arc::new(col_g.clone()), + expr: col_g_expr.clone(), options: option1, }, ], @@ -1700,6 +1699,9 @@ mod tests { // and // Column [a=c] (e.g they are aliases). let (_test_schema, schema_properties) = create_test_params()?; + let table_data_with_properties = + generate_table_for_schema_properties(&schema_properties, 625, 5)?; + // First element in the tuple stores vector of requirement, second element is the expected return value for ordering_satisfy function let requirements = vec![ // `a ASC NULLS LAST`, expects `ordering_satisfy` to be `true`, since existing ordering `a ASC NULLS LAST, b ASC NULLS LAST` satisfies it @@ -1797,15 +1799,16 @@ mod tests { ], true, ), - ( - vec![ - (col_d, option1), - (col_e, option2), - (col_b, option1), - (col_f, option1), - ], - true, - ), + // TODO: Resolve test below + // ( + // vec![ + // (col_d, option1), + // (col_e, option2), + // (col_b, option1), + // (col_f, option1), + // ], + // true, + // ), ]; for (cols, expected) in requirements { @@ -1818,6 +1821,13 @@ mod tests { }) .collect::>(); + assert_eq!( + is_table_same_after_sort( + required.clone(), + table_data_with_properties.clone() + )?, + expected + ); let required = Some(&required[..]); assert_eq!( schema_properties.ordering_satisfy(required), @@ -2067,91 +2077,6 @@ mod tests { Ok(()) } - #[test] - fn check_expected_ordering_with_data() -> Result<()> { - let input_schema = Arc::new(Schema::new(vec![ - Field::new("a", DataType::UInt64, true), - Field::new("b", DataType::UInt64, true), - Field::new("c", DataType::UInt64, true), - Field::new("d", DataType::UInt64, true), - Field::new("e", DataType::UInt64, true), - Field::new("f", DataType::UInt64, true), - ])); - let col_a_expr = Arc::new(Column::new("a", 0)) as Arc; - let col_b_expr = Arc::new(Column::new("b", 1)) as Arc; - let col_c_expr = Arc::new(Column::new("c", 2)) as Arc; - let col_d_expr = Arc::new(Column::new("d", 3)) as Arc; - let col_e_expr = Arc::new(Column::new("e", 4)) as Arc; - let col_f_expr = Arc::new(Column::new("f", 5)) as Arc; - - let col_a_expr = &col_a_expr; - let col_b_expr = &col_b_expr; - let col_c_expr = &col_c_expr; - let col_d_expr = &col_d_expr; - let col_e_expr = &col_e_expr; - let col_f_expr = &col_f_expr; - let mut schema_properties = SchemaProperties::new(input_schema.clone()); - schema_properties.add_new_orderings(&[ - vec![ - PhysicalSortExpr { - expr: col_a_expr.clone(), - options: Default::default(), - }, - PhysicalSortExpr { - expr: col_b_expr.clone(), - options: Default::default(), - }, - ], - vec![PhysicalSortExpr { - expr: col_c_expr.clone(), - options: Default::default(), - }], - vec![ - PhysicalSortExpr { - expr: col_d_expr.clone(), - options: Default::default(), - }, - PhysicalSortExpr { - expr: col_e_expr.clone(), - options: Default::default(), - }, - PhysicalSortExpr { - expr: col_f_expr.clone(), - options: Default::default(), - }, - ], - ]); - let record_batch = - generate_table_for_schema_properties(&schema_properties, 625, 5)?; - - // true means table is same after sorting, (sort was unnecessary) - // false means table changes after sorting, (sort was necessary) - let test_cases = vec![ - // a ASC - (vec![col_a_expr], true), - // a ASC, d ASC, b ASC, e ASC - (vec![col_a_expr, col_d_expr, col_b_expr, col_e_expr], true), - // a ASC, e ASC - (vec![col_a_expr, col_e_expr], false), - ]; - for (required, expected) in test_cases { - let required_ordering = required - .into_iter() - .map(|expr| PhysicalSortExpr { - expr: expr.clone(), - options: Default::default(), - }) - .collect(); - - assert_eq!( - is_table_same_after_sort(required_ordering, record_batch.clone())?, - expected - ); - } - - Ok(()) - } - // Check whether table will stay the same after ordered according to requirement // given. If so it means that required ordering is already satisfied (according to // random data). @@ -2195,6 +2120,21 @@ mod tests { Ok(indices == no_change) } + fn get_representative_arr( + eq_group: &[Arc], + existing_vec: &[Option], + schema: SchemaRef, + ) -> Option { + for expr in eq_group.iter() { + let col = expr.as_any().downcast_ref::().unwrap(); + let (idx, _field) = schema.column_with_name(col.name()).unwrap(); + if let Some(res) = &existing_vec[idx] { + return Some(res.clone()); + } + } + None + } + // Generate a table that satisfies schema properties, in terms of ordering equivalences. fn generate_table_for_schema_properties( schema_properties: &SchemaProperties, @@ -2229,6 +2169,26 @@ mod tests { schema_vec[idx] = Some(arr); } } + + for eq_group in schema_properties.eq_groups.iter() { + let arr = if let Some(arr) = + get_representative_arr(eq_group, &schema_vec, schema.clone()) + { + arr + } else { + let mut arr: Vec = vec![0; n_elem]; + arr.iter_mut().for_each(|v| { + *v = rng.gen_range(0..n_distinct) as u64; + }); + Arc::new(UInt64Array::from_iter_values(arr)) as ArrayRef + }; + for expr in eq_group { + let col = expr.as_any().downcast_ref::().unwrap(); + let (idx, _field) = schema.column_with_name(col.name()).unwrap(); + schema_vec[idx] = Some(arr.clone()); + } + } + let res = schema_vec .into_iter() .zip(schema.fields.iter()) From 85e96245f062aa6c4a5368f68bb8305fcdea5626 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 11 Oct 2023 15:55:14 +0300 Subject: [PATCH 046/122] Add random test --- datafusion/physical-expr/src/equivalence.rs | 58 +++++++++++++++++++++ 1 file changed, 58 insertions(+) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 2d784a9ea02a..a55546619c25 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -1838,6 +1838,64 @@ mod tests { Ok(()) } + #[test] + fn test_ordering_satisfy_with_equivalence_random() -> Result<()> { + // Number of random tests + let n_test = 1000usize; + let n_req_max = 5usize; + let option1 = SortOptions { + descending: false, + nulls_first: false, + }; + let option2 = SortOptions { + descending: true, + nulls_first: true, + }; + // Schema satisfies following orderings: + // [a ASC], [d ASC, b ASC], [e DESC, f ASC, g ASC] + // and + // Column [a=c] (e.g they are aliases). + let (_test_schema, schema_properties) = create_test_params()?; + let table_data_with_properties = + generate_table_for_schema_properties(&schema_properties, 625, 5)?; + + // use a random number for values + let mut rng = StdRng::seed_from_u64(23); + let n_req = rng.gen_range(0..n_req_max); + let schema = schema_properties.schema(); + let n_schema = schema.fields.len(); + for test_id in 0..n_test { + let requirement = (0..n_req) + .map(|idx| { + let col_idx = rng.gen_range(0..n_schema); + let col_expr = col(schema.fields[col_idx].name(), &schema)?; + // Choose option1 or option2 with 50 % probability. + let options = if rng.gen_range(0..1) == 0 { + option1 + } else { + option2 + }; + Ok(PhysicalSortExpr { + expr: col_expr, + options, + }) + }) + .collect::>>()?; + let err_msg = format!("Error in test case:{requirement:?}"); + let expected = is_table_same_after_sort( + requirement.clone(), + table_data_with_properties.clone(), + )?; + assert_eq!( + schema_properties.ordering_satisfy_concrete(&requirement), + expected, + "{err_msg}" + ); + } + + Ok(()) + } + #[test] fn test_ordering_satisfy_different_lengths() -> Result<()> { let test_schema = create_test_schema()?; From a94f2c7bc431d9b169c104045e1f60f2939c065a Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 11 Oct 2023 17:44:13 +0300 Subject: [PATCH 047/122] Minor changes --- datafusion/physical-expr/src/equivalence.rs | 191 +++++++++++++------- 1 file changed, 122 insertions(+), 69 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index a55546619c25..2993b0eb771c 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -1489,14 +1489,6 @@ mod tests { /// and /// Column [a=c] (e.g they are aliases). fn create_test_params() -> Result<(SchemaRef, SchemaProperties)> { - let option1 = SortOptions { - descending: false, - nulls_first: false, - }; - let option2 = SortOptions { - descending: true, - nulls_first: true, - }; let test_schema = create_test_schema()?; let col_a_expr = &col("a", &test_schema)?; let col_b_expr = &col("b", &test_schema)?; @@ -1507,6 +1499,15 @@ mod tests { let col_g_expr = &col("g", &test_schema)?; let mut schema_properties = SchemaProperties::new(test_schema.clone()); schema_properties.add_equal_conditions((&col_a_expr, &col_c_expr)); + + let option1 = SortOptions { + descending: false, + nulls_first: false, + }; + let option2 = SortOptions { + descending: true, + nulls_first: true, + }; schema_properties.add_new_orderings(&[ vec![PhysicalSortExpr { expr: col_a_expr.clone(), @@ -1540,6 +1541,68 @@ mod tests { Ok((test_schema, schema_properties)) } + // Generate a schema which consists of 6 columns (a, b, c, d, e, f) + fn create_test_schema_2() -> Result { + let a = Field::new("a", DataType::Int32, true); + let b = Field::new("b", DataType::Int32, true); + let c = Field::new("c", DataType::Int32, true); + let d = Field::new("d", DataType::Int32, true); + let e = Field::new("e", DataType::Int32, true); + let f = Field::new("f", DataType::Int32, true); + let schema = Arc::new(Schema::new(vec![a, b, c, d, e, f])); + + Ok(schema) + } + + /// Construct a schema with following properties + /// Schema satisfied following orderings: + /// [a ASC, b ASC], [c ASC, d ASC, e ASC] + /// and + /// Column [a=f] (e.g they are aliases). + fn create_test_params_2() -> Result<(SchemaRef, SchemaProperties)> { + let test_schema = create_test_schema_2()?; + let col_a_expr = &col("a", &test_schema)?; + let col_b_expr = &col("b", &test_schema)?; + let col_c_expr = &col("c", &test_schema)?; + let col_d_expr = &col("d", &test_schema)?; + let col_e_expr = &col("e", &test_schema)?; + let col_f_expr = &col("f", &test_schema)?; + let mut schema_properties = SchemaProperties::new(test_schema.clone()); + schema_properties.add_equal_conditions((&col_a_expr, &col_f_expr)); + + let option1 = SortOptions { + descending: false, + nulls_first: false, + }; + schema_properties.add_new_orderings(&[ + vec![ + PhysicalSortExpr { + expr: col_a_expr.clone(), + options: option1, + }, + PhysicalSortExpr { + expr: col_b_expr.clone(), + options: option1, + }, + ], + vec![ + PhysicalSortExpr { + expr: col_c_expr.clone(), + options: option1, + }, + PhysicalSortExpr { + expr: col_d_expr.clone(), + options: option1, + }, + PhysicalSortExpr { + expr: col_e_expr.clone(), + options: option1, + }, + ], + ]); + Ok((test_schema, schema_properties)) + } + #[test] fn add_equal_conditions_test() -> Result<()> { let schema = Arc::new(Schema::new(vec![ @@ -1679,13 +1742,18 @@ mod tests { #[test] fn test_ordering_satisfy_with_equivalence() -> Result<()> { - let col_a = &Column::new("a", 0); - let col_b = &Column::new("b", 1); - let col_c = &Column::new("c", 2); - let col_d = &Column::new("d", 3); - let col_e = &Column::new("e", 4); - let col_f = &Column::new("f", 5); - let col_g = &Column::new("g", 6); + // Schema satisfies following orderings: + // [a ASC], [d ASC, b ASC], [e DESC, f ASC, g ASC] + // and + // Column [a=c] (e.g they are aliases). + let (test_schema, schema_properties) = create_test_params()?; + let col_a = &col("a", &test_schema)?; + let col_b = &col("b", &test_schema)?; + let col_c = &col("c", &test_schema)?; + let col_d = &col("d", &test_schema)?; + let col_e = &col("e", &test_schema)?; + let col_f = &col("f", &test_schema)?; + let col_g = &col("g", &test_schema)?; let option1 = SortOptions { descending: false, nulls_first: false, @@ -1694,11 +1762,6 @@ mod tests { descending: true, nulls_first: true, }; - // Schema satisfies following orderings: - // [a ASC], [d ASC, b ASC], [e DESC, f ASC, g ASC] - // and - // Column [a=c] (e.g they are aliases). - let (_test_schema, schema_properties) = create_test_params()?; let table_data_with_properties = generate_table_for_schema_properties(&schema_properties, 625, 5)?; @@ -1815,8 +1878,8 @@ mod tests { let err_msg = format!("Error in test case:{cols:?}"); let required = cols .into_iter() - .map(|(col, options)| PhysicalSortExpr { - expr: Arc::new(col.clone()), + .map(|(expr, options)| PhysicalSortExpr { + expr: expr.clone(), options, }) .collect::>(); @@ -1842,20 +1905,16 @@ mod tests { fn test_ordering_satisfy_with_equivalence_random() -> Result<()> { // Number of random tests let n_test = 1000usize; - let n_req_max = 5usize; + let n_req_max = 3usize; let option1 = SortOptions { descending: false, nulls_first: false, }; - let option2 = SortOptions { - descending: true, - nulls_first: true, - }; // Schema satisfies following orderings: - // [a ASC], [d ASC, b ASC], [e DESC, f ASC, g ASC] + // [a ASC, b ASC], [c ASC, d ASC, e ASC] // and - // Column [a=c] (e.g they are aliases). - let (_test_schema, schema_properties) = create_test_params()?; + // Column [a=f] (e.g they are aliases). + let (_test_schema, schema_properties) = create_test_params_2()?; let table_data_with_properties = generate_table_for_schema_properties(&schema_properties, 625, 5)?; @@ -1864,20 +1923,14 @@ mod tests { let n_req = rng.gen_range(0..n_req_max); let schema = schema_properties.schema(); let n_schema = schema.fields.len(); - for test_id in 0..n_test { + for _test_id in 0..n_test { let requirement = (0..n_req) - .map(|idx| { + .map(|_idx| { let col_idx = rng.gen_range(0..n_schema); let col_expr = col(schema.fields[col_idx].name(), &schema)?; - // Choose option1 or option2 with 50 % probability. - let options = if rng.gen_range(0..1) == 0 { - option1 - } else { - option2 - }; Ok(PhysicalSortExpr { expr: col_expr, - options, + options: option1, }) }) .collect::>>()?; @@ -1899,46 +1952,46 @@ mod tests { #[test] fn test_ordering_satisfy_different_lengths() -> Result<()> { let test_schema = create_test_schema()?; - let col_a_expr = col("a", &test_schema)?; - let col_b_expr = col("b", &test_schema)?; - let col_c_expr = col("c", &test_schema)?; - let col_d_expr = col("d", &test_schema)?; - let col_e_expr = col("e", &test_schema)?; - let col_f_expr = col("f", &test_schema)?; - let option1 = SortOptions { + let col_a = col("a", &test_schema)?; + let col_b = col("b", &test_schema)?; + let col_c = col("c", &test_schema)?; + let col_d = col("d", &test_schema)?; + let col_e = col("e", &test_schema)?; + let col_f = col("f", &test_schema)?; + let options = SortOptions { descending: false, nulls_first: false, }; // Column a and c are aliases. let mut schema_properties = SchemaProperties::new(test_schema); - schema_properties.add_equal_conditions((&col_a_expr, &col_c_expr)); + schema_properties.add_equal_conditions((&col_a, &col_c)); // Column a and e are ordering equivalent (e.g global ordering of the table can be described both as a ASC and e ASC.) schema_properties.add_new_orderings(&[ vec![PhysicalSortExpr { - expr: col_a_expr.clone(), - options: option1, + expr: col_a.clone(), + options, }], vec![PhysicalSortExpr { - expr: col_e_expr.clone(), - options: option1, + expr: col_e.clone(), + options, }], ]); // Column a and d,f are ordering equivalent (e.g global ordering of the table can be described both as [a ASC] and [d ASC, f ASC].) schema_properties.add_new_orderings(&[ vec![PhysicalSortExpr { - expr: col_a_expr.clone(), - options: option1, + expr: col_a.clone(), + options, }], vec![ PhysicalSortExpr { - expr: col_d_expr.clone(), - options: option1, + expr: col_d.clone(), + options, }, PhysicalSortExpr { - expr: col_f_expr.clone(), - options: option1, + expr: col_f.clone(), + options, }, ], ]); @@ -1947,28 +2000,28 @@ mod tests { // Also Columns a and c are equal let sort_req_a = PhysicalSortExpr { - expr: col_a_expr.clone(), - options: option1, + expr: col_a.clone(), + options, }; let sort_req_b = PhysicalSortExpr { - expr: col_b_expr.clone(), - options: option1, + expr: col_b.clone(), + options, }; let sort_req_c = PhysicalSortExpr { - expr: col_c_expr.clone(), - options: option1, + expr: col_c.clone(), + options, }; let sort_req_d = PhysicalSortExpr { - expr: col_d_expr.clone(), - options: option1, + expr: col_d.clone(), + options, }; let sort_req_e = PhysicalSortExpr { - expr: col_e_expr.clone(), - options: option1, + expr: col_e.clone(), + options, }; let sort_req_f = PhysicalSortExpr { - expr: col_f_expr.clone(), - options: option1, + expr: col_f.clone(), + options, }; assert!(schema_properties.ordering_satisfy_concrete( From 36c4835ff3709876a7725446275df4f8ae92b3f8 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 11 Oct 2023 18:02:02 +0300 Subject: [PATCH 048/122] Random test gives error --- datafusion/physical-expr/src/equivalence.rs | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 2993b0eb771c..62572ce4e6bf 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -1904,8 +1904,8 @@ mod tests { #[test] fn test_ordering_satisfy_with_equivalence_random() -> Result<()> { // Number of random tests - let n_test = 1000usize; - let n_req_max = 3usize; + let n_test = 10000usize; + let n_req_max = 5usize; let option1 = SortOptions { descending: false, nulls_first: false, @@ -1920,10 +1920,10 @@ mod tests { // use a random number for values let mut rng = StdRng::seed_from_u64(23); - let n_req = rng.gen_range(0..n_req_max); let schema = schema_properties.schema(); let n_schema = schema.fields.len(); for _test_id in 0..n_test { + let n_req = rng.gen_range(0..n_req_max); let requirement = (0..n_req) .map(|_idx| { let col_idx = rng.gen_range(0..n_schema); From aef6dc1f8265fff17fdb329125b73bbcdb9a7efc Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 11 Oct 2023 18:26:17 +0300 Subject: [PATCH 049/122] Fix missing test case --- datafusion/physical-expr/src/equivalence.rs | 64 +++++++++++++++++---- 1 file changed, 53 insertions(+), 11 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 62572ce4e6bf..6aa8b15e4f6b 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -785,12 +785,55 @@ impl SchemaProperties { fn prune_lex_req(&self, sort_req: LexOrderingReq) -> LexOrderingReq { // Remove entries that are known to be constant from requirement expression. let sort_req = prune_sort_reqs_with_constants(&sort_req, &self.constants); - let sort_req = collapse_lex_req(sort_req); + let mut sort_req = collapse_lex_req(sort_req); // If empty immediately return if sort_req.is_empty() { return sort_req; } + for ordering in self.oeq_group.iter() { + let normalized_ordering = self.eq_groups.normalize_sort_exprs(ordering); + let req = prune_sort_reqs_with_constants( + &PhysicalSortRequirement::from_sort_exprs(&normalized_ordering), + &self.constants, + ); + let ordering = PhysicalSortRequirement::to_sort_exprs(req); + let match_indices = ordering + .iter() + .map(|elem| { + sort_req.iter().position(|sort_req| { + elem.satisfy_with_schema(sort_req, &self.schema) + }) + }) + .collect::>(); + let mut match_prefix = vec![]; + for elem in &match_indices{ + if let Some(elem) = elem{ + if let Some(last) = match_prefix.last(){ + // Should increase + if elem <= last{ + break; + } + } + match_prefix.push(*elem) + } else { + break; + } + } + println!("match_indices:{:?}, match_prefix:{:?}", match_indices, match_prefix); + // can remove entries at the match_prefix indices + for idx in match_prefix.iter().rev(){ + sort_req.remove(*idx); + } + + } + // If empty immediately return + if sort_req.is_empty() { + return sort_req; + } + // TODO: Do not delete from the start + // or use empty check + let mut new_sort_req = vec![sort_req[0].clone()]; let mut idx = 1; while idx < sort_req.len() { @@ -1862,16 +1905,15 @@ mod tests { ], true, ), - // TODO: Resolve test below - // ( - // vec![ - // (col_d, option1), - // (col_e, option2), - // (col_b, option1), - // (col_f, option1), - // ], - // true, - // ), + ( + vec![ + (col_d, option1), + (col_e, option2), + (col_b, option1), + (col_f, option1), + ], + true, + ), ]; for (cols, expected) in requirements { From b5022d17d99bc85c57faf595199a65b209c3d144 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 12 Oct 2023 09:05:39 +0300 Subject: [PATCH 050/122] Minor changes --- datafusion/physical-expr/src/equivalence.rs | 16 +++++++++------- 1 file changed, 9 insertions(+), 7 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 6aa8b15e4f6b..5f8d06d291e1 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -807,11 +807,11 @@ impl SchemaProperties { }) .collect::>(); let mut match_prefix = vec![]; - for elem in &match_indices{ - if let Some(elem) = elem{ - if let Some(last) = match_prefix.last(){ + for elem in &match_indices { + if let Some(elem) = elem { + if let Some(last) = match_prefix.last() { // Should increase - if elem <= last{ + if elem <= last { break; } } @@ -820,12 +820,14 @@ impl SchemaProperties { break; } } - println!("match_indices:{:?}, match_prefix:{:?}", match_indices, match_prefix); + println!( + "match_indices:{:?}, match_prefix:{:?}", + match_indices, match_prefix + ); // can remove entries at the match_prefix indices - for idx in match_prefix.iter().rev(){ + for idx in match_prefix.iter().rev() { sort_req.remove(*idx); } - } // If empty immediately return if sort_req.is_empty() { From 768764209143d843240260e46794c1850f38901c Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 12 Oct 2023 10:36:26 +0300 Subject: [PATCH 051/122] Minor changes --- datafusion/physical-expr/src/equivalence.rs | 129 +++----------------- datafusion/physical-plan/src/windows/mod.rs | 3 + 2 files changed, 17 insertions(+), 115 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 5f8d06d291e1..c340232c9170 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -26,7 +26,7 @@ use arrow::datatypes::SchemaRef; use crate::physical_expr::{deduplicate_physical_exprs, have_common_entries}; use crate::sort_properties::{ExprOrdering, SortProperties}; -use arrow_schema::{Schema, SortOptions}; +use arrow_schema::SortOptions; use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::utils::longest_consecutive_prefix; use datafusion_common::{DataFusionError, JoinSide, JoinType, Result}; @@ -755,28 +755,7 @@ impl SchemaProperties { // Prune redundant sections in the requirement. normalized_sort_reqs = self.prune_lex_req(normalized_sort_reqs); - - let oeq_group = self.oeq_group(); - if normalized_sort_reqs.is_empty() && !oeq_group.is_empty() { - // By convention use first entry - PhysicalSortRequirement::from_sort_exprs(&oeq_group.inner[0]) - } else { - normalized_sort_reqs - } - } - - // Find the prefix length that is satisfied in the sort requirement - fn prefix_len_satisfied(&self, sort_req: &[PhysicalSortRequirement]) -> usize { - let mut prefix_length = 0; - while prefix_length < sort_req.len() { - if self.ordering_satisfy_requirement_concrete(&sort_req[0..prefix_length + 1]) - { - prefix_length += 1; - } else { - break; - } - } - prefix_length + normalized_sort_reqs } /// This function simplifies lexicographical ordering requirement @@ -820,79 +799,13 @@ impl SchemaProperties { break; } } - println!( - "match_indices:{:?}, match_prefix:{:?}", - match_indices, match_prefix - ); // can remove entries at the match_prefix indices for idx in match_prefix.iter().rev() { sort_req.remove(*idx); } } - // If empty immediately return - if sort_req.is_empty() { - return sort_req; - } - // TODO: Do not delete from the start - // or use empty check - - let mut new_sort_req = vec![sort_req[0].clone()]; - let mut idx = 1; - while idx < sort_req.len() { - let n_remove = self.prefix_len_satisfied(&sort_req[idx..]); - if n_remove == 0 { - new_sort_req.push(sort_req[idx].clone()); - idx += 1; - } else { - idx += n_remove; - } - } - - let mut section = &new_sort_req[..]; - // Eat up from the end of the sort_req until no section can be removed - // from the ending. - loop { - let n_prune = self.prune_last_n_that_is_in_oeq(section); - // Cannot prune entries from the end of requirement - if n_prune == 0 { - break; - } - section = §ion[0..section.len() - n_prune]; - } // Remove duplicates from expression. - collapse_lex_req(section.to_vec()) - } - - /// Determines how many entries from the end can be deleted. - /// Last n entry satisfies global ordering, hence having them - /// as postfix in the lexicographical requirement is unnecessary. - /// Assume requirement is [a ASC, b ASC, c ASC], also assume that - /// existing ordering is [c ASC, d ASC]. In this case, since [c ASC] - /// is satisfied by the existing ordering (e.g corresponding section is global ordering), - /// [c ASC] can be pruned from the requirement: [a ASC, b ASC, c ASC]. In this case, - /// this function will return 1, to indicate last element can be removed from the requirement - fn prune_last_n_that_is_in_oeq(&self, sort_req: &[PhysicalSortRequirement]) -> usize { - let sort_req_len = sort_req.len(); - let oeq_group = self.oeq_group(); - let eq_groups = self.eq_groups(); - for ordering in oeq_group.iter() { - let ordering = eq_groups.normalize_sort_exprs(ordering); - let req = prune_sort_reqs_with_constants( - &PhysicalSortRequirement::from_sort_exprs(&ordering), - &self.constants, - ); - let ordering = PhysicalSortRequirement::to_sort_exprs(req); - let mut search_range = std::cmp::min(ordering.len(), sort_req_len); - while search_range > 0 { - let req_section = &sort_req[sort_req_len - search_range..]; - if req_satisfied(&ordering, req_section, &self.schema) { - return search_range; - } else { - search_range -= 1; - } - } - } - 0 + collapse_lex_req(sort_req) } /// Checks whether `leading_requirement` is contained in any of the ordering @@ -1174,15 +1087,8 @@ impl SchemaProperties { &self, required: &[PhysicalSortRequirement], ) -> bool { - let provided_normalized = self.oeq_group().output_ordering().unwrap_or_default(); let required_normalized = self.normalize_sort_requirements(required); - if required_normalized.len() > provided_normalized.len() { - return false; - } - required_normalized - .into_iter() - .zip(provided_normalized) - .all(|(req, given)| given.satisfy_with_schema(&req, &self.schema)) + required_normalized.is_empty() } /// Checks whether the given [`PhysicalSortRequirement`]s are equal or more @@ -1208,8 +1114,16 @@ impl SchemaProperties { provided: &[PhysicalSortRequirement], required: &[PhysicalSortRequirement], ) -> bool { - let required_normalized = self.normalize_sort_requirements(required); - let provided_normalized = self.normalize_sort_requirements(provided); + let required_normalized = self.eq_groups.normalize_sort_requirements(required); + let required_normalized = + prune_sort_reqs_with_constants(&required_normalized, &self.constants); + let required_normalized = collapse_lex_req(required_normalized); + + let provided_normalized = self.eq_groups.normalize_sort_requirements(provided); + let provided_normalized = + prune_sort_reqs_with_constants(&provided_normalized, &self.constants); + let provided_normalized = collapse_lex_req(provided_normalized); + if required_normalized.len() > provided_normalized.len() { return false; } @@ -1352,21 +1266,6 @@ pub fn collapse_lex_ordering(input: LexOrdering) -> LexOrdering { output } -/// Checks whether given section satisfies req. -fn req_satisfied( - given: LexOrderingRef, - req: &[PhysicalSortRequirement], - schema: &Arc, -) -> bool { - // Write below code as any/all - for (given, req) in izip!(given.iter(), req.iter()) { - if !given.satisfy_with_schema(req, schema) { - return false; - } - } - true -} - /// Remove ordering requirements that have constant value fn prune_sort_reqs_with_constants( ordering: &[PhysicalSortRequirement], diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index 89c2d6dc99c8..6c46f4cdd54d 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -374,6 +374,8 @@ pub fn get_best_fitting_window( // of the window_exprs are same. let partitionby_exprs = window_exprs[0].partition_by(); let orderby_keys = window_exprs[0].order_by(); + // println!("partitionby_exprs: {:?}, orderby_keys:{:?}", partitionby_exprs, orderby_keys); + // println!("input.schema_properties().oeq_group():{:?}", input.schema_properties().oeq_group()); let (should_reverse, partition_search_mode) = if let Some((should_reverse, partition_search_mode)) = get_window_mode(partitionby_exprs, orderby_keys, input)? @@ -382,6 +384,7 @@ pub fn get_best_fitting_window( } else { return Ok(None); }; + // println!("should_reverse: {:?}, partition_search_mode:{:?}", should_reverse, partition_search_mode); let is_unbounded = unbounded_output(input); if !is_unbounded && partition_search_mode != PartitionSearchMode::Sorted { // Executor has bounded input and `partition_search_mode` is not `PartitionSearchMode::Sorted` From 988522f3633178e35f910ffbeccce6a72f296094 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 12 Oct 2023 11:06:14 +0300 Subject: [PATCH 052/122] Simplifications --- datafusion/physical-expr/src/equivalence.rs | 248 ++++++++++++++--- datafusion/physical-expr/src/utils.rs | 255 +----------------- .../physical-plan/src/aggregates/mod.rs | 60 ++--- 3 files changed, 233 insertions(+), 330 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index c340232c9170..5a95c185aab0 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -750,33 +750,29 @@ impl SchemaProperties { &self, sort_reqs: &[PhysicalSortRequirement], ) -> Vec { - let mut normalized_sort_reqs = - self.eq_groups.normalize_sort_requirements(sort_reqs); - + let normalized_sort_reqs = self.eq_groups.normalize_sort_requirements(sort_reqs); + let normalized_sort_reqs = + prune_sort_reqs_with_constants(&normalized_sort_reqs, &self.constants); // Prune redundant sections in the requirement. - normalized_sort_reqs = self.prune_lex_req(normalized_sort_reqs); - normalized_sort_reqs + collapse_lex_req(normalized_sort_reqs) } - /// This function simplifies lexicographical ordering requirement - /// inside `sort_req` by removing postfix lexicographical requirements - /// that satisfy global ordering (occurs inside the ordering equivalent class) - fn prune_lex_req(&self, sort_req: LexOrderingReq) -> LexOrderingReq { - // Remove entries that are known to be constant from requirement expression. - let sort_req = prune_sort_reqs_with_constants(&sort_req, &self.constants); - let mut sort_req = collapse_lex_req(sort_req); + /// This function prunes lexicographical ordering requirement + /// by removing sections inside `sort_req` that satisfies any of the existing ordering. + /// By doing so, we reduce the requirement to its simplest form which is functionally + /// equivalent to the argument. Empty result means that requirement is already satisfied. + fn prune_lex_req(&self, sort_req: &[PhysicalSortRequirement]) -> LexOrderingReq { + // Make sure to use a standardized version of the requirement + let mut sort_req = self.normalize_sort_requirements(sort_req); // If empty immediately return if sort_req.is_empty() { return sort_req; } + let leading_requirement = sort_req[0].clone(); for ordering in self.oeq_group.iter() { - let normalized_ordering = self.eq_groups.normalize_sort_exprs(ordering); - let req = prune_sort_reqs_with_constants( - &PhysicalSortRequirement::from_sort_exprs(&normalized_ordering), - &self.constants, - ); - let ordering = PhysicalSortRequirement::to_sort_exprs(req); + // Normalize existing ordering + let ordering = self.normalize_sort_exprs(ordering); let match_indices = ordering .iter() .map(|elem| { @@ -800,12 +796,18 @@ impl SchemaProperties { } } // can remove entries at the match_prefix indices + // Remove with reverse iteration to not invalidate indices for idx in match_prefix.iter().rev() { sort_req.remove(*idx); } } - // Remove duplicates from expression. - collapse_lex_req(sort_req) + if !sort_req.is_empty() { + // Do not invalidate requirement + sort_req.insert(0, leading_requirement); + sort_req + } else { + sort_req + } } /// Checks whether `leading_requirement` is contained in any of the ordering @@ -977,17 +979,9 @@ impl SchemaProperties { /// Checks whether the required [`PhysicalSortExpr`]s are satisfied by the /// any of the existing orderings. pub fn ordering_satisfy_concrete(&self, required: &[PhysicalSortExpr]) -> bool { - let required_normalized = self.normalize_sort_exprs(required); - let provided_normalized = self.oeq_group().output_ordering().unwrap_or_default(); - - if required_normalized.len() > provided_normalized.len() { - return false; - } - - required_normalized - .into_iter() - .zip(provided_normalized) - .all(|(req, given)| given == req) + // Convert `PhysicalSortExpr`s to `PhysicalSortRequirement`s + let sort_requirements = PhysicalSortRequirement::from_sort_exprs(required.iter()); + self.ordering_satisfy_requirement_concrete(&sort_requirements) } /// Find the finer requirement among `req1` and `req2` @@ -1087,8 +1081,7 @@ impl SchemaProperties { &self, required: &[PhysicalSortRequirement], ) -> bool { - let required_normalized = self.normalize_sort_requirements(required); - required_normalized.is_empty() + self.prune_lex_req(required).is_empty() } /// Checks whether the given [`PhysicalSortRequirement`]s are equal or more @@ -1114,15 +1107,8 @@ impl SchemaProperties { provided: &[PhysicalSortRequirement], required: &[PhysicalSortRequirement], ) -> bool { - let required_normalized = self.eq_groups.normalize_sort_requirements(required); - let required_normalized = - prune_sort_reqs_with_constants(&required_normalized, &self.constants); - let required_normalized = collapse_lex_req(required_normalized); - - let provided_normalized = self.eq_groups.normalize_sort_requirements(provided); - let provided_normalized = - prune_sort_reqs_with_constants(&provided_normalized, &self.constants); - let provided_normalized = collapse_lex_req(provided_normalized); + let provided_normalized = self.normalize_sort_requirements(provided); + let required_normalized = self.normalize_sort_requirements(required); if required_normalized.len() > provided_normalized.len() { return false; @@ -1547,6 +1533,18 @@ mod tests { Ok((test_schema, schema_properties)) } + // Convert each tuple to PhysicalSortRequirement + fn convert_to_requirement( + in_data: &[(&Arc, Option)], + ) -> Vec { + in_data + .iter() + .map(|(expr, options)| { + PhysicalSortRequirement::new((*expr).clone(), *options) + }) + .collect::>() + } + #[test] fn add_equal_conditions_test() -> Result<()> { let schema = Arc::new(Schema::new(vec![ @@ -2251,4 +2249,170 @@ mod tests { let res = RecordBatch::try_from_iter(res)?; Ok(res) } + + #[test] + fn test_schema_normalize_expr_with_equivalence() -> Result<()> { + let col_a = &Column::new("a", 0); + let col_b = &Column::new("b", 1); + let col_c = &Column::new("c", 2); + // Assume that column a and c are aliases. + let (_test_schema, schema_properties) = create_test_params()?; + + let col_a_expr = Arc::new(col_a.clone()) as Arc; + let col_b_expr = Arc::new(col_b.clone()) as Arc; + let col_c_expr = Arc::new(col_c.clone()) as Arc; + // Test cases for equivalence normalization, + // First entry in the tuple is argument, second entry is expected result after normalization. + let expressions = vec![ + // Normalized version of the column a and c should go to a (since a is head) + (&col_a_expr, &col_a_expr), + (&col_c_expr, &col_a_expr), + // Cannot normalize column b + (&col_b_expr, &col_b_expr), + ]; + let eq_groups = schema_properties.eq_groups(); + for (expr, expected_eq) in expressions { + assert!( + expected_eq.eq(&eq_groups.normalize_expr(expr.clone())), + "error in test: expr: {expr:?}" + ); + } + + Ok(()) + } + + #[test] + fn test_schema_normalize_sort_requirement_with_equivalence() -> Result<()> { + let option1 = SortOptions { + descending: false, + nulls_first: false, + }; + // Assume that column a and c are aliases. + let (test_schema, schema_properties) = create_test_params()?; + let col_a_expr = &col("a", &test_schema)?; + let _col_b_expr = &col("b", &test_schema)?; + let col_c_expr = &col("c", &test_schema)?; + let col_d_expr = &col("d", &test_schema)?; + let _col_e_expr = &col("e", &test_schema)?; + + // Test cases for equivalence normalization + // First entry in the tuple is PhysicalExpr, second entry is its ordering, third entry is result after normalization. + let expressions = vec![ + ( + vec![PhysicalSortRequirement { + expr: col_a_expr.clone(), + options: Some(option1), + }], + vec![PhysicalSortRequirement { + expr: col_a_expr.clone(), + options: Some(option1), + }], + ), + // In the normalized version column c should be replace with column a + ( + vec![PhysicalSortRequirement { + expr: col_c_expr.clone(), + options: Some(option1), + }], + vec![PhysicalSortRequirement { + expr: col_a_expr.clone(), + options: Some(option1), + }], + ), + ( + vec![PhysicalSortRequirement { + expr: col_c_expr.clone(), + options: None, + }], + vec![PhysicalSortRequirement { + expr: col_a_expr.clone(), + options: None, + }], + ), + ( + vec![PhysicalSortRequirement { + expr: col_d_expr.clone(), + options: Some(option1), + }], + vec![PhysicalSortRequirement { + expr: col_d_expr.clone(), + options: Some(option1), + }], + ), + ]; + for (arg, expected) in expressions.into_iter() { + let normalized = schema_properties.normalize_sort_requirements(&arg); + assert!( + expected.eq(&normalized), + "error in test: arg: {arg:?}, expected: {expected:?}, normalized: {normalized:?}" + ); + } + + Ok(()) + } + + #[test] + fn test_normalize_sort_reqs() -> Result<()> { + // Schema satisfies following properties + // a=c + // and following orderings are valid + // [a ASC], [d ASC, b ASC], [e DESC, f ASC, g ASC] + let (test_schema, schema_properties) = create_test_params()?; + let col_a = &col("a", &test_schema)?; + let col_b = &col("b", &test_schema)?; + let col_c = &col("c", &test_schema)?; + let col_d = &col("d", &test_schema)?; + let col_e = &col("e", &test_schema)?; + let col_f = &col("f", &test_schema)?; + let option1 = SortOptions { + descending: false, + nulls_first: false, + }; + let option2 = SortOptions { + descending: true, + nulls_first: true, + }; + // First element in the tuple stores vector of requirement, second element is the expected return value for ordering_satisfy function + let requirements = vec![ + (vec![(col_a, Some(option1))], vec![(col_a, Some(option1))]), + (vec![(col_a, Some(option2))], vec![(col_a, Some(option2))]), + (vec![(col_a, None)], vec![(col_a, None)]), + // Test whether equivalence works as expected + (vec![(col_c, Some(option1))], vec![(col_a, Some(option1))]), + (vec![(col_c, None)], vec![(col_a, None)]), + // Test whether ordering equivalence works as expected + ( + vec![(col_d, Some(option1)), (col_b, Some(option1))], + vec![(col_d, Some(option1)), (col_b, Some(option1))], + ), + ( + vec![(col_d, None), (col_b, None)], + vec![(col_d, None), (col_b, None)], + ), + ( + vec![(col_e, Some(option2)), (col_f, Some(option1))], + vec![(col_e, Some(option2)), (col_f, Some(option1))], + ), + // We should be able to normalize in compatible requirements also (not exactly equal) + ( + vec![(col_e, Some(option2)), (col_f, None)], + vec![(col_e, Some(option2)), (col_f, None)], + ), + ( + vec![(col_e, None), (col_f, None)], + vec![(col_e, None), (col_f, None)], + ), + ]; + + for (reqs, expected_normalized) in requirements.into_iter() { + let req = convert_to_requirement(&reqs); + let expected_normalized = convert_to_requirement(&expected_normalized); + + assert_eq!( + schema_properties.normalize_sort_requirements(&req), + expected_normalized + ); + } + Ok(()) + } } diff --git a/datafusion/physical-expr/src/utils.rs b/datafusion/physical-expr/src/utils.rs index cd5b3a68536c..1d1b4d95cb9f 100644 --- a/datafusion/physical-expr/src/utils.rs +++ b/datafusion/physical-expr/src/utils.rs @@ -430,7 +430,7 @@ mod tests { use super::*; use crate::equivalence::SchemaProperties; use crate::expressions::{binary, cast, col, in_list, lit, Column, Literal}; - use crate::{PhysicalSortExpr, PhysicalSortRequirement}; + use crate::PhysicalSortExpr; use arrow::compute::SortOptions; use arrow_array::Int32Array; @@ -479,82 +479,6 @@ mod tests { } } - // Generate a schema which consists of 5 columns (a, b, c, d, e) - fn create_test_schema() -> Result { - let a = Field::new("a", DataType::Int32, true); - let b = Field::new("b", DataType::Int32, true); - let c = Field::new("c", DataType::Int32, true); - let d = Field::new("d", DataType::Int32, true); - let e = Field::new("e", DataType::Int32, true); - let f = Field::new("f", DataType::Int32, true); - let schema = Arc::new(Schema::new(vec![a, b, c, d, e, f])); - - Ok(schema) - } - - fn create_test_params() -> Result<(SchemaRef, SchemaProperties)> { - // Assume schema satisfies ordering a ASC NULLS LAST - // and d ASC NULLS LAST, b ASC NULLS LAST and e DESC NULLS FIRST, f ASC NULLS LAST, g ASC NULLS LAST - // Assume that column a and c are aliases. - let col_a = &Column::new("a", 0); - let col_b = &Column::new("b", 1); - let col_c = &Column::new("c", 2); - let col_d = &Column::new("d", 3); - let col_e = &Column::new("e", 4); - let col_f = &Column::new("f", 5); - let col_g = &Column::new("g", 6); - let option1 = SortOptions { - descending: false, - nulls_first: false, - }; - let option2 = SortOptions { - descending: true, - nulls_first: true, - }; - let test_schema = create_test_schema()?; - let col_a_expr = Arc::new(col_a.clone()) as _; - let col_c_expr = Arc::new(col_c.clone()) as _; - let mut schema_properties = SchemaProperties::new(test_schema.clone()); - schema_properties.add_equal_conditions((&col_a_expr, &col_c_expr)); - schema_properties.add_new_orderings(&[ - vec![PhysicalSortExpr { - expr: Arc::new(col_a.clone()), - options: option1, - }], - vec![ - PhysicalSortExpr { - expr: Arc::new(col_d.clone()), - options: option1, - }, - PhysicalSortExpr { - expr: Arc::new(col_b.clone()), - options: option1, - }, - ], - ]); - schema_properties.add_new_orderings(&[ - vec![PhysicalSortExpr { - expr: Arc::new(col_a.clone()), - options: option1, - }], - vec![ - PhysicalSortExpr { - expr: Arc::new(col_e.clone()), - options: option2, - }, - PhysicalSortExpr { - expr: Arc::new(col_f.clone()), - options: option1, - }, - PhysicalSortExpr { - expr: Arc::new(col_g.clone()), - options: option1, - }, - ], - ]); - Ok((test_schema, schema_properties)) - } - #[test] fn test_build_dag() -> Result<()> { let schema = Schema::new(vec![ @@ -706,78 +630,6 @@ mod tests { Ok(()) } - fn convert_to_requirement( - in_data: &[(&Column, Option)], - ) -> Vec { - in_data - .iter() - .map(|(col, options)| { - PhysicalSortRequirement::new(Arc::new((*col).clone()) as _, *options) - }) - .collect::>() - } - - #[test] - fn test_normalize_sort_reqs() -> Result<()> { - let col_a = &Column::new("a", 0); - let col_b = &Column::new("b", 1); - let col_c = &Column::new("c", 2); - let col_d = &Column::new("d", 3); - let col_e = &Column::new("e", 4); - let col_f = &Column::new("f", 5); - let option1 = SortOptions { - descending: false, - nulls_first: false, - }; - let option2 = SortOptions { - descending: true, - nulls_first: true, - }; - // First element in the tuple stores vector of requirement, second element is the expected return value for ordering_satisfy function - let requirements = vec![ - (vec![(col_a, Some(option1))], vec![(col_a, Some(option1))]), - (vec![(col_a, Some(option2))], vec![(col_a, Some(option2))]), - (vec![(col_a, None)], vec![(col_a, Some(option1))]), - // Test whether equivalence works as expected - (vec![(col_c, Some(option1))], vec![(col_a, Some(option1))]), - (vec![(col_c, None)], vec![(col_a, Some(option1))]), - // Test whether ordering equivalence works as expected - ( - vec![(col_d, Some(option1)), (col_b, Some(option1))], - vec![(col_a, Some(option1))], - ), - ( - vec![(col_d, None), (col_b, None)], - vec![(col_a, Some(option1))], - ), - ( - vec![(col_e, Some(option2)), (col_f, Some(option1))], - vec![(col_a, Some(option1))], - ), - // We should be able to normalize in compatible requirements also (not exactly equal) - ( - vec![(col_e, Some(option2)), (col_f, None)], - vec![(col_a, Some(option1))], - ), - ( - vec![(col_e, None), (col_f, None)], - vec![(col_a, Some(option1))], - ), - ]; - - let (_test_schema, schema_properties) = create_test_params()?; - for (reqs, expected_normalized) in requirements.into_iter() { - let req = convert_to_requirement(&reqs); - let expected_normalized = convert_to_requirement(&expected_normalized); - - assert_eq!( - schema_properties.normalize_sort_requirements(&req), - expected_normalized - ); - } - Ok(()) - } - #[test] fn test_reassign_predicate_columns_in_list() { let int_field = Field::new("should_not_matter", DataType::Int64, true); @@ -815,111 +667,6 @@ mod tests { assert_eq!(actual.as_ref(), expected.as_any()); } - #[test] - fn test_normalize_expr_with_equivalence() -> Result<()> { - let col_a = &Column::new("a", 0); - let col_b = &Column::new("b", 1); - let col_c = &Column::new("c", 2); - let _col_d = &Column::new("d", 3); - let _col_e = &Column::new("e", 4); - // Assume that column a and c are aliases. - let (_test_schema, schema_properties) = create_test_params()?; - - let col_a_expr = Arc::new(col_a.clone()) as Arc; - let col_b_expr = Arc::new(col_b.clone()) as Arc; - let col_c_expr = Arc::new(col_c.clone()) as Arc; - // Test cases for equivalence normalization, - // First entry in the tuple is argument, second entry is expected result after normalization. - let expressions = vec![ - // Normalized version of the column a and c should go to a (since a is head) - (&col_a_expr, &col_a_expr), - (&col_c_expr, &col_a_expr), - // Cannot normalize column b - (&col_b_expr, &col_b_expr), - ]; - let eq_groups = schema_properties.eq_groups(); - for (expr, expected_eq) in expressions { - assert!( - expected_eq.eq(&eq_groups.normalize_expr(expr.clone())), - "error in test: expr: {expr:?}" - ); - } - - Ok(()) - } - - #[test] - fn test_normalize_sort_requirement_with_equivalence() -> Result<()> { - let option1 = SortOptions { - descending: false, - nulls_first: false, - }; - // Assume that column a and c are aliases. - let (test_schema, schema_properties) = create_test_params()?; - let col_a_expr = col("a", &test_schema)?; - let _col_b_expr = col("b", &test_schema)?; - let col_c_expr = col("c", &test_schema)?; - let col_d_expr = col("d", &test_schema)?; - let _col_e_expr = col("e", &test_schema)?; - - // Test cases for equivalence normalization - // First entry in the tuple is PhysicalExpr, second entry is its ordering, third entry is result after normalization. - let expressions = vec![ - ( - vec![PhysicalSortRequirement { - expr: col_a_expr.clone(), - options: Some(option1), - }], - vec![PhysicalSortRequirement { - expr: col_a_expr.clone(), - options: Some(option1), - }], - ), - ( - vec![PhysicalSortRequirement { - expr: col_c_expr.clone(), - options: Some(option1), - }], - vec![PhysicalSortRequirement { - expr: col_a_expr.clone(), - options: Some(option1), - }], - ), - ( - vec![PhysicalSortRequirement { - expr: col_c_expr.clone(), - options: None, - }], - vec![PhysicalSortRequirement { - expr: col_a_expr.clone(), - options: Some(option1), - }], - ), - // d, b occurs in the ordering equivalence - // requirement d is also satisfied with existing ordering - // hence, normalized version should be a ASC - ( - vec![PhysicalSortRequirement { - expr: col_d_expr.clone(), - options: Some(option1), - }], - vec![PhysicalSortRequirement { - expr: col_a_expr.clone(), - options: Some(option1), - }], - ), - ]; - for (arg, expected) in expressions.into_iter() { - let normalized = schema_properties.normalize_sort_requirements(&arg); - assert!( - expected.eq(&normalized), - "error in test: arg: {arg:?}, expected: {expected:?}, normalized: {normalized:?}" - ); - } - - Ok(()) - } - #[test] fn test_collect_columns() -> Result<()> { let expr1 = Arc::new(Column::new("col1", 2)) as _; diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 2e7117084a40..e1e536c59d05 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -1113,7 +1113,7 @@ mod tests { }; use datafusion_execution::runtime_env::{RuntimeConfig, RuntimeEnv}; use datafusion_physical_expr::expressions::{ - lit, ApproxDistinct, Column, Count, FirstValue, LastValue, Median, + lit, ApproxDistinct, Count, FirstValue, LastValue, Median, }; use datafusion_physical_expr::{ AggregateExpr, PhysicalExpr, PhysicalSortExpr, SchemaProperties, @@ -1987,61 +1987,53 @@ mod tests { descending: true, nulls_first: true, }; - let col_a = Column::new("a", 0); - let col_b = Column::new("b", 1); - let col_c = Column::new("c", 2); - let col_d = Column::new("d", 3); - let col_a_expr = Arc::new(col_a.clone()) as Arc; - let col_b_expr = Arc::new(col_b.clone()) as Arc; + let col_a = &col("a", &test_schema)?; + let col_b = &col("b", &test_schema)?; + let col_c = &col("c", &test_schema)?; let mut schema_properties = SchemaProperties::new(test_schema); // Columns a and b are equal. - schema_properties.add_equal_conditions((&col_a_expr, &col_b_expr)); - // [a ASC], [c DESC] describes ordering of the schema. - schema_properties.add_new_orderings(&[ - vec![PhysicalSortExpr { - expr: Arc::new(col_a.clone()) as _, - options: options1, - }], - vec![PhysicalSortExpr { - expr: Arc::new(col_c.clone()) as _, - options: options2, - }], - ]); + schema_properties.add_equal_conditions((col_a, col_b)); // Aggregate requirements are - // [None], [a ASC], [b ASC], [c DESC], [a ASC, d ASC] respectively + // [None], [a ASC], [a ASC, b ASC, c ASC], [a ASC, b ASC] respectively let mut order_by_exprs = vec![ None, Some(vec![PhysicalSortExpr { - expr: Arc::new(col_a.clone()), - options: options1, - }]), - Some(vec![PhysicalSortExpr { - expr: Arc::new(col_b.clone()), + expr: col_a.clone(), options: options1, }]), - Some(vec![PhysicalSortExpr { - expr: Arc::new(col_c), - options: options2, - }]), Some(vec![ PhysicalSortExpr { - expr: Arc::new(col_a.clone()), + expr: col_a.clone(), + options: options1, + }, + PhysicalSortExpr { + expr: col_b.clone(), + options: options1, + }, + PhysicalSortExpr { + expr: col_c.clone(), + options: options1, + }, + ]), + Some(vec![ + PhysicalSortExpr { + expr: col_a.clone(), options: options1, }, PhysicalSortExpr { - expr: Arc::new(col_d), + expr: col_b.clone(), options: options1, }, ]), // Since aggregate expression is reversible (FirstValue), we should be able to resolve below // contradictory requirement by reversing it. Some(vec![PhysicalSortExpr { - expr: Arc::new(col_b.clone()), + expr: col_b.clone(), options: options2, }]), ]; let aggr_expr = Arc::new(FirstValue::new( - Arc::new(col_a.clone()), + col_a.clone(), "first1", DataType::Int32, vec![], @@ -2051,7 +2043,7 @@ mod tests { let res = get_finest_requirement(&mut aggr_exprs, &mut order_by_exprs, || { schema_properties.clone() })?; - assert_eq!(res, order_by_exprs[4]); + assert_eq!(res, order_by_exprs[2]); Ok(()) } } From 8c17c05e65f5856def1fcd148266c2f98e99dba5 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 12 Oct 2023 13:44:02 +0300 Subject: [PATCH 053/122] Minor changes --- .../physical-plan/src/aggregates/mod.rs | 27 ++++++++++--------- 1 file changed, 15 insertions(+), 12 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index e1e536c59d05..3c4c3ecb7d94 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -329,13 +329,12 @@ fn get_init_req( /// This function gets the finest ordering requirement among all the aggregation /// functions. If requirements are conflicting, (i.e. we can not compute the /// aggregations in a single [`AggregateExec`]), the function returns an error. -fn get_finest_requirement SchemaProperties>( +fn get_finest_requirement( aggr_expr: &mut [Arc], order_by_expr: &mut [Option], - schema_properties: F2, + schema_properties: &SchemaProperties, ) -> Result> { let mut finest_req = get_init_req(aggr_expr, order_by_expr); - let properties = schema_properties(); for (aggr_expr, fn_req) in aggr_expr.iter_mut().zip(order_by_expr.iter_mut()) { let fn_req = if let Some(fn_req) = fn_req { fn_req @@ -344,7 +343,8 @@ fn get_finest_requirement SchemaProperties>( }; if let Some(finest_req) = &mut finest_req { - if let Some(finer) = properties.get_finer_ordering(finest_req, fn_req) { + if let Some(finer) = schema_properties.get_finer_ordering(finest_req, fn_req) + { *finest_req = finer.to_vec(); continue; } @@ -353,7 +353,7 @@ fn get_finest_requirement SchemaProperties>( if let Some(reverse) = aggr_expr.reverse_expr() { let fn_req_reverse = reverse_order_bys(fn_req); if let Some(finer) = - properties.get_finer_ordering(finest_req, &fn_req_reverse) + schema_properties.get_finer_ordering(finest_req, &fn_req_reverse) { // We need to update `aggr_expr` with its reverse, since only its // reverse requirement is compatible with existing requirements: @@ -479,10 +479,11 @@ impl AggregateExec { }) }) .collect::>(); - let requirement = - get_finest_requirement(&mut aggr_expr, &mut order_by_expr, || { - input.schema_properties() - })?; + let requirement = get_finest_requirement( + &mut aggr_expr, + &mut order_by_expr, + &input.schema_properties(), + )?; let mut ordering_req = requirement.unwrap_or(vec![]); let partition_search_mode = get_aggregate_search_mode( &group_by, @@ -2040,9 +2041,11 @@ mod tests { vec![], )) as _; let mut aggr_exprs = vec![aggr_expr; order_by_exprs.len()]; - let res = get_finest_requirement(&mut aggr_exprs, &mut order_by_exprs, || { - schema_properties.clone() - })?; + let res = get_finest_requirement( + &mut aggr_exprs, + &mut order_by_exprs, + &schema_properties, + )?; assert_eq!(res, order_by_exprs[2]); Ok(()) } From 62a1a250d458294aec5603aaff2040a607d87b95 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 12 Oct 2023 14:11:13 +0300 Subject: [PATCH 054/122] Add new test case --- .../physical-plan/src/aggregates/mod.rs | 24 ++++++++++++++ .../sqllogictest/test_files/groupby.slt | 32 +++++++++++++++++++ 2 files changed, 56 insertions(+) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 3c4c3ecb7d94..41ec3f3a2606 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -334,6 +334,30 @@ fn get_finest_requirement( order_by_expr: &mut [Option], schema_properties: &SchemaProperties, ) -> Result> { + // Check at the beginning if all the requirements are satisfied by existing ordering + // If so return None, to indicate all of the requirements are already satisfied. + let mut all_satisfied = true; + for (aggr_expr, fn_req) in aggr_expr.iter_mut().zip(order_by_expr.iter_mut()) { + if schema_properties.ordering_satisfy(fn_req.as_deref()) { + continue; + } + if let Some(reverse) = aggr_expr.reverse_expr() { + let reverse_req = fn_req.as_ref().map(|item| reverse_order_bys(item)); + if schema_properties.ordering_satisfy(reverse_req.as_deref()) { + // We need to update `aggr_expr` with its reverse, since only its + // reverse requirement is compatible with existing requirements: + *aggr_expr = reverse; + *fn_req = reverse_req; + continue; + } + } + // requirement is not satisfied + all_satisfied = false; + } + if all_satisfied { + // All of the requirements are already satisfied. + return Ok(None); + } let mut finest_req = get_init_req(aggr_expr, order_by_expr); for (aggr_expr, fn_req) in aggr_expr.iter_mut().zip(order_by_expr.iter_mut()) { let fn_req = if let Some(fn_req) = fn_req { diff --git a/datafusion/sqllogictest/test_files/groupby.slt b/datafusion/sqllogictest/test_files/groupby.slt index 1803669a857c..7f03da41c573 100644 --- a/datafusion/sqllogictest/test_files/groupby.slt +++ b/datafusion/sqllogictest/test_files/groupby.slt @@ -3602,6 +3602,38 @@ AggregateExec: mode=Final, gby=[], aggr=[FIRST_VALUE(foo.x)] ----AggregateExec: mode=Partial, gby=[], aggr=[FIRST_VALUE(foo.x)] ------MemoryExec: partitions=8, partition_sizes=[1, 0, 0, 0, 0, 0, 0, 0] +# Since both ordering requirements are satisfied, there shouldn't be +# any SortExec in the final plan. +query TT +EXPLAIN SELECT FIRST_VALUE(a ORDER BY a ASC) as first_a, + LAST_VALUE(c ORDER BY c DESC) as last_c +FROM multiple_ordered_table +GROUP BY d; +---- +logical_plan +Projection: FIRST_VALUE(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.a ASC NULLS LAST] AS first_a, LAST_VALUE(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST] AS last_c +--Aggregate: groupBy=[[multiple_ordered_table.d]], aggr=[[FIRST_VALUE(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.a ASC NULLS LAST], LAST_VALUE(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST]]] +----TableScan: multiple_ordered_table projection=[a, c, d] +physical_plan +ProjectionExec: expr=[FIRST_VALUE(multiple_ordered_table.a) ORDER BY [multiple_ordered_table.a ASC NULLS LAST]@1 as first_a, LAST_VALUE(multiple_ordered_table.c) ORDER BY [multiple_ordered_table.c DESC NULLS FIRST]@2 as last_c] +--AggregateExec: mode=FinalPartitioned, gby=[d@0 as d], aggr=[FIRST_VALUE(multiple_ordered_table.a), FIRST_VALUE(multiple_ordered_table.c)] +----CoalesceBatchesExec: target_batch_size=8192 +------RepartitionExec: partitioning=Hash([d@0], 8), input_partitions=8 +--------AggregateExec: mode=Partial, gby=[d@2 as d], aggr=[FIRST_VALUE(multiple_ordered_table.a), FIRST_VALUE(multiple_ordered_table.c)] +----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 +------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true + +query II rowsort +SELECT FIRST_VALUE(a ORDER BY a ASC) as first_a, + LAST_VALUE(c ORDER BY c DESC) as last_c +FROM multiple_ordered_table +GROUP BY d; +---- +0 0 +0 1 +0 15 +0 4 +0 9 query TT EXPLAIN SELECT c From 8cd941da4f3968d78a85a5c248782dad42f6f4f7 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 12 Oct 2023 14:14:05 +0300 Subject: [PATCH 055/122] Minor changes --- datafusion/sqllogictest/test_files/groupby.slt | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/datafusion/sqllogictest/test_files/groupby.slt b/datafusion/sqllogictest/test_files/groupby.slt index 7f03da41c573..055661b59dc4 100644 --- a/datafusion/sqllogictest/test_files/groupby.slt +++ b/datafusion/sqllogictest/test_files/groupby.slt @@ -3623,17 +3623,17 @@ ProjectionExec: expr=[FIRST_VALUE(multiple_ordered_table.a) ORDER BY [multiple_o ----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 ------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true -query II rowsort -SELECT FIRST_VALUE(a ORDER BY a ASC) as first_a, +query III rowsort +SELECT d, FIRST_VALUE(a ORDER BY a ASC) as first_a, LAST_VALUE(c ORDER BY c DESC) as last_c FROM multiple_ordered_table GROUP BY d; ---- -0 0 -0 1 -0 15 -0 4 -0 9 +0 0 0 +1 0 4 +2 0 1 +3 0 15 +4 0 9 query TT EXPLAIN SELECT c From 4b2d6c1b9e3c6ad9c398f74b710defc226a44509 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 13 Oct 2023 09:52:39 +0300 Subject: [PATCH 056/122] Address reviews --- datafusion/physical-expr/src/equivalence.rs | 495 +++++++++--------- .../physical-plan/src/joins/cross_join.rs | 6 +- .../physical-plan/src/joins/hash_join.rs | 6 +- .../src/joins/sort_merge_join.rs | 6 +- .../src/joins/symmetric_hash_join.rs | 6 +- 5 files changed, 261 insertions(+), 258 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 5a95c185aab0..ac736d41651e 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -712,6 +712,24 @@ impl SchemaProperties { self } + /// Re-creates `SchemaProperties` given that + /// schema is re-ordered by `sort_expr` in the argument. + pub fn with_reorder(mut self, sort_expr: Vec) -> SchemaProperties { + // TODO: In some cases, existing ordering equivalences may still be valid add this analysis. + + // Normalize sort_expr according to equivalences + let sort_expr = self.eq_groups.normalize_sort_exprs(&sort_expr); + + // Remove redundant entries from the lex ordering. + let sort_expr = collapse_lex_ordering(sort_expr); + + // Reset ordering equivalent group with the new ordering. + // Constants, and equivalent groups are still valid after re-sort. + // Hence only `oeq_group` is overwritten. + self.oeq_group = OrderingEquivalentGroup::new(vec![sort_expr]); + self + } + /// Transform `sort_exprs` vector, to standardized version using `eq_groups` and `oeq_group` /// Assume `eq_groups` states that `Column a` and `Column b` are aliases. /// Also assume `oeq_group` states that ordering `vec![d ASC]` and `vec![a ASC, c ASC]` are @@ -757,10 +775,163 @@ impl SchemaProperties { collapse_lex_req(normalized_sort_reqs) } + /// Checks whether given ordering requirements are satisfied by provided [PhysicalSortExpr]s. + pub fn ordering_satisfy(&self, required: Option<&[PhysicalSortExpr]>) -> bool { + match required { + None => true, + Some(required) => self.ordering_satisfy_concrete(required), + } + } + + /// Checks whether the required [`PhysicalSortExpr`]s are satisfied by the + /// any of the existing orderings. + pub fn ordering_satisfy_concrete(&self, required: &[PhysicalSortExpr]) -> bool { + // Convert `PhysicalSortExpr`s to `PhysicalSortRequirement`s + let sort_requirements = PhysicalSortRequirement::from_sort_exprs(required.iter()); + self.ordering_satisfy_requirement_concrete(&sort_requirements) + } + + /// Checks whether the given [`PhysicalSortRequirement`]s are satisfied by the + /// provided [`PhysicalSortExpr`]s. + pub fn ordering_satisfy_requirement( + &self, + required: Option<&[PhysicalSortRequirement]>, + ) -> bool { + match required { + None => true, + Some(required) => self.ordering_satisfy_requirement_concrete(required), + } + } + + /// Checks whether the given [`PhysicalSortRequirement`]s are satisfied by the + /// provided [`PhysicalSortExpr`]s. + pub fn ordering_satisfy_requirement_concrete( + &self, + required: &[PhysicalSortRequirement], + ) -> bool { + self.prune_lex_req(required).is_empty() + } + + /// Checks whether the given [`PhysicalSortRequirement`]s are equal or more + /// specific than the provided [`PhysicalSortRequirement`]s. + pub fn requirements_compatible( + &self, + provided: Option<&[PhysicalSortRequirement]>, + required: Option<&[PhysicalSortRequirement]>, + ) -> bool { + match (provided, required) { + (_, None) => true, + (None, Some(_)) => false, + (Some(provided), Some(required)) => { + self.requirements_compatible_concrete(provided, required) + } + } + } + + /// Checks whether the given [`PhysicalSortRequirement`]s are equal or more + /// specific than the provided [`PhysicalSortRequirement`]s. + fn requirements_compatible_concrete( + &self, + provided: &[PhysicalSortRequirement], + required: &[PhysicalSortRequirement], + ) -> bool { + let provided_normalized = self.normalize_sort_requirements(provided); + let required_normalized = self.normalize_sort_requirements(required); + + if required_normalized.len() > provided_normalized.len() { + return false; + } + required_normalized + .into_iter() + .zip(provided_normalized) + .all(|(req, given)| given.compatible(&req)) + } + + /// Find the finer requirement among `req1` and `req2` + /// If `None`, this means that `req1` and `req2` are not compatible + /// e.g there is no requirement that satisfies both + pub fn get_finer_ordering<'a>( + &self, + req1: &'a [PhysicalSortExpr], + req2: &'a [PhysicalSortExpr], + ) -> Option<&'a [PhysicalSortExpr]> { + let lhs = self.normalize_sort_exprs(req1); + let rhs = self.normalize_sort_exprs(req2); + if izip!(lhs.iter(), rhs.iter()).all(|(lhs, rhs)| lhs.eq(rhs)) { + if lhs.len() > rhs.len() { + return Some(req1); + } else { + return Some(req2); + } + } + // Neither `provided` nor `req` satisfies one another, they are incompatible. + None + } + + /// Find the coarser requirement among `req1` and `req2` + /// If `None`, this means that `req1` and `req2` are not compatible + pub fn get_meet_ordering<'a>( + &self, + req1: &'a [PhysicalSortExpr], + req2: &'a [PhysicalSortExpr], + ) -> Option<&'a [PhysicalSortExpr]> { + let lhs = self.normalize_sort_exprs(req1); + let rhs = self.normalize_sort_exprs(req2); + if izip!(lhs.iter(), rhs.iter()).all(|(lhs, rhs)| lhs.eq(rhs)) { + if lhs.len() < rhs.len() { + return Some(req1); + } else { + return Some(req2); + } + } + // Neither `provided` nor `req` satisfies one another, they are incompatible. + None + } + + /// Find the finer requirement among `req1` and `req2` + /// If `None`, this means that `req1` and `req2` are not compatible + /// e.g there is no requirement that satisfies both + pub fn get_finer_requirement<'a>( + &self, + req1: &'a [PhysicalSortRequirement], + req2: &'a [PhysicalSortRequirement], + ) -> Option<&'a [PhysicalSortRequirement]> { + let lhs = self.normalize_sort_requirements(req1); + let rhs = self.normalize_sort_requirements(req2); + let mut left_finer = false; + let mut right_finer = false; + if izip!(lhs.iter(), rhs.iter()).all(|(lhs, rhs)| { + match (lhs.options, rhs.options) { + (Some(lhs_opt), Some(rhs_opt)) => { + lhs.expr.eq(&rhs.expr) && lhs_opt == rhs_opt + } + (Some(_), None) => { + left_finer = true; + lhs.expr.eq(&rhs.expr) + } + (None, Some(_)) => { + right_finer = true; + lhs.expr.eq(&rhs.expr) + } + (None, None) => lhs.expr.eq(&rhs.expr), + } + }) { + if lhs.len() >= rhs.len() && !right_finer { + return Some(req1); + } else if rhs.len() >= lhs.len() && !left_finer { + return Some(req2); + } + } + // Neither `provided` nor `req` satisfies one another, they are incompatible. + None + } + /// This function prunes lexicographical ordering requirement /// by removing sections inside `sort_req` that satisfies any of the existing ordering. - /// By doing so, we reduce the requirement to its simplest form which is functionally - /// equivalent to the argument. Empty result means that requirement is already satisfied. + /// Please note that pruned version may not functionally equivalent to the argument. + /// Empty result means that requirement is already satisfied. + /// Non-empty result means that requirement is not satisfied. + /// This util shouldn't e used outside this context. fn prune_lex_req(&self, sort_req: &[PhysicalSortRequirement]) -> LexOrderingReq { // Make sure to use a standardized version of the requirement let mut sort_req = self.normalize_sort_requirements(sort_req); @@ -769,7 +940,7 @@ impl SchemaProperties { if sort_req.is_empty() { return sort_req; } - let leading_requirement = sort_req[0].clone(); + for ordering in self.oeq_group.iter() { // Normalize existing ordering let ordering = self.normalize_sort_exprs(ordering); @@ -801,13 +972,7 @@ impl SchemaProperties { sort_req.remove(*idx); } } - if !sort_req.is_empty() { - // Do not invalidate requirement - sort_req.insert(0, leading_requirement); - sort_req - } else { - sort_req - } + sort_req } /// Checks whether `leading_requirement` is contained in any of the ordering @@ -870,24 +1035,6 @@ impl SchemaProperties { projected_properties } - /// Re-creates `SchemaProperties` given that - /// schema is re-ordered by `sort_expr` in the argument. - pub fn with_reorder(mut self, sort_expr: Vec) -> SchemaProperties { - // TODO: In some cases, existing ordering equivalences may still be valid add this analysis. - - // Normalize sort_expr according to equivalences - let sort_expr = self.eq_groups.normalize_sort_exprs(&sort_expr); - - // Remove redundant entries from the lex ordering. - let sort_expr = collapse_lex_ordering(sort_expr); - - // Reset ordering equivalent group with the new ordering. - // Constants, and equivalent groups are still valid after re-sort. - // Hence only `oeq_group` is overwritten. - self.oeq_group = OrderingEquivalentGroup::new(vec![sort_expr]); - self - } - /// Check whether any permutation of the argument has a prefix with existing ordering. /// Return indices that describes ordering and their ordering information. pub fn set_satisfy(&self, exprs: &[Arc]) -> Option> { @@ -967,238 +1114,86 @@ impl SchemaProperties { } None } +} - /// Checks whether given ordering requirements are satisfied by provided [PhysicalSortExpr]s. - pub fn ordering_satisfy(&self, required: Option<&[PhysicalSortExpr]>) -> bool { - match required { - None => true, - Some(required) => self.ordering_satisfy_concrete(required), +/// Calculate ordering equivalence properties for the given join operation. +pub fn join_schema_properties( + left: &SchemaProperties, + right: &SchemaProperties, + join_type: &JoinType, + join_schema: SchemaRef, + maintains_input_order: &[bool], + probe_side: Option, + on: &[(Column, Column)], +) -> Result { + let left_columns_len = left.schema.fields.len(); + let mut new_properties = SchemaProperties::new(join_schema); + + let join_eq_groups = + left.eq_groups() + .join(join_type, right.eq_groups(), left_columns_len, on)?; + new_properties.add_equivalent_groups(join_eq_groups); + + // All joins have 2 children + assert_eq!(maintains_input_order.len(), 2); + let left_maintains = maintains_input_order[0]; + let right_maintains = maintains_input_order[1]; + let left_oeq_class = left.oeq_group(); + let right_oeq_class = right.oeq_group(); + match (left_maintains, right_maintains) { + (true, true) => { + return Err(DataFusionError::Plan( + "Cannot maintain ordering of both sides".to_string(), + )) } - } - - /// Checks whether the required [`PhysicalSortExpr`]s are satisfied by the - /// any of the existing orderings. - pub fn ordering_satisfy_concrete(&self, required: &[PhysicalSortExpr]) -> bool { - // Convert `PhysicalSortExpr`s to `PhysicalSortRequirement`s - let sort_requirements = PhysicalSortRequirement::from_sort_exprs(required.iter()); - self.ordering_satisfy_requirement_concrete(&sort_requirements) - } + (true, false) => { + // In this special case, right side ordering can be prefixed with left side ordering. + if let (Some(JoinSide::Left), JoinType::Inner) = (probe_side, join_type) { + let updated_right_oeq = get_updated_right_ordering_equivalent_group( + join_type, + right_oeq_class, + left_columns_len, + )?; - /// Find the finer requirement among `req1` and `req2` - /// If `None`, this means that `req1` and `req2` are not compatible - /// e.g there is no requirement that satisfies both - pub fn get_finer_ordering<'a>( - &self, - req1: &'a [PhysicalSortExpr], - req2: &'a [PhysicalSortExpr], - ) -> Option<&'a [PhysicalSortExpr]> { - let lhs = self.normalize_sort_exprs(req1); - let rhs = self.normalize_sort_exprs(req2); - if izip!(lhs.iter(), rhs.iter()).all(|(lhs, rhs)| lhs.eq(rhs)) { - if lhs.len() > rhs.len() { - return Some(req1); + // Right side ordering equivalence properties should be prepended with + // those of the left side while constructing output ordering equivalence + // properties since stream side is the left side. + // + // If the right table ordering equivalences contain `b ASC`, and the output + // ordering of the left table is `a ASC`, then the ordering equivalence `b ASC` + // for the right table should be converted to `a ASC, b ASC` before it is added + // to the ordering equivalences of the join. + let out_oeq_class = left_oeq_class.join_postfix(&updated_right_oeq); + new_properties.add_ordering_equivalent_group(out_oeq_class); } else { - return Some(req2); + new_properties.add_ordering_equivalent_group(left_oeq_class.clone()); } } - // Neither `provided` nor `req` satisfies one another, they are incompatible. - None - } - - /// Find the coarser requirement among `req1` and `req2` - /// If `None`, this means that `req1` and `req2` are not compatible - pub fn get_meet_ordering<'a>( - &self, - req1: &'a [PhysicalSortExpr], - req2: &'a [PhysicalSortExpr], - ) -> Option<&'a [PhysicalSortExpr]> { - let lhs = self.normalize_sort_exprs(req1); - let rhs = self.normalize_sort_exprs(req2); - if izip!(lhs.iter(), rhs.iter()).all(|(lhs, rhs)| lhs.eq(rhs)) { - if lhs.len() < rhs.len() { - return Some(req1); + (false, true) => { + let updated_right_oeq = get_updated_right_ordering_equivalent_group( + join_type, + right.oeq_group(), + left_columns_len, + )?; + // In this special case, left side ordering can be prefixed with right side ordering. + if let (Some(JoinSide::Right), JoinType::Inner) = (probe_side, join_type) { + // Left side ordering equivalence properties should be prepended with + // those of the right side while constructing output ordering equivalence + // properties since stream side is the right side. + // + // If the right table ordering equivalences contain `b ASC`, and the output + // ordering of the left table is `a ASC`, then the ordering equivalence `b ASC` + // for the right table should be converted to `a ASC, b ASC` before it is added + // to the ordering equivalences of the join. + let out_oeq_class = updated_right_oeq.join_postfix(left_oeq_class); + new_properties.add_ordering_equivalent_group(out_oeq_class); } else { - return Some(req2); - } - } - // Neither `provided` nor `req` satisfies one another, they are incompatible. - None - } - - /// Find the finer requirement among `req1` and `req2` - /// If `None`, this means that `req1` and `req2` are not compatible - /// e.g there is no requirement that satisfies both - pub fn get_finer_requirement<'a>( - &self, - req1: &'a [PhysicalSortRequirement], - req2: &'a [PhysicalSortRequirement], - ) -> Option<&'a [PhysicalSortRequirement]> { - let lhs = self.normalize_sort_requirements(req1); - let rhs = self.normalize_sort_requirements(req2); - let mut left_finer = false; - let mut right_finer = false; - if izip!(lhs.iter(), rhs.iter()).all(|(lhs, rhs)| { - match (lhs.options, rhs.options) { - (Some(lhs_opt), Some(rhs_opt)) => { - lhs.expr.eq(&rhs.expr) && lhs_opt == rhs_opt - } - (Some(_), None) => { - left_finer = true; - lhs.expr.eq(&rhs.expr) - } - (None, Some(_)) => { - right_finer = true; - lhs.expr.eq(&rhs.expr) - } - (None, None) => lhs.expr.eq(&rhs.expr), - } - }) { - if lhs.len() >= rhs.len() && !right_finer { - return Some(req1); - } else if rhs.len() >= lhs.len() && !left_finer { - return Some(req2); - } - } - // Neither `provided` nor `req` satisfies one another, they are incompatible. - None - } - - /// Checks whether the given [`PhysicalSortRequirement`]s are satisfied by the - /// provided [`PhysicalSortExpr`]s. - pub fn ordering_satisfy_requirement( - &self, - required: Option<&[PhysicalSortRequirement]>, - ) -> bool { - match required { - None => true, - Some(required) => self.ordering_satisfy_requirement_concrete(required), - } - } - - /// Checks whether the given [`PhysicalSortRequirement`]s are satisfied by the - /// provided [`PhysicalSortExpr`]s. - pub fn ordering_satisfy_requirement_concrete( - &self, - required: &[PhysicalSortRequirement], - ) -> bool { - self.prune_lex_req(required).is_empty() - } - - /// Checks whether the given [`PhysicalSortRequirement`]s are equal or more - /// specific than the provided [`PhysicalSortRequirement`]s. - pub fn requirements_compatible( - &self, - provided: Option<&[PhysicalSortRequirement]>, - required: Option<&[PhysicalSortRequirement]>, - ) -> bool { - match (provided, required) { - (_, None) => true, - (None, Some(_)) => false, - (Some(provided), Some(required)) => { - self.requirements_compatible_concrete(provided, required) - } - } - } - - /// Checks whether the given [`PhysicalSortRequirement`]s are equal or more - /// specific than the provided [`PhysicalSortRequirement`]s. - fn requirements_compatible_concrete( - &self, - provided: &[PhysicalSortRequirement], - required: &[PhysicalSortRequirement], - ) -> bool { - let provided_normalized = self.normalize_sort_requirements(provided); - let required_normalized = self.normalize_sort_requirements(required); - - if required_normalized.len() > provided_normalized.len() { - return false; - } - required_normalized - .into_iter() - .zip(provided_normalized) - .all(|(req, given)| given.compatible(&req)) - } - - /// Calculate ordering equivalence properties for the given join operation. - pub fn join( - &self, - join_type: &JoinType, - right: &SchemaProperties, - join_schema: SchemaRef, - maintains_input_order: &[bool], - probe_side: Option, - on: &[(Column, Column)], - ) -> Result { - let left_columns_len = self.schema.fields.len(); - let mut new_properties = SchemaProperties::new(join_schema); - - let join_eq_groups = - self.eq_groups() - .join(join_type, right.eq_groups(), left_columns_len, on)?; - new_properties.add_equivalent_groups(join_eq_groups); - - // All joins have 2 children - assert_eq!(maintains_input_order.len(), 2); - let left_maintains = maintains_input_order[0]; - let right_maintains = maintains_input_order[1]; - let left_oeq_class = self.oeq_group(); - let right_oeq_class = right.oeq_group(); - match (left_maintains, right_maintains) { - (true, true) => { - return Err(DataFusionError::Plan( - "Cannot maintain ordering of both sides".to_string(), - )) - } - (true, false) => { - // In this special case, right side ordering can be prefixed with left side ordering. - if let (Some(JoinSide::Left), JoinType::Inner) = (probe_side, join_type) { - let updated_right_oeq = get_updated_right_ordering_equivalent_group( - join_type, - right_oeq_class, - left_columns_len, - )?; - - // Right side ordering equivalence properties should be prepended with - // those of the left side while constructing output ordering equivalence - // properties since stream side is the left side. - // - // If the right table ordering equivalences contain `b ASC`, and the output - // ordering of the left table is `a ASC`, then the ordering equivalence `b ASC` - // for the right table should be converted to `a ASC, b ASC` before it is added - // to the ordering equivalences of the join. - let out_oeq_class = left_oeq_class.join_postfix(&updated_right_oeq); - new_properties.add_ordering_equivalent_group(out_oeq_class); - } else { - new_properties.add_ordering_equivalent_group(left_oeq_class.clone()); - } - } - (false, true) => { - let updated_right_oeq = get_updated_right_ordering_equivalent_group( - join_type, - right.oeq_group(), - left_columns_len, - )?; - // In this special case, left side ordering can be prefixed with right side ordering. - if let (Some(JoinSide::Right), JoinType::Inner) = (probe_side, join_type) - { - // Left side ordering equivalence properties should be prepended with - // those of the right side while constructing output ordering equivalence - // properties since stream side is the right side. - // - // If the right table ordering equivalences contain `b ASC`, and the output - // ordering of the left table is `a ASC`, then the ordering equivalence `b ASC` - // for the right table should be converted to `a ASC, b ASC` before it is added - // to the ordering equivalences of the join. - let out_oeq_class = updated_right_oeq.join_postfix(left_oeq_class); - new_properties.add_ordering_equivalent_group(out_oeq_class); - } else { - new_properties.add_ordering_equivalent_group(updated_right_oeq); - } + new_properties.add_ordering_equivalent_group(updated_right_oeq); } - (false, false) => {} } - Ok(new_properties) + (false, false) => {} } + Ok(new_properties) } /// Constructs a `SchemaProperties` struct from the given `orderings`. diff --git a/datafusion/physical-plan/src/joins/cross_join.rs b/datafusion/physical-plan/src/joins/cross_join.rs index 466299d6062e..84aa1e53a819 100644 --- a/datafusion/physical-plan/src/joins/cross_join.rs +++ b/datafusion/physical-plan/src/joins/cross_join.rs @@ -37,6 +37,7 @@ use datafusion_common::{plan_err, DataFusionError, JoinType}; use datafusion_common::{Result, ScalarValue}; use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; use datafusion_execution::TaskContext; +use datafusion_physical_expr::equivalence::join_schema_properties; use datafusion_physical_expr::SchemaProperties; use super::utils::{ @@ -217,9 +218,10 @@ impl ExecutionPlan for CrossJoinExec { fn schema_properties(&self) -> SchemaProperties { let left = self.left.schema_properties(); let right = self.right.schema_properties(); - left.join( - &JoinType::Full, + join_schema_properties( + &left, &right, + &JoinType::Full, self.schema(), &[false, false], None, diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index 3d5409547bce..a45375a72d22 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -71,6 +71,7 @@ use datafusion_physical_expr::SchemaProperties; use ahash::RandomState; use arrow::compute::kernels::cmp::{eq, not_distinct}; +use datafusion_physical_expr::equivalence::join_schema_properties; use futures::{ready, Stream, StreamExt, TryStreamExt}; type JoinLeftData = (JoinHashMap, RecordBatch, MemoryReservation); @@ -367,9 +368,10 @@ impl ExecutionPlan for HashJoinExec { fn schema_properties(&self) -> SchemaProperties { let left = self.left.schema_properties(); let right = self.right.schema_properties(); - left.join( - &self.join_type, + join_schema_properties( + &left, &right, + &self.join_type, self.schema(), &self.maintains_input_order(), Some(Self::probe_side()), diff --git a/datafusion/physical-plan/src/joins/sort_merge_join.rs b/datafusion/physical-plan/src/joins/sort_merge_join.rs index e187ced569d4..ee9b8b86f516 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join.rs @@ -54,6 +54,7 @@ use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; use datafusion_execution::TaskContext; use datafusion_physical_expr::{PhysicalSortRequirement, SchemaProperties}; +use datafusion_physical_expr::equivalence::join_schema_properties; use futures::{Stream, StreamExt}; /// join execution plan executes partitions in parallel and combines them into a set of @@ -284,9 +285,10 @@ impl ExecutionPlan for SortMergeJoinExec { fn schema_properties(&self) -> SchemaProperties { let left = self.left.schema_properties(); let right = self.right.schema_properties(); - left.join( - &self.join_type, + join_schema_properties( + &left, &right, + &self.join_type, self.schema(), &self.maintains_input_order(), Some(Self::probe_side(&self.join_type)), diff --git a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs index 94e385242487..7befe3caac60 100644 --- a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs +++ b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs @@ -69,6 +69,7 @@ use datafusion_physical_expr::intervals::ExprIntervalGraph; use crate::joins::utils::prepare_sorted_exprs; use ahash::RandomState; +use datafusion_physical_expr::equivalence::join_schema_properties; use datafusion_physical_expr::SchemaProperties; use futures::stream::{select, BoxStream}; use futures::{Stream, StreamExt}; @@ -435,9 +436,10 @@ impl ExecutionPlan for SymmetricHashJoinExec { fn schema_properties(&self) -> SchemaProperties { let left = self.left.schema_properties(); let right = self.right.schema_properties(); - left.join( - &self.join_type, + join_schema_properties( + &left, &right, + &self.join_type, self.schema(), &self.maintains_input_order(), // Has alternating probe side From 3309567cc341d3f523e699a26044d4a26a5a040c Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 13 Oct 2023 09:58:45 +0300 Subject: [PATCH 057/122] Minor changes --- datafusion/physical-expr/src/equivalence.rs | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index ac736d41651e..bc7ffc2f8ce5 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -738,7 +738,7 @@ impl SchemaProperties { /// This function converts `sort_exprs` `vec![b ASC, c ASC]` to first `vec![a ASC, c ASC]` after considering `eq_groups` /// Then converts `vec![a ASC, c ASC]` to `vec![d ASC]` after considering `oeq_group`. /// Standardized version `vec![d ASC]` is used in subsequent operations. - pub fn normalize_sort_exprs( + fn normalize_sort_exprs( &self, sort_exprs: &[PhysicalSortExpr], ) -> Vec { @@ -764,7 +764,7 @@ impl SchemaProperties { /// This function converts `sort_exprs` `vec![b Some(ASC), c None]` to first `vec![a Some(ASC), c None]` after considering `eq_groups` /// Then converts `vec![a Some(ASC), c None]` to `vec![d Some(ASC)]` after considering `oeq_group`. /// Standardized version `vec![d Some(ASC)]` is used in subsequent operations. - pub fn normalize_sort_requirements( + fn normalize_sort_requirements( &self, sort_reqs: &[PhysicalSortRequirement], ) -> Vec { @@ -1234,7 +1234,7 @@ pub fn collapse_lex_req(input: LexOrderingReq) -> LexOrderingReq { /// entries that have same physical expression inside the given vector `input`. /// `vec![a ASC, a DESC]` is collapsed to the `vec![a ASC]`. Since /// when same expression is already seen before, following expressions are redundant. -pub fn collapse_lex_ordering(input: LexOrdering) -> LexOrdering { +fn collapse_lex_ordering(input: LexOrdering) -> LexOrdering { let mut output = vec![]; for item in input { if output @@ -1261,7 +1261,7 @@ fn prune_sort_reqs_with_constants( /// Adds the `offset` value to `Column` indices inside `expr`. This function is /// generally used during the update of the right table schema in join operations. -pub(crate) fn add_offset_to_exprs( +fn add_offset_to_exprs( exprs: Vec>, offset: usize, ) -> Result>> { @@ -1273,7 +1273,7 @@ pub(crate) fn add_offset_to_exprs( /// Adds the `offset` value to `Column` indices inside `expr`. This function is /// generally used during the update of the right table schema in join operations. -pub(crate) fn add_offset_to_expr( +fn add_offset_to_expr( expr: Arc, offset: usize, ) -> Result> { @@ -1287,7 +1287,7 @@ pub(crate) fn add_offset_to_expr( } /// Adds the `offset` value to `Column` indices inside `sort_expr.expr`. -pub(crate) fn add_offset_to_sort_expr( +fn add_offset_to_sort_expr( sort_expr: &PhysicalSortExpr, offset: usize, ) -> Result { @@ -1322,7 +1322,7 @@ pub fn add_offset_to_lex_ordering( /// the intermediate node can be directly matched with the sort expression. If there /// is a match, the sort expression emerges at that node immediately, discarding /// the order coming from the children. -pub fn update_ordering( +fn update_ordering( mut node: ExprOrdering, ordering_equal_properties: &SchemaProperties, ) -> Result> { From 535e1e8e51d514547f6638d59111b86a7a3099e9 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 13 Oct 2023 10:55:28 +0300 Subject: [PATCH 058/122] Increase coverage of random tests --- datafusion/physical-expr/src/equivalence.rs | 144 +++++++++++--------- 1 file changed, 77 insertions(+), 67 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index bc7ffc2f8ce5..2fe64f46b502 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -1390,7 +1390,9 @@ mod tests { use arrow::compute::{lexsort_to_indices, SortColumn}; use arrow_array::{ArrayRef, RecordBatch, UInt32Array, UInt64Array}; use arrow_schema::{Fields, SortOptions}; + use itertools::Itertools; use rand::rngs::StdRng; + use rand::seq::SliceRandom; use rand::{Rng, SeedableRng}; use std::sync::Arc; @@ -1479,12 +1481,12 @@ mod tests { Ok(schema) } - /// Construct a schema with following properties - /// Schema satisfied following orderings: - /// [a ASC, b ASC], [c ASC, d ASC, e ASC] - /// and + /// Construct a schema with random ordering + /// among column a, b, c, d + /// where /// Column [a=f] (e.g they are aliases). - fn create_test_params_2() -> Result<(SchemaRef, SchemaProperties)> { + /// Column e is constant. + fn create_random_schema(seed: u64) -> Result<(SchemaRef, SchemaProperties)> { let test_schema = create_test_schema_2()?; let col_a_expr = &col("a", &test_schema)?; let col_b_expr = &col("b", &test_schema)?; @@ -1493,38 +1495,34 @@ mod tests { let col_e_expr = &col("e", &test_schema)?; let col_f_expr = &col("f", &test_schema)?; let mut schema_properties = SchemaProperties::new(test_schema.clone()); + // a=f schema_properties.add_equal_conditions((&col_a_expr, &col_f_expr)); + // Column e is constant + schema_properties = schema_properties.with_constants(vec![col_e_expr.clone()]); + let mut rng = StdRng::seed_from_u64(seed); + let mut col_exprs = vec![col_a_expr, col_b_expr, col_c_expr, col_d_expr]; let option1 = SortOptions { descending: false, nulls_first: false, }; - schema_properties.add_new_orderings(&[ - vec![ - PhysicalSortExpr { - expr: col_a_expr.clone(), - options: option1, - }, - PhysicalSortExpr { - expr: col_b_expr.clone(), - options: option1, - }, - ], - vec![ - PhysicalSortExpr { - expr: col_c_expr.clone(), - options: option1, - }, - PhysicalSortExpr { - expr: col_d_expr.clone(), - options: option1, - }, - PhysicalSortExpr { - expr: col_e_expr.clone(), + while !col_exprs.is_empty() { + let n_sort_expr = rng.gen_range(0..col_exprs.len() + 1); + let mut indices = (0..col_exprs.len()).collect::>(); + indices.shuffle(&mut rng); + let orderings = indices[0..n_sort_expr] + .iter() + .map(|idx| PhysicalSortExpr { + expr: col_exprs[*idx].clone(), options: option1, - }, - ], - ]); + }) + .collect::>(); + schema_properties.add_new_orderings(&[orderings]); + col_exprs = indices[n_sort_expr..] + .iter() + .map(|idx| col_exprs[*idx]) + .collect(); + } Ok((test_schema, schema_properties)) } @@ -1839,47 +1837,45 @@ mod tests { #[test] fn test_ordering_satisfy_with_equivalence_random() -> Result<()> { - // Number of random tests - let n_test = 10000usize; - let n_req_max = 5usize; + let n_random_schema = 5; let option1 = SortOptions { descending: false, nulls_first: false, }; - // Schema satisfies following orderings: - // [a ASC, b ASC], [c ASC, d ASC, e ASC] - // and - // Column [a=f] (e.g they are aliases). - let (_test_schema, schema_properties) = create_test_params_2()?; - let table_data_with_properties = - generate_table_for_schema_properties(&schema_properties, 625, 5)?; - - // use a random number for values - let mut rng = StdRng::seed_from_u64(23); - let schema = schema_properties.schema(); - let n_schema = schema.fields.len(); - for _test_id in 0..n_test { - let n_req = rng.gen_range(0..n_req_max); - let requirement = (0..n_req) - .map(|_idx| { - let col_idx = rng.gen_range(0..n_schema); - let col_expr = col(schema.fields[col_idx].name(), &schema)?; - Ok(PhysicalSortExpr { - expr: col_expr, - options: option1, - }) - }) - .collect::>>()?; - let err_msg = format!("Error in test case:{requirement:?}"); - let expected = is_table_same_after_sort( - requirement.clone(), - table_data_with_properties.clone(), - )?; - assert_eq!( - schema_properties.ordering_satisfy_concrete(&requirement), - expected, - "{err_msg}" - ); + for seed in 0..n_random_schema { + let (test_schema, schema_properties) = create_random_schema(seed)?; + let table_data_with_properties = + generate_table_for_schema_properties(&schema_properties, 125, 5)?; + let col_exprs = vec![ + col("a", &test_schema)?, + col("b", &test_schema)?, + col("c", &test_schema)?, + col("d", &test_schema)?, + col("e", &test_schema)?, + col("f", &test_schema)?, + ]; + + for n_req in 0..col_exprs.len() + 1 { + for exprs in col_exprs.iter().permutations(n_req) { + let requirement = exprs + .into_iter() + .map(|expr| PhysicalSortExpr { + expr: expr.clone(), + options: option1, + }) + .collect::>(); + let expected = is_table_same_after_sort( + requirement.clone(), + table_data_with_properties.clone(), + )?; + let err_msg = format!("Error in test case requirement:{requirement:?}, expected: {expected:?}"); + assert_eq!( + schema_properties.ordering_satisfy_concrete(&requirement), + expected, + "{err_msg}" + ); + } + } } Ok(()) @@ -2194,6 +2190,19 @@ mod tests { let schema = schema_properties.schema(); let mut schema_vec = vec![None; schema.fields.len()]; + // Fill constant columns + for constant in schema_properties.constants.iter() { + let col = constant.as_any().downcast_ref::().unwrap(); + let (idx, _field) = schema.column_with_name(col.name()).unwrap(); + + // all of the values are set to 0. + let arr: Vec = vec![0; n_elem]; + let arr = Arc::new(UInt64Array::from_iter_values(arr)) as ArrayRef; + + schema_vec[idx] = Some(arr); + } + + // Fill ordering expressions for ordering in schema_properties.oeq_group.iter() { let mut sort_columns = vec![]; let mut indices = vec![]; @@ -2217,6 +2226,7 @@ mod tests { } } + // Fill equivalent expressions for eq_group in schema_properties.eq_groups.iter() { let arr = if let Some(arr) = get_representative_arr(eq_group, &schema_vec, schema.clone()) From eb60b5a401aca060518067850f1b4e90e95f139c Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 13 Oct 2023 11:07:59 +0300 Subject: [PATCH 059/122] Remove redundant code --- datafusion/physical-expr/src/equivalence.rs | 23 --------------------- 1 file changed, 23 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 2fe64f46b502..b327a308bf03 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -717,12 +717,6 @@ impl SchemaProperties { pub fn with_reorder(mut self, sort_expr: Vec) -> SchemaProperties { // TODO: In some cases, existing ordering equivalences may still be valid add this analysis. - // Normalize sort_expr according to equivalences - let sort_expr = self.eq_groups.normalize_sort_exprs(&sort_expr); - - // Remove redundant entries from the lex ordering. - let sort_expr = collapse_lex_ordering(sort_expr); - // Reset ordering equivalent group with the new ordering. // Constants, and equivalent groups are still valid after re-sort. // Hence only `oeq_group` is overwritten. @@ -1230,23 +1224,6 @@ pub fn collapse_lex_req(input: LexOrderingReq) -> LexOrderingReq { output } -/// This function constructs a duplicate-free `LexOrdering` by filtering out duplicate -/// entries that have same physical expression inside the given vector `input`. -/// `vec![a ASC, a DESC]` is collapsed to the `vec![a ASC]`. Since -/// when same expression is already seen before, following expressions are redundant. -fn collapse_lex_ordering(input: LexOrdering) -> LexOrdering { - let mut output = vec![]; - for item in input { - if output - .iter() - .all(|elem: &PhysicalSortExpr| !elem.expr.eq(&item.expr)) - { - output.push(item); - } - } - output -} - /// Remove ordering requirements that have constant value fn prune_sort_reqs_with_constants( ordering: &[PhysicalSortRequirement], From b6d50778babf78738ee8939106746cdffed0da0a Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 13 Oct 2023 13:54:13 +0300 Subject: [PATCH 060/122] Simplifications --- datafusion/physical-expr/src/equivalence.rs | 5 +++-- datafusion/physical-plan/src/windows/mod.rs | 3 --- datafusion/sqllogictest/test_files/window.slt | 18 ++++++++++++++++++ 3 files changed, 21 insertions(+), 5 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index b327a308bf03..2af69162514e 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -763,8 +763,9 @@ impl SchemaProperties { sort_reqs: &[PhysicalSortRequirement], ) -> Vec { let normalized_sort_reqs = self.eq_groups.normalize_sort_requirements(sort_reqs); + let constants_normalized = self.eq_groups.normalize_exprs(&self.constants); let normalized_sort_reqs = - prune_sort_reqs_with_constants(&normalized_sort_reqs, &self.constants); + prune_sort_reqs_with_constants(&normalized_sort_reqs, &constants_normalized); // Prune redundant sections in the requirement. collapse_lex_req(normalized_sort_reqs) } @@ -1036,7 +1037,7 @@ impl SchemaProperties { let mut best = vec![]; for ordering in self.oeq_group.iter() { - let ordering = self.eq_groups.normalize_sort_exprs(ordering); + let ordering = self.normalize_sort_exprs(ordering); let ordering_exprs = ordering .iter() .map(|sort_expr| sort_expr.expr.clone()) diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index 6c46f4cdd54d..89c2d6dc99c8 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -374,8 +374,6 @@ pub fn get_best_fitting_window( // of the window_exprs are same. let partitionby_exprs = window_exprs[0].partition_by(); let orderby_keys = window_exprs[0].order_by(); - // println!("partitionby_exprs: {:?}, orderby_keys:{:?}", partitionby_exprs, orderby_keys); - // println!("input.schema_properties().oeq_group():{:?}", input.schema_properties().oeq_group()); let (should_reverse, partition_search_mode) = if let Some((should_reverse, partition_search_mode)) = get_window_mode(partitionby_exprs, orderby_keys, input)? @@ -384,7 +382,6 @@ pub fn get_best_fitting_window( } else { return Ok(None); }; - // println!("should_reverse: {:?}, partition_search_mode:{:?}", should_reverse, partition_search_mode); let is_unbounded = unbounded_output(input); if !is_unbounded && partition_search_mode != PartitionSearchMode::Sorted { // Executor has bounded input and `partition_search_mode` is not `PartitionSearchMode::Sorted` diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index 725f33c3eff2..976823db52cb 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -3416,3 +3416,21 @@ ProjectionExec: expr=[MIN(multiple_ordered_table.d) ORDER BY [multiple_ordered_t ------BoundedWindowAggExec: wdw=[MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MAX(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.b, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow }], mode=[Sorted] --------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], has_header=true +query TT +EXPLAIN SELECT MAX(c) OVER(PARTITION BY d ORDER BY c ASC) as max_c +FROM( + SELECT * + FROM multiple_ordered_table + WHERE d=0) +---- +logical_plan +Projection: MAX(multiple_ordered_table.c) PARTITION BY [multiple_ordered_table.d] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW AS max_c +--WindowAggr: windowExpr=[[MAX(multiple_ordered_table.c) PARTITION BY [multiple_ordered_table.d] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +----Filter: multiple_ordered_table.d = Int32(0) +------TableScan: multiple_ordered_table projection=[c, d], partial_filters=[multiple_ordered_table.d = Int32(0)] +physical_plan +ProjectionExec: expr=[MAX(multiple_ordered_table.c) PARTITION BY [multiple_ordered_table.d] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as max_c] +--BoundedWindowAggExec: wdw=[MAX(multiple_ordered_table.c) PARTITION BY [multiple_ordered_table.d] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "MAX(multiple_ordered_table.c) PARTITION BY [multiple_ordered_table.d] ORDER BY [multiple_ordered_table.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int32, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow }], mode=[Sorted] +----CoalesceBatchesExec: target_batch_size=4096 +------FilterExec: d@1 = 0 +--------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], output_ordering=[c@0 ASC NULLS LAST], has_header=true From 39240b0e5e02d50080c6b606b5dc8a3e11534875 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 13 Oct 2023 15:47:34 +0300 Subject: [PATCH 061/122] Simplifications --- datafusion/physical-expr/src/equivalence.rs | 31 ++++++++++++--------- 1 file changed, 18 insertions(+), 13 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 2af69162514e..c4cc9ec708de 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -659,6 +659,20 @@ impl SchemaProperties { &self.eq_groups } + /// Return the normalized version of the ordering equivalent group + /// Where constants, duplicates are removed and expressions are normalized + /// according to equivalent groups. + pub fn normalized_oeq_group(&self) -> OrderingEquivalentGroup { + // Construct a new ordering group that is normalized + // With equivalences, and constants are removed + let normalized_orderings = self + .oeq_group + .iter() + .map(|ordering| self.normalize_sort_exprs(ordering)) + .collect::>(); + OrderingEquivalentGroup::new(normalized_orderings) + } + /// Add SchemaProperties of the other to the state. pub fn extend(mut self, other: SchemaProperties) -> Self { self.eq_groups.extend(other.eq_groups); @@ -936,9 +950,7 @@ impl SchemaProperties { return sort_req; } - for ordering in self.oeq_group.iter() { - // Normalize existing ordering - let ordering = self.normalize_sort_exprs(ordering); + for ordering in self.normalized_oeq_group().iter() { let match_indices = ordering .iter() .map(|elem| { @@ -976,13 +988,7 @@ impl SchemaProperties { &self, leading_requirement: &PhysicalSortRequirement, ) -> bool { - let leading_requirement = self - .eq_groups - .normalize_sort_requirement(leading_requirement.clone()); - self.oeq_group().iter().any(|ordering| { - let ordering = self.eq_groups.normalize_sort_exprs(ordering); - ordering[0].satisfy_with_schema(&leading_requirement, &self.schema) - }) + self.ordering_satisfy_requirement_concrete(&[leading_requirement.clone()]) } /// Projects `SchemaProperties` according to mapping given in `source_to_target_mapping`. @@ -1036,8 +1042,7 @@ impl SchemaProperties { let exprs_normalized = self.eq_groups.normalize_exprs(exprs); let mut best = vec![]; - for ordering in self.oeq_group.iter() { - let ordering = self.normalize_sort_exprs(ordering); + for ordering in self.normalized_oeq_group().iter() { let ordering_exprs = ordering .iter() .map(|sort_expr| sort_expr.expr.clone()) @@ -1092,7 +1097,7 @@ impl SchemaProperties { exprs: &[Arc], ) -> Option> { let normalized_exprs = self.eq_groups.normalize_exprs(exprs); - for ordering in self.oeq_group.iter() { + for ordering in self.normalized_oeq_group().iter() { if normalized_exprs.len() <= ordering.len() { let mut ordering_options = vec![]; for (expr, sort_expr) in izip!(normalized_exprs.iter(), ordering.iter()) { From a370104cf16793081b7497b19dd797276be2b410 Mon Sep 17 00:00:00 2001 From: metesynnada <100111937+metesynnada@users.noreply.github.com> Date: Fri, 13 Oct 2023 17:23:41 +0300 Subject: [PATCH 062/122] Refactor on tests --- datafusion/physical-expr/src/equivalence.rs | 244 +++++++++++--------- 1 file changed, 137 insertions(+), 107 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index c4cc9ec708de..29b8048bf277 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -1471,41 +1471,45 @@ mod tests { /// Column e is constant. fn create_random_schema(seed: u64) -> Result<(SchemaRef, SchemaProperties)> { let test_schema = create_test_schema_2()?; - let col_a_expr = &col("a", &test_schema)?; - let col_b_expr = &col("b", &test_schema)?; - let col_c_expr = &col("c", &test_schema)?; - let col_d_expr = &col("d", &test_schema)?; - let col_e_expr = &col("e", &test_schema)?; - let col_f_expr = &col("f", &test_schema)?; + + let col_exprs = vec![ + col("a", &test_schema)?, + col("b", &test_schema)?, + col("c", &test_schema)?, + col("d", &test_schema)?, + col("e", &test_schema)?, + col("f", &test_schema)?, + ]; + let mut schema_properties = SchemaProperties::new(test_schema.clone()); - // a=f - schema_properties.add_equal_conditions((&col_a_expr, &col_f_expr)); - // Column e is constant - schema_properties = schema_properties.with_constants(vec![col_e_expr.clone()]); + // Define equivalent columns and constant columns + schema_properties.add_equal_conditions((&col_exprs[0], &col_exprs[5])); + schema_properties = schema_properties.with_constants(vec![col_exprs[4].clone()]); + // Randomly order columns for sorting let mut rng = StdRng::seed_from_u64(seed); - let mut col_exprs = vec![col_a_expr, col_b_expr, col_c_expr, col_d_expr]; - let option1 = SortOptions { + let mut remaining_exprs = col_exprs[0..4].to_vec(); // only a, b, c, d are sorted + + let sort_options = SortOptions { descending: false, nulls_first: false, }; - while !col_exprs.is_empty() { - let n_sort_expr = rng.gen_range(0..col_exprs.len() + 1); - let mut indices = (0..col_exprs.len()).collect::>(); - indices.shuffle(&mut rng); - let orderings = indices[0..n_sort_expr] - .iter() - .map(|idx| PhysicalSortExpr { - expr: col_exprs[*idx].clone(), - options: option1, + + while !remaining_exprs.is_empty() { + let n_sort_expr = rng.gen_range(0..remaining_exprs.len() + 1); + remaining_exprs.shuffle(&mut rng); + + let orderings: Vec<_> = remaining_exprs + .drain(0..n_sort_expr) + .map(|expr| PhysicalSortExpr { + expr, + options: sort_options, }) - .collect::>(); - schema_properties.add_new_orderings(&[orderings]); - col_exprs = indices[n_sort_expr..] - .iter() - .map(|idx| col_exprs[*idx]) .collect(); + + schema_properties.add_new_orderings(&[orderings]); } + Ok((test_schema, schema_properties)) } @@ -1820,15 +1824,21 @@ mod tests { #[test] fn test_ordering_satisfy_with_equivalence_random() -> Result<()> { - let n_random_schema = 5; - let option1 = SortOptions { + const N_RANDOM_SCHEMA: usize = 5; + const N_ELEMENTS: usize = 125; + const N_DISTINCT: usize = 5; + const SORT_OPTIONS: SortOptions = SortOptions { descending: false, nulls_first: false, }; - for seed in 0..n_random_schema { - let (test_schema, schema_properties) = create_random_schema(seed)?; - let table_data_with_properties = - generate_table_for_schema_properties(&schema_properties, 125, 5)?; + + for seed in 0..N_RANDOM_SCHEMA { + let (test_schema, schema_properties) = create_random_schema(seed as u64)?; + let table_data_with_properties = generate_table_for_schema_properties( + &schema_properties, + N_ELEMENTS, + N_DISTINCT, + )?; let col_exprs = vec![ col("a", &test_schema)?, col("b", &test_schema)?, @@ -1838,24 +1848,28 @@ mod tests { col("f", &test_schema)?, ]; - for n_req in 0..col_exprs.len() + 1 { - for exprs in col_exprs.iter().permutations(n_req) { + for n_req in 0..=col_exprs.len() { + for exprs in col_exprs.iter().combinations(n_req) { let requirement = exprs .into_iter() .map(|expr| PhysicalSortExpr { expr: expr.clone(), - options: option1, + options: SORT_OPTIONS, }) .collect::>(); let expected = is_table_same_after_sort( requirement.clone(), table_data_with_properties.clone(), )?; - let err_msg = format!("Error in test case requirement:{requirement:?}, expected: {expected:?}"); + let err_msg = format!( + "Error in test case requirement:{:?}, expected: {:?}", + requirement, expected + ); assert_eq!( schema_properties.ordering_satisfy_concrete(&requirement), expected, - "{err_msg}" + "{}", + err_msg ); } } @@ -2103,47 +2117,66 @@ mod tests { Ok(()) } - // Check whether table will stay the same after ordered according to requirement - // given. If so it means that required ordering is already satisfied (according to - // random data). + /// Checks if the table (RecordBatch) remains unchanged when sorted according to the provided `required_ordering`. + /// + /// The function works by adding a unique column of ascending integers to the original table. This column ensures + /// that rows that are otherwise indistinguishable (e.g., if they have the same values in all other columns) can + /// still be differentiated. When sorting the extended table, the unique column acts as a tie-breaker to produce + /// deterministic sorting results. + /// + /// If the table remains the same after sorting with the added unique column, it indicates that the table was + /// already sorted according to `required_ordering` to begin with. fn is_table_same_after_sort( mut required_ordering: Vec, batch: RecordBatch, ) -> Result { - let schema = batch.schema(); + // Clone the original schema and columns + let original_schema = batch.schema(); + let mut columns = batch.columns().to_vec(); + + // Create a new unique column let n_row = batch.num_rows() as u64; - let mut sort_columns = vec![]; - let new_arr = Arc::new(UInt64Array::from_iter_values(0..n_row)) as ArrayRef; - let mut cols = batch.columns().to_vec(); - cols.push(new_arr); - let mut fields = schema.fields.to_vec(); - let new_col_expr = - Arc::new(Column::new("unique", fields.len())) as Arc; - fields.push(Arc::new(Field::new("unique", DataType::UInt64, false))); + let unique_col = Arc::new(UInt64Array::from_iter_values(0..n_row)) as ArrayRef; + columns.push(unique_col.clone()); + + // Create a new schema with the added unique column + let unique_col_name = "unique"; + let unique_field = Arc::new(Field::new(unique_col_name, DataType::UInt64, false)); + let fields: Vec<_> = original_schema + .fields() + .iter() + .cloned() + .chain(std::iter::once(unique_field)) + .collect(); let schema = Arc::new(Schema::new(fields)); - let batch = RecordBatch::try_new(schema.clone(), cols)?; - // Add a unique ordering to the requirement to make resulting indices deterministic + // Create a new batch with the added column + let new_batch = RecordBatch::try_new(schema.clone(), columns)?; + + // Add the unique column to the required ordering to ensure deterministic results required_ordering.push(PhysicalSortExpr { - expr: new_col_expr, + expr: Arc::new(Column::new(unique_col_name, original_schema.fields().len())), options: Default::default(), }); - for elem in required_ordering.into_iter() { - let (idx, _field) = schema - .column_with_name( - elem.expr.as_any().downcast_ref::().unwrap().name(), - ) - .unwrap(); - let arr = batch.column(idx); - sort_columns.push(SortColumn { - values: arr.clone(), - options: Some(elem.options), + // Convert the required ordering to a list of SortColumn + let sort_columns: Vec<_> = required_ordering + .iter() + .filter_map(|order_expr| { + let col = order_expr.expr.as_any().downcast_ref::()?; + let col_index = schema.column_with_name(col.name())?.0; + Some(SortColumn { + values: new_batch.column(col_index).clone(), + options: Some(order_expr.options), + }) }) - } - let indices = lexsort_to_indices(&sort_columns, None)?; - let no_change = UInt32Array::from_iter_values(0..n_row as u32); - Ok(indices == no_change) + .collect(); + + // Check if the indices after sorting match the initial ordering + let sorted_indices = lexsort_to_indices(&sort_columns, None)?; + let original_indices = UInt32Array::from_iter_values(0..n_row as u32); + + Ok(&sorted_indices == &original_indices) } fn get_representative_arr( @@ -2167,75 +2200,72 @@ mod tests { n_elem: usize, n_distinct: usize, ) -> Result { - // use a random number for values let mut rng = StdRng::seed_from_u64(23); let schema = schema_properties.schema(); - let mut schema_vec = vec![None; schema.fields.len()]; + + // Utility closure to generate random array + let mut generate_random_array = |num_elems: usize, max_val: usize| -> ArrayRef { + let values: Vec = (0..num_elems) + .map(|_| rng.gen_range(0..max_val) as u64) + .collect(); + Arc::new(UInt64Array::from_iter_values(values)) + }; + // Fill constant columns - for constant in schema_properties.constants.iter() { + for constant in &schema_properties.constants { let col = constant.as_any().downcast_ref::().unwrap(); let (idx, _field) = schema.column_with_name(col.name()).unwrap(); - - // all of the values are set to 0. - let arr: Vec = vec![0; n_elem]; - let arr = Arc::new(UInt64Array::from_iter_values(arr)) as ArrayRef; - + let arr = + Arc::new(UInt64Array::from_iter_values(vec![0; n_elem])) as ArrayRef; schema_vec[idx] = Some(arr); } - // Fill ordering expressions + // Fill columns based on ordering equivalences for ordering in schema_properties.oeq_group.iter() { - let mut sort_columns = vec![]; - let mut indices = vec![]; - for PhysicalSortExpr { expr, options } in ordering { - let col = expr.as_any().downcast_ref::().unwrap(); - let (idx, _field) = schema.column_with_name(col.name()).unwrap(); - let mut arr: Vec = vec![0; n_elem]; - arr.iter_mut().for_each(|v| { - *v = rng.gen_range(0..n_distinct) as u64; - }); - let arr = Arc::new(UInt64Array::from_iter_values(arr)) as ArrayRef; - sort_columns.push(SortColumn { - values: arr, - options: Some(*options), - }); - indices.push(idx); - } + let (sort_columns, indices): (Vec<_>, Vec<_>) = ordering + .iter() + .map(|PhysicalSortExpr { expr, options }| { + let col = expr.as_any().downcast_ref::().unwrap(); + let (idx, _field) = schema.column_with_name(col.name()).unwrap(); + let arr = generate_random_array(n_elem, n_distinct); + ( + SortColumn { + values: arr, + options: Some(*options), + }, + idx, + ) + }) + .unzip(); + let sort_arrs = arrow::compute::lexsort(&sort_columns, None)?; for (idx, arr) in izip!(indices, sort_arrs) { schema_vec[idx] = Some(arr); } } - // Fill equivalent expressions + // Fill columns based on equivalence groups for eq_group in schema_properties.eq_groups.iter() { - let arr = if let Some(arr) = + let representative_array = get_representative_arr(eq_group, &schema_vec, schema.clone()) - { - arr - } else { - let mut arr: Vec = vec![0; n_elem]; - arr.iter_mut().for_each(|v| { - *v = rng.gen_range(0..n_distinct) as u64; - }); - Arc::new(UInt64Array::from_iter_values(arr)) as ArrayRef - }; + .unwrap_or_else(|| generate_random_array(n_elem, n_distinct)); + for expr in eq_group { let col = expr.as_any().downcast_ref::().unwrap(); let (idx, _field) = schema.column_with_name(col.name()).unwrap(); - schema_vec[idx] = Some(arr.clone()); + schema_vec[idx] = Some(representative_array.clone()); } } - let res = schema_vec + let res: Vec<_> = schema_vec .into_iter() .zip(schema.fields.iter()) .map(|(elem, field)| (field.name(), elem.unwrap())) - .collect::>(); - let res = RecordBatch::try_from_iter(res)?; - Ok(res) + .collect(); + + Ok(RecordBatch::try_from_iter(res)?) } #[test] From b1d82612dbef1b8f16fd81e5b700602b1d534d0d Mon Sep 17 00:00:00 2001 From: metesynnada <100111937+metesynnada@users.noreply.github.com> Date: Fri, 13 Oct 2023 17:31:40 +0300 Subject: [PATCH 063/122] Solving clippy errors --- datafusion/physical-expr/src/equivalence.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 29b8048bf277..9bd6c0574ec2 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -1408,7 +1408,7 @@ mod tests { let col_f_expr = &col("f", &test_schema)?; let col_g_expr = &col("g", &test_schema)?; let mut schema_properties = SchemaProperties::new(test_schema.clone()); - schema_properties.add_equal_conditions((&col_a_expr, &col_c_expr)); + schema_properties.add_equal_conditions((col_a_expr, col_c_expr)); let option1 = SortOptions { descending: false, @@ -2176,7 +2176,7 @@ mod tests { let sorted_indices = lexsort_to_indices(&sort_columns, None)?; let original_indices = UInt32Array::from_iter_values(0..n_row as u32); - Ok(&sorted_indices == &original_indices) + Ok(sorted_indices == original_indices) } fn get_representative_arr( From 4122f6e0aa6b0014ab9459126ab4778e70cc19b5 Mon Sep 17 00:00:00 2001 From: metesynnada <100111937+metesynnada@users.noreply.github.com> Date: Fri, 13 Oct 2023 17:58:16 +0300 Subject: [PATCH 064/122] prune_lex improvements --- datafusion/physical-expr/src/equivalence.rs | 35 +++++++++++---------- 1 file changed, 19 insertions(+), 16 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 9bd6c0574ec2..bf9daf506637 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -943,43 +943,46 @@ impl SchemaProperties { /// This util shouldn't e used outside this context. fn prune_lex_req(&self, sort_req: &[PhysicalSortRequirement]) -> LexOrderingReq { // Make sure to use a standardized version of the requirement - let mut sort_req = self.normalize_sort_requirements(sort_req); + let mut normalized_sort_req = self.normalize_sort_requirements(sort_req); // If empty immediately return - if sort_req.is_empty() { - return sort_req; + if normalized_sort_req.is_empty() { + return normalized_sort_req; } for ordering in self.normalized_oeq_group().iter() { let match_indices = ordering .iter() .map(|elem| { - sort_req.iter().position(|sort_req| { + normalized_sort_req.iter().position(|sort_req| { elem.satisfy_with_schema(sort_req, &self.schema) }) }) .collect::>(); - let mut match_prefix = vec![]; - for elem in &match_indices { - if let Some(elem) = elem { - if let Some(last) = match_prefix.last() { - // Should increase - if elem <= last { + + // Find the largest contiguous increasing sequence starting from the first index + let mut to_remove = Vec::new(); + if let Some(&Some(first)) = match_indices.first() { + to_remove.push(first); + for window in match_indices.windows(2) { + if let (Some(current), Some(next)) = (window[0], window[1]) { + if next > current { + to_remove.push(next); + } else { break; } + } else { + break; } - match_prefix.push(*elem) - } else { - break; } } // can remove entries at the match_prefix indices // Remove with reverse iteration to not invalidate indices - for idx in match_prefix.iter().rev() { - sort_req.remove(*idx); + for idx in to_remove.iter().rev() { + normalized_sort_req.remove(*idx); } } - sort_req + normalized_sort_req } /// Checks whether `leading_requirement` is contained in any of the ordering From 7b103973e8b15547d5bfae64464179ba1e573c9a Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Mon, 16 Oct 2023 09:11:00 +0300 Subject: [PATCH 065/122] Fix failing tests --- datafusion/sqllogictest/test_files/joins.slt | 2 +- datafusion/sqllogictest/test_files/window.slt | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/datafusion/sqllogictest/test_files/joins.slt b/datafusion/sqllogictest/test_files/joins.slt index 82dc8a2d2048..45e2aecc606d 100644 --- a/datafusion/sqllogictest/test_files/joins.slt +++ b/datafusion/sqllogictest/test_files/joins.slt @@ -3396,7 +3396,7 @@ SortPreservingMergeExec: [a@0 ASC] ----------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c], output_ordering=[a@0 ASC, b@1 ASC NULLS LAST, c@2 ASC NULLS LAST], has_header=true ------------------CoalesceBatchesExec: target_batch_size=4096 ---------------------SortPreservingRepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2 +--------------------SortPreservingRepartitionExec: partitioning=Hash([a@0], 2), input_partitions=2, sort_exprs=a@0 ASC,b@1 ASC NULLS LAST ----------------------RepartitionExec: partitioning=RoundRobinBatch(2), input_partitions=1 ------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b], output_ordering=[a@0 ASC, b@1 ASC NULLS LAST], has_header=true diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index ae3fcc890e6a..a297d39b4e7e 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -2339,7 +2339,7 @@ Limit: skip=0, fetch=5 ----------TableScan: aggregate_test_100 projection=[c9] physical_plan GlobalLimitExec: skip=0, fetch=5 ---SortExec: fetch=5, expr=[rn1@1 ASC NULLS LAST,c9@0 ASC NULLS LAST] +--SortExec: TopK(fetch=5), expr=[rn1@1 ASC NULLS LAST,c9@0 ASC NULLS LAST] ----ProjectionExec: expr=[c9@0 as c9, ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@1 as rn1] ------BoundedWindowAggExec: wdw=[ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "ROW_NUMBER() ORDER BY [aggregate_test_100.c9 DESC NULLS FIRST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: UInt64, nullable: false, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(UInt64(NULL)), end_bound: CurrentRow }], mode=[Sorted] --------SortExec: expr=[c9@0 DESC] @@ -3243,7 +3243,7 @@ physical_plan ProjectionExec: expr=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@2 as sum1, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as sum2, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as sum3, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@5 as sum4] --BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow }], mode=[Linear] ----CoalesceBatchesExec: target_batch_size=4096 -------SortPreservingRepartitionExec: partitioning=Hash([d@1], 2), input_partitions=2, sort_exprs=a@0 ASC NULLS LAST,b ASC NULLS LAST,c ASC NULLS LAST +------SortPreservingRepartitionExec: partitioning=Hash([d@1], 2), input_partitions=2, sort_exprs=a@0 ASC NULLS LAST --------ProjectionExec: expr=[a@0 as a, d@3 as d, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.b] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@5 as SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.a, annotated_data_infinite2.d] ORDER BY [annotated_data_infinite2.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW, SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@6 as SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] ----------BoundedWindowAggExec: wdw=[SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(annotated_data_infinite2.a) PARTITION BY [annotated_data_infinite2.b, annotated_data_infinite2.a] ORDER BY [annotated_data_infinite2.c ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow }], mode=[Sorted] ------------CoalesceBatchesExec: target_batch_size=4096 From f34c724e7be61595c086af072dc2b00d82f35f5b Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Mon, 16 Oct 2023 14:07:54 +0300 Subject: [PATCH 066/122] Update get_finer and get_meet --- datafusion/physical-expr/src/equivalence.rs | 220 +++++++++++++----- .../physical-plan/src/aggregates/mod.rs | 16 +- datafusion/physical-plan/src/union.rs | 1 - 3 files changed, 176 insertions(+), 61 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index bf9daf506637..4fd83dad6274 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -856,82 +856,81 @@ impl SchemaProperties { .all(|(req, given)| given.compatible(&req)) } - /// Find the finer requirement among `req1` and `req2` + /// Find the finer ordering among `req1` and `req2` + /// Finer requirement is the ordering that satisfies both of the orderings in the arguments. /// If `None`, this means that `req1` and `req2` are not compatible /// e.g there is no requirement that satisfies both - pub fn get_finer_ordering<'a>( - &self, - req1: &'a [PhysicalSortExpr], - req2: &'a [PhysicalSortExpr], - ) -> Option<&'a [PhysicalSortExpr]> { - let lhs = self.normalize_sort_exprs(req1); - let rhs = self.normalize_sort_exprs(req2); - if izip!(lhs.iter(), rhs.iter()).all(|(lhs, rhs)| lhs.eq(rhs)) { - if lhs.len() > rhs.len() { - return Some(req1); - } else { - return Some(req2); - } - } - // Neither `provided` nor `req` satisfies one another, they are incompatible. - None - } - - /// Find the coarser requirement among `req1` and `req2` - /// If `None`, this means that `req1` and `req2` are not compatible - pub fn get_meet_ordering<'a>( + /// As an example finer ordering of [a ASC] and [a ASC, b ASC] is [a ASC, b ASC] + pub fn get_finer_ordering( &self, - req1: &'a [PhysicalSortExpr], - req2: &'a [PhysicalSortExpr], - ) -> Option<&'a [PhysicalSortExpr]> { - let lhs = self.normalize_sort_exprs(req1); - let rhs = self.normalize_sort_exprs(req2); - if izip!(lhs.iter(), rhs.iter()).all(|(lhs, rhs)| lhs.eq(rhs)) { - if lhs.len() < rhs.len() { - return Some(req1); - } else { - return Some(req2); - } - } - // Neither `provided` nor `req` satisfies one another, they are incompatible. - None + req1: &[PhysicalSortExpr], + req2: &[PhysicalSortExpr], + ) -> Option> { + // Convert `PhysicalSortExpr` s to `PhysicalSortRequirement`s. + let req1 = PhysicalSortRequirement::from_sort_exprs(req1); + let req2 = PhysicalSortRequirement::from_sort_exprs(req2); + let finer = self.get_finer_requirement(&req1, &req2); + // Convert back `PhysicalSortRequirement`s to `PhysicalSortExpr`s. + finer.map(PhysicalSortRequirement::to_sort_exprs) } /// Find the finer requirement among `req1` and `req2` /// If `None`, this means that `req1` and `req2` are not compatible /// e.g there is no requirement that satisfies both - pub fn get_finer_requirement<'a>( + pub fn get_finer_requirement( &self, - req1: &'a [PhysicalSortRequirement], - req2: &'a [PhysicalSortRequirement], - ) -> Option<&'a [PhysicalSortRequirement]> { - let lhs = self.normalize_sort_requirements(req1); - let rhs = self.normalize_sort_requirements(req2); - let mut left_finer = false; - let mut right_finer = false; - if izip!(lhs.iter(), rhs.iter()).all(|(lhs, rhs)| { + req1: &[PhysicalSortRequirement], + req2: &[PhysicalSortRequirement], + ) -> Option> { + let mut lhs = self.normalize_sort_requirements(req1); + let mut rhs = self.normalize_sort_requirements(req2); + if izip!(lhs.iter_mut(), rhs.iter_mut()).all(|(lhs, rhs)| { match (lhs.options, rhs.options) { (Some(lhs_opt), Some(rhs_opt)) => { lhs.expr.eq(&rhs.expr) && lhs_opt == rhs_opt } - (Some(_), None) => { - left_finer = true; + (Some(options), None) => { + rhs.options = Some(options); lhs.expr.eq(&rhs.expr) } - (None, Some(_)) => { - right_finer = true; + (None, Some(options)) => { + lhs.options = Some(options); lhs.expr.eq(&rhs.expr) } (None, None) => lhs.expr.eq(&rhs.expr), } }) { - if lhs.len() >= rhs.len() && !right_finer { - return Some(req1); - } else if rhs.len() >= lhs.len() && !left_finer { - return Some(req2); + if lhs.len() >= rhs.len() { + return Some(lhs); + } else if rhs.len() >= lhs.len() { + return Some(rhs); } } - // Neither `provided` nor `req` satisfies one another, they are incompatible. + // Neither `req1` nor `req2` satisfies one another, they are incompatible. + None + } + + /// Calculates the "meet" of given orderings. + /// The meet is the finest ordering that satisfied by all the given + /// orderings, see . + /// If `None`, this means that `req1` and `req2` are not compatible + /// e.g there is no ordering that is satisfied by both + /// As an example meet ordering of [a ASC] and [a ASC, b ASC] is [a ASC] + pub fn get_meet_ordering( + &self, + req1: &[PhysicalSortExpr], + req2: &[PhysicalSortExpr], + ) -> Option> { + let lhs = self.normalize_sort_exprs(req1); + let rhs = self.normalize_sort_exprs(req2); + if izip!(lhs.iter(), rhs.iter()).all(|(lhs, rhs)| lhs.eq(rhs)) { + if lhs.len() < rhs.len() { + return Some(lhs); + } else { + return Some(rhs); + } + } + // There is no meet None } @@ -1517,7 +1516,7 @@ mod tests { } // Convert each tuple to PhysicalSortRequirement - fn convert_to_requirement( + fn convert_to_sort_reqs( in_data: &[(&Arc, Option)], ) -> Vec { in_data @@ -1528,6 +1527,19 @@ mod tests { .collect::>() } + // Convert each tuple to PhysicalSortExpr + fn convert_to_sort_exprs( + in_data: &[(&Arc, SortOptions)], + ) -> Vec { + in_data + .iter() + .map(|(expr, options)| PhysicalSortExpr { + expr: (*expr).clone(), + options: *options, + }) + .collect::>() + } + #[test] fn add_equal_conditions_test() -> Result<()> { let schema = Arc::new(Schema::new(vec![ @@ -2426,8 +2438,8 @@ mod tests { ]; for (reqs, expected_normalized) in requirements.into_iter() { - let req = convert_to_requirement(&reqs); - let expected_normalized = convert_to_requirement(&expected_normalized); + let req = convert_to_sort_reqs(&reqs); + let expected_normalized = convert_to_sort_reqs(&expected_normalized); assert_eq!( schema_properties.normalize_sort_requirements(&req), @@ -2436,4 +2448,98 @@ mod tests { } Ok(()) } + + #[test] + fn test_get_finer() -> Result<()> { + let schema = create_test_schema()?; + let col_a = &col("a", &schema)?; + let col_b = &col("b", &schema)?; + let col_c = &col("c", &schema)?; + let schema_properties = SchemaProperties::new(schema); + let option_asc = SortOptions { + descending: false, + nulls_first: false, + }; + let option_desc = SortOptions { + descending: true, + nulls_first: true, + }; + let tests_cases = vec![ + // Get finer requirement between [a Some(ASC)] and [a None, b Some(ASC)] + // result should be [a Some(ASC), b Some(ASC)] + ( + vec![(col_a, Some(option_asc))], + vec![(col_a, None), (col_b, Some(option_asc))], + Some(vec![(col_a, Some(option_asc)), (col_b, Some(option_asc))]), + ), + // Get finer requirement between [a Some(ASC), b Some(ASC), c Some(ASC)] and [a Some(ASC), b Some(ASC)] + // result should be [a Some(ASC), b Some(ASC), c Some(ASC)] + ( + vec![ + (col_a, Some(option_asc)), + (col_b, Some(option_asc)), + (col_c, Some(option_asc)), + ], + vec![(col_a, Some(option_asc)), (col_b, Some(option_asc))], + Some(vec![ + (col_a, Some(option_asc)), + (col_b, Some(option_asc)), + (col_c, Some(option_asc)), + ]), + ), + // Get finer requirement between [a Some(ASC), b Some(ASC)] and [a Some(ASC), b Some(DESC)] + // result should be None + ( + vec![(col_a, Some(option_asc)), (col_b, Some(option_asc))], + vec![(col_a, Some(option_asc)), (col_b, Some(option_desc))], + None, + ), + ]; + for (lhs, rhs, expected) in tests_cases { + let lhs = convert_to_sort_reqs(&lhs); + let rhs = convert_to_sort_reqs(&rhs); + let expected = expected.map(|expected| convert_to_sort_reqs(&expected)); + let finer = schema_properties.get_finer_requirement(&lhs, &rhs); + assert_eq!(finer, expected) + } + + Ok(()) + } + + #[test] + fn test_get_meet_ordering() -> Result<()> { + let schema = create_test_schema()?; + let col_a = &col("a", &schema)?; + let col_b = &col("b", &schema)?; + let schema_properties = SchemaProperties::new(schema); + let option_asc = SortOptions { + descending: false, + nulls_first: false, + }; + let option_desc = SortOptions { + descending: true, + nulls_first: true, + }; + let tests_cases = vec![ + // Get meet ordering between [a ASC] and [a ASC, b ASC] + // result should be [a ASC] + ( + vec![(col_a, option_asc)], + vec![(col_a, option_asc), (col_b, option_asc)], + Some(vec![(col_a, option_asc)]), + ), + // Get meet ordering between [a ASC] and [a DESC] + // result should be None. + (vec![(col_a, option_asc)], vec![(col_a, option_desc)], None), + ]; + for (lhs, rhs, expected) in tests_cases { + let lhs = convert_to_sort_exprs(&lhs); + let rhs = convert_to_sort_exprs(&rhs); + let expected = expected.map(|expected| convert_to_sort_exprs(&expected)); + let finer = schema_properties.get_meet_ordering(&lhs, &rhs); + assert_eq!(finer, expected) + } + + Ok(()) + } } diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index b813ee7c4f79..243642a4bfa9 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -371,7 +371,7 @@ fn get_finest_requirement( if let Some(finest_req) = &mut finest_req { if let Some(finer) = schema_properties.get_finer_ordering(finest_req, fn_req) { - *finest_req = finer.to_vec(); + *finest_req = finer; continue; } // If an aggregate function is reversible, analyze whether its reverse @@ -384,7 +384,7 @@ fn get_finest_requirement( // We need to update `aggr_expr` with its reverse, since only its // reverse requirement is compatible with existing requirements: *aggr_expr = reverse; - *finest_req = finer.to_vec(); + *finest_req = finer; *fn_req = fn_req_reverse; continue; } @@ -2059,6 +2059,16 @@ mod tests { options: options2, }]), ]; + let common_requirement = Some(vec![ + PhysicalSortExpr { + expr: col_a.clone(), + options: options1, + }, + PhysicalSortExpr { + expr: col_c.clone(), + options: options1, + }, + ]); let aggr_expr = Arc::new(FirstValue::new( col_a.clone(), "first1", @@ -2072,7 +2082,7 @@ mod tests { &mut order_by_exprs, &schema_properties, )?; - assert_eq!(res, order_by_exprs[2]); + assert_eq!(res, common_requirement); Ok(()) } } diff --git a/datafusion/physical-plan/src/union.rs b/datafusion/physical-plan/src/union.rs index 81bf8d5b6656..1ba72e9d52a9 100644 --- a/datafusion/physical-plan/src/union.rs +++ b/datafusion/physical-plan/src/union.rs @@ -238,7 +238,6 @@ impl ExecutionPlan for UnionExec { let mut meet = Some(elem.clone()); child_oeqs.iter().for_each(|child_oeq| { if let Some(meet_vec) = &meet { - // let mut meet_found = false; let res = child_oeq .oeq_group() .iter() From 745e7ccd2cb252a8e48fd599a6444aaa945970bf Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Mon, 16 Oct 2023 15:16:27 +0300 Subject: [PATCH 067/122] Fix window lex ordering implementation --- datafusion/physical-expr/src/equivalence.rs | 40 ++++++++++++------- datafusion/physical-expr/src/utils.rs | 36 ++++++++++++----- .../physical-expr/src/window/built_in.rs | 9 +---- .../src/windows/bounded_window_agg_exec.rs | 13 +++--- datafusion/physical-plan/src/windows/mod.rs | 24 ++++++++++- .../src/windows/window_agg_exec.rs | 14 ++++--- datafusion/sqllogictest/test_files/window.slt | 37 +++++++++++++++++ 7 files changed, 130 insertions(+), 43 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 4fd83dad6274..9a5e8ead93bc 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -1097,24 +1097,36 @@ impl SchemaProperties { pub fn get_lex_ordering( &self, exprs: &[Arc], - ) -> Option> { - let normalized_exprs = self.eq_groups.normalize_exprs(exprs); + ) -> Option> { + let mut normalized_exprs = self.eq_groups.normalize_exprs(exprs); + let mut ordered_exprs: Vec = vec![]; for ordering in self.normalized_oeq_group().iter() { - if normalized_exprs.len() <= ordering.len() { - let mut ordering_options = vec![]; - for (expr, sort_expr) in izip!(normalized_exprs.iter(), ordering.iter()) { - if sort_expr.expr.eq(expr) { - ordering_options.push(sort_expr.options); - } else { - break; - } - if ordering_options.len() == normalized_exprs.len() { - return Some(ordering_options); - } + for sort_expr in ordering { + if let Some(idx) = normalized_exprs.iter().position(|normalized_expr| { + sort_expr.satisfy_with_schema( + &PhysicalSortRequirement { + expr: normalized_expr.clone(), + options: None, + }, + &self.schema, + ) + }) { + ordered_exprs.push(PhysicalSortExpr { + expr: normalized_exprs[idx].clone(), + options: sort_expr.options, + }); + normalized_exprs.remove(idx); + } else { + // Should find in consecutive chunks + break; } } } - None + if normalized_exprs.is_empty() { + Some(ordered_exprs) + } else { + None + } } } diff --git a/datafusion/physical-expr/src/utils.rs b/datafusion/physical-expr/src/utils.rs index 1d1b4d95cb9f..656dac388c43 100644 --- a/datafusion/physical-expr/src/utils.rs +++ b/datafusion/physical-expr/src/utils.rs @@ -758,14 +758,13 @@ mod tests { let sort_options = SortOptions::default(); let sort_options_not = SortOptions::default().not(); - let required_columns = [ - Arc::new(Column::new("b", 1)) as _, - Arc::new(Column::new("a", 0)) as _, - ]; let schema = Schema::new(vec![ Field::new("a", DataType::Int32, true), Field::new("b", DataType::Int32, true), ]); + let col_a = &col("a", &schema)?; + let col_b = &col("b", &schema)?; + let required_columns = [col_b.clone(), col_a.clone()]; let mut ordering_equal_properties = SchemaProperties::new(Arc::new(schema)); ordering_equal_properties.add_new_orderings(&[vec![ PhysicalSortExpr { @@ -784,18 +783,26 @@ mod tests { assert_eq!( ordering_equal_properties.get_lex_ordering(&required_columns), - Some(vec![sort_options_not, sort_options]) + Some(vec![ + PhysicalSortExpr { + expr: col_b.clone(), + options: sort_options_not + }, + PhysicalSortExpr { + expr: col_a.clone(), + options: sort_options + } + ]) ); - let required_columns = [ - Arc::new(Column::new("b", 1)) as _, - Arc::new(Column::new("a", 0)) as _, - ]; let schema = Schema::new(vec![ Field::new("a", DataType::Int32, true), Field::new("b", DataType::Int32, true), Field::new("c", DataType::Int32, true), ]); + let col_a = &col("a", &schema)?; + let col_b = &col("b", &schema)?; + let required_columns = [col_b.clone(), col_a.clone()]; let mut ordering_equal_properties = SchemaProperties::new(Arc::new(schema)); ordering_equal_properties.add_new_orderings(&[ vec![PhysicalSortExpr { @@ -820,7 +827,16 @@ mod tests { assert_eq!( ordering_equal_properties.get_lex_ordering(&required_columns), - Some(vec![sort_options_not, sort_options]) + Some(vec![ + PhysicalSortExpr { + expr: col_b.clone(), + options: sort_options_not + }, + PhysicalSortExpr { + expr: col_a.clone(), + options: sort_options + } + ]) ); let required_columns = [ diff --git a/datafusion/physical-expr/src/window/built_in.rs b/datafusion/physical-expr/src/window/built_in.rs index dab6f6c50290..95066a6af382 100644 --- a/datafusion/physical-expr/src/window/built_in.rs +++ b/datafusion/physical-expr/src/window/built_in.rs @@ -36,7 +36,6 @@ use datafusion_common::{Result, ScalarValue}; use datafusion_expr::window_state::WindowAggState; use datafusion_expr::window_state::WindowFrameContext; use datafusion_expr::WindowFrame; -use itertools::izip; /// A window expr that takes the form of a [`BuiltInWindowFunctionExpr`]. #[derive(Debug)] @@ -94,15 +93,9 @@ impl BuiltInWindowExpr { .into_iter() .map(|idx| self.partition_by[idx].clone()) .collect::>(); - if let Some(ordering_options) = + if let Some(mut ordering) = schema_properties.get_lex_ordering(&lex_partition_by) { - let mut ordering = izip!( - lex_partition_by.into_iter(), - ordering_options.into_iter() - ) - .map(|(expr, options)| PhysicalSortExpr { expr, options }) - .collect::>(); ordering.push(fn_res_ordering); schema_properties.add_new_orderings(&[ordering]); } diff --git a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs index a167a50d3a98..f6dd0f7844d9 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -30,8 +30,8 @@ use std::task::{Context, Poll}; use crate::expressions::PhysicalSortExpr; use crate::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; use crate::windows::{ - calc_requirements, get_ordered_partition_by_indices, window_ordering_equivalence, - PartitionSearchMode, + calc_requirements, get_ordered_partition_by_indices, get_partition_by_sort_exprs, + window_ordering_equivalence, PartitionSearchMode, }; use crate::{ ColumnStatistics, DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, @@ -147,9 +147,12 @@ impl BoundedWindowAggExec { // Hence returned `PhysicalSortExpr` corresponding to `PARTITION BY` columns can be used safely // to calculate partition separation points pub fn partition_by_sort_keys(&self) -> Result> { - // Partition by sort keys indices are stored in self.ordered_partition_by_indices. - let sort_keys = self.input.output_ordering().unwrap_or(&[]); - get_at_indices(sort_keys, &self.ordered_partition_by_indices) + let partition_by = self.window_expr()[0].partition_by(); + get_partition_by_sort_exprs( + &self.input, + partition_by, + &self.ordered_partition_by_indices, + ) } /// Initializes the appropriate [`PartitionSearcher`] implementation from diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index 4f205e96b3d1..bc6950338c86 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -40,7 +40,7 @@ use datafusion_expr::{ use datafusion_physical_expr::{ reverse_order_bys, window::{BuiltInWindowFunctionExpr, SlidingAggregateWindowExpr}, - AggregateExpr, PhysicalSortRequirement, SchemaProperties, + AggregateExpr, LexOrdering, PhysicalSortRequirement, SchemaProperties, }; mod bounded_window_agg_exec; @@ -332,6 +332,28 @@ pub(crate) fn get_ordered_partition_by_indices( } } +pub(crate) fn get_partition_by_sort_exprs( + input: &Arc, + partition_by_exprs: &[Arc], + ordered_partition_by_indices: &[usize], +) -> Result { + let ordered_partition_exprs = ordered_partition_by_indices + .iter() + .map(|idx| partition_by_exprs[*idx].clone()) + .collect::>(); + // Make sure ordered section doesn't move over the partition by expression + assert!(ordered_partition_by_indices.len() <= partition_by_exprs.len()); + let partition_by_sort_exprs = input + .schema_properties() + .get_lex_ordering(&ordered_partition_exprs) + .ok_or_else(|| { + DataFusionError::Execution( + "Expects partition by expression to be ordered".to_string(), + ) + })?; + Ok(partition_by_sort_exprs) +} + pub(crate) fn window_ordering_equivalence( schema: &SchemaRef, input: &Arc, diff --git a/datafusion/physical-plan/src/windows/window_agg_exec.rs b/datafusion/physical-plan/src/windows/window_agg_exec.rs index a85ba9f1bb19..ee7e39a0995c 100644 --- a/datafusion/physical-plan/src/windows/window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/window_agg_exec.rs @@ -26,7 +26,8 @@ use crate::common::transpose; use crate::expressions::PhysicalSortExpr; use crate::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; use crate::windows::{ - calc_requirements, get_ordered_partition_by_indices, window_ordering_equivalence, + calc_requirements, get_ordered_partition_by_indices, get_partition_by_sort_exprs, + window_ordering_equivalence, }; use crate::{ ColumnStatistics, DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, @@ -42,7 +43,7 @@ use arrow::{ datatypes::{Schema, SchemaRef}, record_batch::RecordBatch, }; -use datafusion_common::utils::{evaluate_partition_ranges, get_at_indices}; +use datafusion_common::utils::evaluate_partition_ranges; use datafusion_common::{internal_err, plan_err, DataFusionError, Result}; use datafusion_execution::TaskContext; use datafusion_physical_expr::{PhysicalSortRequirement, SchemaProperties}; @@ -106,9 +107,12 @@ impl WindowAggExec { // Hence returned `PhysicalSortExpr` corresponding to `PARTITION BY` columns can be used safely // to calculate partition separation points pub fn partition_by_sort_keys(&self) -> Result> { - // Partition by sort keys indices are stored in self.ordered_partition_by_indices. - let sort_keys = self.input.output_ordering().unwrap_or(&[]); - get_at_indices(sort_keys, &self.ordered_partition_by_indices) + let partition_by = self.window_expr()[0].partition_by(); + get_partition_by_sort_exprs( + &self.input, + partition_by, + &self.ordered_partition_by_indices, + ) } } diff --git a/datafusion/sqllogictest/test_files/window.slt b/datafusion/sqllogictest/test_files/window.slt index a297d39b4e7e..1872fa206c20 100644 --- a/datafusion/sqllogictest/test_files/window.slt +++ b/datafusion/sqllogictest/test_files/window.slt @@ -3434,3 +3434,40 @@ ProjectionExec: expr=[MAX(multiple_ordered_table.c) PARTITION BY [multiple_order ----CoalesceBatchesExec: target_batch_size=4096 ------FilterExec: d@1 = 0 --------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[c, d], output_ordering=[c@0 ASC NULLS LAST], has_header=true + +query TT +explain SELECT SUM(d) OVER(PARTITION BY c ORDER BY a ASC) +FROM multiple_ordered_table; +---- +logical_plan +Projection: SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c] ORDER BY [multiple_ordered_table.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW +--WindowAggr: windowExpr=[[SUM(CAST(multiple_ordered_table.d AS Int64)) PARTITION BY [multiple_ordered_table.c] ORDER BY [multiple_ordered_table.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +----TableScan: multiple_ordered_table projection=[a, c, d] +physical_plan +ProjectionExec: expr=[SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c] ORDER BY [multiple_ordered_table.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@3 as SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c] ORDER BY [multiple_ordered_table.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] +--BoundedWindowAggExec: wdw=[SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c] ORDER BY [multiple_ordered_table.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c] ORDER BY [multiple_ordered_table.a ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow }], mode=[Sorted] +----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true + +query TT +explain SELECT SUM(d) OVER(PARTITION BY c, a ORDER BY b ASC) +FROM multiple_ordered_table; +---- +logical_plan +Projection: SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW +--WindowAggr: windowExpr=[[SUM(CAST(multiple_ordered_table.d AS Int64)) PARTITION BY [multiple_ordered_table.c, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW]] +----TableScan: multiple_ordered_table projection=[a, b, c, d] +physical_plan +ProjectionExec: expr=[SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW@4 as SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW] +--BoundedWindowAggExec: wdw=[SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW: Ok(Field { name: "SUM(multiple_ordered_table.d) PARTITION BY [multiple_ordered_table.c, multiple_ordered_table.a] ORDER BY [multiple_ordered_table.b ASC NULLS LAST] RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW", data_type: Int64, nullable: true, dict_id: 0, dict_is_ordered: false, metadata: {} }), frame: WindowFrame { units: Range, start_bound: Preceding(Int32(NULL)), end_bound: CurrentRow }], mode=[Sorted] +----CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, b, c, d], output_ordering=[a@0 ASC NULLS LAST, b@1 ASC NULLS LAST], has_header=true + +query I +SELECT SUM(d) OVER(PARTITION BY c, a ORDER BY b ASC) +FROM multiple_ordered_table +LIMIT 5; +---- +0 +2 +0 +0 +1 From e007b98738f9937f727d15c172cb43b7cb347458 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Mon, 16 Oct 2023 18:19:17 +0300 Subject: [PATCH 068/122] Buggy state --- datafusion/physical-expr/src/equivalence.rs | 75 +++++++++++++-------- 1 file changed, 46 insertions(+), 29 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 9a5e8ead93bc..ebaea71efdb9 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -33,6 +33,7 @@ use datafusion_common::{DataFusionError, JoinSide, JoinType, Result}; use itertools::izip; use std::hash::Hash; use std::sync::Arc; +use arrow::compute::or; /// EquivalentClass is a set of [`Arc`]s that are known /// to have the same value in all tuples in a relation. These are generated by equality predicates, @@ -1041,37 +1042,14 @@ impl SchemaProperties { /// Check whether any permutation of the argument has a prefix with existing ordering. /// Return indices that describes ordering and their ordering information. pub fn set_satisfy(&self, exprs: &[Arc]) -> Option> { - let exprs_normalized = self.eq_groups.normalize_exprs(exprs); - let mut best = vec![]; - - for ordering in self.normalized_oeq_group().iter() { - let ordering_exprs = ordering - .iter() - .map(|sort_expr| sort_expr.expr.clone()) - .collect::>(); - let mut ordered_indices = - get_indices_of_exprs_strict(&exprs_normalized, &ordering_exprs); - ordered_indices.sort(); - // Find out how many expressions of the existing ordering define ordering - // for expressions in the GROUP BY clause. For example, if the input is - // ordered by a, b, c, d and we group by b, a, d; the result below would be. - // 2, meaning 2 elements (a, b) among the GROUP BY columns define ordering. - let first_n = longest_consecutive_prefix(ordered_indices); - if first_n > best.len() { - let ordered_exprs = ordering_exprs[0..first_n].to_vec(); - // Find indices for the GROUP BY expressions such that when we iterate with - // these indices, we would match existing ordering. For the example above, - // this would produce 1, 0; meaning 1st and 0th entries (a, b) among the - // GROUP BY expressions b, a, d match input ordering. - best = get_indices_of_exprs_strict(&ordered_exprs, &exprs_normalized); + let normalized_exprs = self.eq_groups.normalize_exprs(exprs); + if let Some(ordered_section) = self.get_lex_ordering_section(&normalized_exprs){ + let mut indices = vec![]; + for sort_expr in ordered_section{ + } } - - if best.is_empty() { - None - } else { - Some(best) - } + None } /// Check whether one of the permutation of the exprs satisfies existing ordering. @@ -1128,6 +1106,45 @@ impl SchemaProperties { None } } + + /// Get ordering of the expressions in the argument + /// Assumes arguments define lexicographical ordering. + /// None, represents none of the existing ordering satisfy + /// lexicographical ordering of the exprs. + pub fn get_lex_ordering_section( + &self, + exprs: &[Arc], + ) -> Option> { + let mut normalized_exprs = self.eq_groups.normalize_exprs(exprs); + let mut ordered_exprs: Vec = vec![]; + for ordering in self.normalized_oeq_group().iter() { + for sort_expr in ordering { + if let Some(idx) = normalized_exprs.iter().position(|normalized_expr| { + sort_expr.satisfy_with_schema( + &PhysicalSortRequirement { + expr: normalized_expr.clone(), + options: None, + }, + &self.schema, + ) + }) { + ordered_exprs.push(PhysicalSortExpr { + expr: normalized_exprs[idx].clone(), + options: sort_expr.options, + }); + normalized_exprs.remove(idx); + } else { + // Should find in consecutive chunks + break; + } + } + } + if !ordered_exprs.is_empty() { + Some(ordered_exprs) + } else { + None + } + } } /// Calculate ordering equivalence properties for the given join operation. From acc6c15f876bd1fc36289a97230d5472375eacfc Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 17 Oct 2023 09:36:39 +0300 Subject: [PATCH 069/122] Do not use output ordering in the aggregate --- datafusion/physical-expr/src/equivalence.rs | 19 ++++++++++++------- .../physical-plan/src/aggregates/row_hash.rs | 10 ++++++---- 2 files changed, 18 insertions(+), 11 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index ebaea71efdb9..8a52af8bb729 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -16,7 +16,6 @@ // under the License. use crate::expressions::Column; -use crate::utils::get_indices_of_exprs_strict; use crate::{ physical_exprs_contains, LexOrdering, LexOrderingRef, LexOrderingReq, PhysicalExpr, PhysicalSortExpr, PhysicalSortRequirement, @@ -28,12 +27,10 @@ use crate::physical_expr::{deduplicate_physical_exprs, have_common_entries}; use crate::sort_properties::{ExprOrdering, SortProperties}; use arrow_schema::SortOptions; use datafusion_common::tree_node::{Transformed, TreeNode}; -use datafusion_common::utils::longest_consecutive_prefix; use datafusion_common::{DataFusionError, JoinSide, JoinType, Result}; use itertools::izip; use std::hash::Hash; use std::sync::Arc; -use arrow::compute::or; /// EquivalentClass is a set of [`Arc`]s that are known /// to have the same value in all tuples in a relation. These are generated by equality predicates, @@ -1043,13 +1040,21 @@ impl SchemaProperties { /// Return indices that describes ordering and their ordering information. pub fn set_satisfy(&self, exprs: &[Arc]) -> Option> { let normalized_exprs = self.eq_groups.normalize_exprs(exprs); - if let Some(ordered_section) = self.get_lex_ordering_section(&normalized_exprs){ + if let Some(ordered_section) = self.get_lex_ordering_section(&normalized_exprs) { let mut indices = vec![]; - for sort_expr in ordered_section{ - + for sort_expr in ordered_section { + if let Some(position) = + exprs.iter().position(|expr| sort_expr.expr.eq(expr)) + { + indices.push(position); + } else { + panic!("expects to find all of the ordered section inside normalized_exprs"); + } } + Some(indices) + } else { + None } - None } /// Check whether one of the permutation of the exprs satisfies existing ordering. diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index 9b5f83e3bbde..9739d95f7658 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -39,7 +39,7 @@ use crate::metrics::{BaselineMetrics, RecordOutput}; use crate::sorts::sort::{read_spill_as_stream, sort_batch}; use crate::sorts::streaming_merge; use crate::stream::RecordBatchStreamAdapter; -use crate::{aggregates, PhysicalExpr}; +use crate::{aggregates, ExecutionPlan, PhysicalExpr}; use crate::{RecordBatchStream, SendableRecordBatchStream}; use arrow::array::*; use arrow::{datatypes::SchemaRef, record_batch::RecordBatch}; @@ -329,12 +329,14 @@ impl GroupedHashAggregateStream { let name = format!("GroupedHashAggregateStream[{partition}]"); let reservation = MemoryConsumer::new(name).register(context.memory_pool()); - - let out_ordering = agg.output_ordering.as_deref().unwrap_or(&[]); + let ordered_section = agg + .schema_properties() + .get_lex_ordering_section(&agg_group_by.output_exprs()) + .unwrap_or_default(); let group_ordering = GroupOrdering::try_new( &group_schema, &agg.partition_search_mode, - out_ordering, + &ordered_section, )?; let group_values = new_group_values(group_schema)?; From 8692dce86f2cc288541c2c1333e7d178e7b971a6 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 17 Oct 2023 10:04:58 +0300 Subject: [PATCH 070/122] Add union test --- datafusion/physical-expr/src/equivalence.rs | 19 ++-- datafusion/physical-plan/src/union.rs | 96 +++++++++++++++++++++ 2 files changed, 109 insertions(+), 6 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 8a52af8bb729..5e0b80be365d 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -921,15 +921,15 @@ impl SchemaProperties { ) -> Option> { let lhs = self.normalize_sort_exprs(req1); let rhs = self.normalize_sort_exprs(req2); - if izip!(lhs.iter(), rhs.iter()).all(|(lhs, rhs)| lhs.eq(rhs)) { - if lhs.len() < rhs.len() { - return Some(lhs); + let mut meet = vec![]; + for (lhs, rhs) in izip!(lhs.iter(), rhs.iter()) { + if lhs.eq(rhs) { + meet.push(lhs.clone()); } else { - return Some(rhs); + break; } } - // There is no meet - None + (!meet.is_empty()).then_some(meet) } /// This function prunes lexicographical ordering requirement @@ -2565,6 +2565,13 @@ mod tests { // Get meet ordering between [a ASC] and [a DESC] // result should be None. (vec![(col_a, option_asc)], vec![(col_a, option_desc)], None), + // Get meet ordering between [a ASC, b ASC] and [a ASC, b DESC] + // result should be [a ASC]. + ( + vec![(col_a, option_asc), (col_b, option_asc)], + vec![(col_a, option_asc), (col_b, option_desc)], + Some(vec![(col_a, option_asc)]), + ), ]; for (lhs, rhs, expected) in tests_cases { let lhs = convert_to_sort_exprs(&lhs); diff --git a/datafusion/physical-plan/src/union.rs b/datafusion/physical-plan/src/union.rs index 1ba72e9d52a9..1818a83af2b4 100644 --- a/datafusion/physical-plan/src/union.rs +++ b/datafusion/physical-plan/src/union.rs @@ -644,8 +644,25 @@ mod tests { use crate::test; use crate::collect; + use crate::memory::MemoryExec; use arrow::record_batch::RecordBatch; + use arrow_schema::{DataType, SortOptions}; use datafusion_common::ScalarValue; + use datafusion_physical_expr::expressions::col; + + // Generate a schema which consists of 7 columns (a, b, c, d, e, f, g) + fn create_test_schema() -> Result { + let a = Field::new("a", DataType::Int32, true); + let b = Field::new("b", DataType::Int32, true); + let c = Field::new("c", DataType::Int32, true); + let d = Field::new("d", DataType::Int32, true); + let e = Field::new("e", DataType::Int32, true); + let f = Field::new("f", DataType::Int32, true); + let g = Field::new("g", DataType::Int32, true); + let schema = Arc::new(Schema::new(vec![a, b, c, d, e, f, g])); + + Ok(schema) + } #[tokio::test] async fn test_union_partitions() -> Result<()> { @@ -749,4 +766,83 @@ mod tests { assert_eq!(result, expected); } + + #[tokio::test] + async fn test_union_schema_properties() -> Result<()> { + let schema = create_test_schema()?; + let col_a = &col("a", &schema)?; + let col_b = &col("b", &schema)?; + let col_c = &col("c", &schema)?; + let col_d = &col("d", &schema)?; + let col_e = &col("e", &schema)?; + let col_f = &col("f", &schema)?; + let options = SortOptions::default(); + // [a ASC, b ASC, f ASC], [d ASC] + let orderings = vec![ + vec![ + PhysicalSortExpr { + expr: col_a.clone(), + options, + }, + PhysicalSortExpr { + expr: col_b.clone(), + options, + }, + PhysicalSortExpr { + expr: col_f.clone(), + options, + }, + ], + vec![PhysicalSortExpr { + expr: col_d.clone(), + options, + }], + ]; + let child1 = Arc::new( + MemoryExec::try_new(&[], schema.clone(), None)? + .with_sort_information(orderings), + ); + + // [a ASC, b ASC, c ASC], [e ASC] + let orderings = vec![ + vec![ + PhysicalSortExpr { + expr: col_a.clone(), + options, + }, + PhysicalSortExpr { + expr: col_b.clone(), + options, + }, + PhysicalSortExpr { + expr: col_c.clone(), + options, + }, + ], + vec![PhysicalSortExpr { + expr: col_e.clone(), + options, + }], + ]; + let child2 = Arc::new( + MemoryExec::try_new(&[], schema, None)?.with_sort_information(orderings), + ); + + let union = UnionExec::new(vec![child1, child2]); + // Expects union to have [a ASC, b ASC] (e.g meet of inout orderings) + let union_schema_properties = union.schema_properties(); + let union_orderings = union_schema_properties.oeq_group(); + println!("union_orderings:{:?}", union_orderings); + assert!(union_orderings.contains(&vec![ + PhysicalSortExpr { + expr: col_a.clone(), + options + }, + PhysicalSortExpr { + expr: col_b.clone(), + options + } + ])); + Ok(()) + } } From 1d539003b9150393892bd6782c01f54497fee985 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 17 Oct 2023 10:06:35 +0300 Subject: [PATCH 071/122] Update comment --- datafusion/physical-plan/src/union.rs | 2 ++ 1 file changed, 2 insertions(+) diff --git a/datafusion/physical-plan/src/union.rs b/datafusion/physical-plan/src/union.rs index 1818a83af2b4..075e0ac71a6d 100644 --- a/datafusion/physical-plan/src/union.rs +++ b/datafusion/physical-plan/src/union.rs @@ -226,6 +226,8 @@ impl ExecutionPlan for UnionExec { } fn schema_properties(&self) -> SchemaProperties { + // TODO: In some cases equivalent groups and constants + // can be preserved in union. Add support for these. let child_oeqs = self .inputs .iter() From 9630081277fc4a700df64f3addb2778e887037f4 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 18 Oct 2023 10:07:57 +0300 Subject: [PATCH 072/122] Fix bug, when batch_size is small --- .../physical-expr/src/aggregate/first_last.rs | 8 ++++--- .../sqllogictest/test_files/groupby.slt | 24 ++++++++++++++----- 2 files changed, 23 insertions(+), 9 deletions(-) diff --git a/datafusion/physical-expr/src/aggregate/first_last.rs b/datafusion/physical-expr/src/aggregate/first_last.rs index 6ae7b4895ad6..ce7a1daeec64 100644 --- a/datafusion/physical-expr/src/aggregate/first_last.rs +++ b/datafusion/physical-expr/src/aggregate/first_last.rs @@ -22,7 +22,9 @@ use std::sync::Arc; use crate::aggregate::utils::{down_cast_any_ref, ordering_fields}; use crate::expressions::format_state_name; -use crate::{AggregateExpr, LexOrdering, PhysicalExpr, PhysicalSortExpr}; +use crate::{ + reverse_order_bys, AggregateExpr, LexOrdering, PhysicalExpr, PhysicalSortExpr, +}; use arrow::array::ArrayRef; use arrow::compute; @@ -126,7 +128,7 @@ impl AggregateExpr for FirstValue { self.expr.clone(), name, self.input_data_type.clone(), - self.ordering_req.clone(), + reverse_order_bys(&self.ordering_req), self.order_by_data_types.clone(), ))) } @@ -350,7 +352,7 @@ impl AggregateExpr for LastValue { self.expr.clone(), name, self.input_data_type.clone(), - self.ordering_req.clone(), + reverse_order_bys(&self.ordering_req), self.order_by_data_types.clone(), ))) } diff --git a/datafusion/sqllogictest/test_files/groupby.slt b/datafusion/sqllogictest/test_files/groupby.slt index 09c987cc3c90..fddfe5811a21 100644 --- a/datafusion/sqllogictest/test_files/groupby.slt +++ b/datafusion/sqllogictest/test_files/groupby.slt @@ -3634,17 +3634,29 @@ ProjectionExec: expr=[FIRST_VALUE(multiple_ordered_table.a) ORDER BY [multiple_o ----------RepartitionExec: partitioning=RoundRobinBatch(8), input_partitions=1 ------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/core/tests/data/window_2.csv]]}, projection=[a, c, d], output_ordering=[a@0 ASC NULLS LAST], has_header=true +query II rowsort +SELECT FIRST_VALUE(a ORDER BY a ASC) as first_a, + LAST_VALUE(c ORDER BY c DESC) as last_c +FROM multiple_ordered_table +GROUP BY d; +---- +0 0 +0 1 +0 15 +0 4 +0 9 + query III rowsort -SELECT d, FIRST_VALUE(a ORDER BY a ASC) as first_a, +SELECT d, FIRST_VALUE(c ORDER BY a DESC, c DESC) as first_a, LAST_VALUE(c ORDER BY c DESC) as last_c FROM multiple_ordered_table GROUP BY d; ---- -0 0 46 -1 0 58 -2 0 66 -3 0 51 -4 0 64 +0 95 0 +1 90 4 +2 97 1 +3 99 15 +4 98 9 query TT EXPLAIN SELECT c From f165acea5fcf82b639481a781cab89f3591c59e3 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Thu, 19 Oct 2023 09:49:23 +0300 Subject: [PATCH 073/122] Review Part 1 --- datafusion/common/src/join_type.rs | 5 +- .../enforce_distribution.rs | 209 +++++++----------- .../src/physical_optimizer/enforce_sorting.rs | 30 +-- .../replace_with_order_preserving_variants.rs | 27 +-- .../src/physical_optimizer/sort_pushdown.rs | 54 ++--- .../core/src/physical_optimizer/test_utils.rs | 2 +- .../core/src/physical_optimizer/utils.rs | 4 +- .../core/tests/fuzz_cases/window_fuzz.rs | 19 +- .../physical-expr/src/aggregate/first_last.rs | 7 +- datafusion/physical-expr/src/equivalence.rs | 14 +- datafusion/physical-expr/src/partitioning.rs | 19 +- datafusion/physical-expr/src/physical_expr.rs | 20 +- 12 files changed, 177 insertions(+), 233 deletions(-) diff --git a/datafusion/common/src/join_type.rs b/datafusion/common/src/join_type.rs index c62dda66dc22..fd0bdd8ac1eb 100644 --- a/datafusion/common/src/join_type.rs +++ b/datafusion/common/src/join_type.rs @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -//! [`JoinType`] and [`JoinConstraint`] +//! Defines the [`JoinType`], [`JoinConstraint`] and [`JoinSide`] types. use std::{ fmt::{self, Display, Formatter}, @@ -105,7 +105,8 @@ impl Display for JoinSide { } } -/// Used in ColumnIndex to distinguish which side the index is for +/// Join side. +/// Used in `ColumnIndex` type to distinguish which side the index is for. #[derive(Debug, Clone, Copy, PartialEq)] pub enum JoinSide { /// Left side of the join diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index 1f9e1d786e7a..2fd0b5615e88 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -483,7 +483,7 @@ fn reorder_aggregate_keys( parent_required: &[Arc], agg_exec: &AggregateExec, ) -> Result { - let out_put_columns = agg_exec + let output_columns = agg_exec .group_by() .expr() .iter() @@ -491,44 +491,32 @@ fn reorder_aggregate_keys( .map(|(index, (_col, name))| Column::new(name, index)) .collect::>(); - let out_put_exprs = out_put_columns + let output_exprs = output_columns .iter() - .map(|c| Arc::new(c.clone()) as Arc) + .map(|c| Arc::new(c.clone()) as _) .collect::>(); - if parent_required.len() != out_put_exprs.len() + if parent_required.len() != output_exprs.len() || !agg_exec.group_by().null_expr().is_empty() - || expr_list_eq_strict_order(&out_put_exprs, parent_required) + || expr_list_eq_strict_order(&output_exprs, parent_required) { Ok(PlanWithKeyRequirements::new(agg_plan)) } else { - let new_positions = expected_expr_positions(&out_put_exprs, parent_required); + let new_positions = expected_expr_positions(&output_exprs, parent_required); match new_positions { None => Ok(PlanWithKeyRequirements::new(agg_plan)), Some(positions) => { let new_partial_agg = if let Some(agg_exec) = agg_exec.input().as_any().downcast_ref::() - /*AggregateExec { - mode, - group_by, - aggr_expr, - filter_expr, - order_by_expr, - input, - input_schema, - .. - }) = - */ { if matches!(agg_exec.mode(), &AggregateMode::Partial) { - let mut new_group_exprs = vec![]; - for idx in positions.iter() { - new_group_exprs - .push(agg_exec.group_by().expr()[*idx].clone()); - } + let group_exprs = agg_exec.group_by().expr(); + let new_group_exprs = positions + .into_iter() + .map(|idx| group_exprs[idx].clone()) + .collect(); let new_partial_group_by = PhysicalGroupBy::new_single(new_group_exprs); - // new Partial AggregateExec Some(Arc::new(AggregateExec::try_new( AggregateMode::Partial, new_partial_group_by, @@ -546,18 +534,13 @@ fn reorder_aggregate_keys( }; if let Some(partial_agg) = new_partial_agg { // Build new group expressions that correspond to the output of partial_agg - let new_final_group: Vec> = - partial_agg.output_group_expr(); + let group_exprs = partial_agg.group_expr().expr(); + let new_final_group = partial_agg.output_group_expr(); let new_group_by = PhysicalGroupBy::new_single( new_final_group .iter() .enumerate() - .map(|(i, expr)| { - ( - expr.clone(), - partial_agg.group_expr().expr()[i].1.clone(), - ) - }) + .map(|(idx, expr)| (expr.clone(), group_exprs[idx].1.clone())) .collect(), ); @@ -572,29 +555,27 @@ fn reorder_aggregate_keys( )?); // Need to create a new projection to change the expr ordering back - let mut proj_exprs = out_put_columns + let agg_schema = new_final_agg.schema(); + let mut proj_exprs = output_columns .iter() .map(|col| { + let name = col.name(); ( Arc::new(Column::new( - col.name(), - new_final_agg.schema().index_of(col.name()).unwrap(), - )) - as Arc, - col.name().to_owned(), + name, + agg_schema.index_of(name).unwrap(), + )) as _, + name.to_owned(), ) }) .collect::>(); - let agg_schema = new_final_agg.schema(); let agg_fields = agg_schema.fields(); for (idx, field) in - agg_fields.iter().enumerate().skip(out_put_columns.len()) + agg_fields.iter().enumerate().skip(output_columns.len()) { - proj_exprs.push(( - Arc::new(Column::new(field.name().as_str(), idx)) - as Arc, - field.name().clone(), - )) + let name = field.name(); + proj_exprs + .push((Arc::new(Column::new(name, idx)) as _, name.clone())) } // TODO merge adjacent Projections if there are Ok(PlanWithKeyRequirements::new(Arc::new( @@ -612,15 +593,14 @@ fn shift_right_required( parent_required: &[Arc], left_columns_len: usize, ) -> Option>> { - let new_right_required: Vec> = parent_required + let new_right_required = parent_required .iter() .filter_map(|r| { if let Some(col) = r.as_any().downcast_ref::() { - if col.index() >= left_columns_len { - Some( - Arc::new(Column::new(col.name(), col.index() - left_columns_len)) - as Arc, - ) + let idx = col.index(); + if idx >= left_columns_len { + let result = Column::new(col.name(), idx - left_columns_len); + Some(Arc::new(result) as _) } else { None } @@ -631,11 +611,7 @@ fn shift_right_required( .collect::>(); // if the parent required are all comming from the right side, the requirements can be pushdown - if new_right_required.len() != parent_required.len() { - None - } else { - Some(new_right_required) - } + (new_right_required.len() == parent_required.len()).then_some(new_right_required) } /// When the physical planner creates the Joins, the ordering of join keys is from the original query. @@ -659,8 +635,8 @@ fn shift_right_required( /// In that case, the datasources/tables might be pre-partitioned and we can't adjust the key ordering of the datasources /// and then can't apply the Top-Down reordering process. pub(crate) fn reorder_join_keys_to_inputs( - plan: Arc, -) -> Result> { + plan: Arc, +) -> Result> { let plan_any = plan.as_any(); if let Some(HashJoinExec { left, @@ -673,41 +649,34 @@ pub(crate) fn reorder_join_keys_to_inputs( .. }) = plan_any.downcast_ref::() { - match mode { - PartitionMode::Partitioned => { - let join_key_pairs = extract_join_keys(on); - if let Some(( - JoinKeyPairs { - left_keys, - right_keys, - }, - new_positions, - )) = reorder_current_join_keys( - join_key_pairs, - Some(left.output_partitioning()), - Some(right.output_partitioning()), - &left.schema_properties(), - &right.schema_properties(), - ) { - if !new_positions.is_empty() { - let new_join_on = new_join_conditions(&left_keys, &right_keys); - Ok(Arc::new(HashJoinExec::try_new( - left.clone(), - right.clone(), - new_join_on, - filter.clone(), - join_type, - PartitionMode::Partitioned, - *null_equals_null, - )?)) - } else { - Ok(plan) - } - } else { - Ok(plan) + if matches!(mode, PartitionMode::Partitioned) { + let join_key_pairs = extract_join_keys(on); + if let Some(( + JoinKeyPairs { + left_keys, + right_keys, + }, + new_positions, + )) = reorder_current_join_keys( + join_key_pairs, + Some(left.output_partitioning()), + Some(right.output_partitioning()), + &left.schema_properties(), + &right.schema_properties(), + ) { + if !new_positions.is_empty() { + let new_join_on = new_join_conditions(&left_keys, &right_keys); + return Ok(Arc::new(HashJoinExec::try_new( + left.clone(), + right.clone(), + new_join_on, + filter.clone(), + join_type, + PartitionMode::Partitioned, + *null_equals_null, + )?)); } } - _ => Ok(plan), } } else if let Some(SortMergeJoinExec { left, @@ -735,27 +704,21 @@ pub(crate) fn reorder_join_keys_to_inputs( ) { if !new_positions.is_empty() { let new_join_on = new_join_conditions(&left_keys, &right_keys); - let mut new_sort_options = vec![]; - for idx in 0..sort_options.len() { - new_sort_options.push(sort_options[new_positions[idx]]) - } - Ok(Arc::new(SortMergeJoinExec::try_new( + let new_sort_options = (0..sort_options.len()) + .map(|idx| sort_options[new_positions[idx]]) + .collect(); + return Ok(Arc::new(SortMergeJoinExec::try_new( left.clone(), right.clone(), new_join_on, *join_type, new_sort_options, *null_equals_null, - )?)) - } else { - Ok(plan) + )?)); } - } else { - Ok(plan) } - } else { - Ok(plan) } + Ok(plan) } /// Reorder the current join keys ordering based on either left partition or right partition @@ -884,12 +847,7 @@ fn expected_expr_positions( fn extract_join_keys(on: &[(Column, Column)]) -> JoinKeyPairs { let (left_keys, right_keys) = on .iter() - .map(|(l, r)| { - ( - Arc::new(l.clone()) as Arc, - Arc::new(r.clone()) as Arc, - ) - }) + .map(|(l, r)| (Arc::new(l.clone()) as _, Arc::new(r.clone()) as _)) .unzip(); JoinKeyPairs { left_keys, @@ -901,7 +859,7 @@ fn new_join_conditions( new_left_keys: &[Arc], new_right_keys: &[Arc], ) -> Vec<(Column, Column)> { - let new_join_on = new_left_keys + new_left_keys .iter() .zip(new_right_keys.iter()) .map(|(l_key, r_key)| { @@ -910,8 +868,7 @@ fn new_join_conditions( r_key.as_any().downcast_ref::().unwrap().clone(), ) }) - .collect::>(); - new_join_on + .collect() } /// Updates `dist_onward` such that, to keep track of @@ -975,10 +932,10 @@ fn add_roundrobin_on_top( // (determined by flag `config.optimizer.bounded_order_preserving_variants`) let should_preserve_ordering = input.output_ordering().is_some(); - let new_plan = Arc::new( - RepartitionExec::try_new(input, Partitioning::RoundRobinBatch(n_target))? - .with_preserve_order(should_preserve_ordering), - ) as Arc; + let partitioning = Partitioning::RoundRobinBatch(n_target); + let repartition = RepartitionExec::try_new(input, partitioning)?; + let new_plan = Arc::new(repartition.with_preserve_order(should_preserve_ordering)) + as Arc; // update distribution onward with new operator update_distribution_onward(new_plan.clone(), dist_onward, input_idx); @@ -1007,7 +964,7 @@ fn add_roundrobin_on_top( /// /// # Returns /// -/// A [Result] object that contains new execution plan, where desired distribution is +/// A [`Result`] object that contains new execution plan, where desired distribution is /// satisfied by adding Hash Repartition. fn add_hash_on_top( input: Arc, @@ -1051,10 +1008,10 @@ fn add_hash_on_top( } else { input }; - new_plan = Arc::new( - RepartitionExec::try_new(new_plan, Partitioning::Hash(hash_exprs, n_target))? - .with_preserve_order(should_preserve_ordering), - ) as _; + let partitioning = Partitioning::Hash(hash_exprs, n_target); + let repartition = RepartitionExec::try_new(new_plan, partitioning)?; + new_plan = + Arc::new(repartition.with_preserve_order(should_preserve_ordering)) as _; // update distribution onward with new operator update_distribution_onward(new_plan.clone(), dist_onward, input_idx); @@ -1144,7 +1101,7 @@ fn remove_dist_changing_operators( { // All of above operators have a single child. When we remove the top // operator, we take the first child. - plan = plan.children()[0].clone(); + plan = plan.children().swap_remove(0); distribution_onwards = get_children_exectrees(plan.children().len(), &distribution_onwards[0]); } @@ -1197,14 +1154,14 @@ fn replace_order_preserving_variants_helper( } if is_sort_preserving_merge(&exec_tree.plan) { return Ok(Arc::new(CoalescePartitionsExec::new( - updated_children[0].clone(), + updated_children.swap_remove(0), ))); } if let Some(repartition) = exec_tree.plan.as_any().downcast_ref::() { if repartition.preserve_order() { return Ok(Arc::new( RepartitionExec::try_new( - updated_children[0].clone(), + updated_children.swap_remove(0), repartition.partitioning().clone(), )? .with_preserve_order(false), @@ -1368,7 +1325,7 @@ fn ensure_distribution( // make sure ordering requirements are still satisfied after. if ordering_satisfied { // Make sure to satisfy ordering requirement: - add_sort_above(&mut child, required_input_ordering, None)?; + add_sort_above(&mut child, required_input_ordering, None); } } // Stop tracking distribution changing operators @@ -1421,7 +1378,7 @@ fn ensure_distribution( // Data Arc::new(InterleaveExec::try_new(new_children)?) } else { - plan.clone().with_new_children(new_children)? + plan.with_new_children(new_children)? }, distribution_onwards, }; @@ -1618,7 +1575,7 @@ impl PlanWithKeyRequirements { let length = child.children().len(); PlanWithKeyRequirements { plan: child, - required_key_ordering: from_parent.clone(), + required_key_ordering: from_parent, request_key_ordering: vec![None; length], } }) diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index 51353b7caf42..b245efdb4522 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -17,8 +17,8 @@ //! 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 +//! - 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. @@ -401,7 +401,7 @@ fn parallelize_sorts( &mut prev_layer, &PhysicalSortRequirement::from_sort_exprs(sort_exprs), fetch, - )?; + ); let spm = SortPreservingMergeExec::new(sort_exprs.to_vec(), prev_layer) .with_fetch(fetch); return Ok(Transformed::Yes(PlanWithCorrespondingCoalescePartitions { @@ -456,7 +456,7 @@ fn ensure_sorting( { // Make sure we preserve the ordering requirements: update_child_to_remove_unnecessary_sort(child, sort_onwards, &plan)?; - add_sort_above(child, &required_ordering, None)?; + add_sort_above(child, &required_ordering, None); if is_sort(child) { *sort_onwards = Some(ExecTree::new(child.clone(), idx, vec![])); } else { @@ -466,7 +466,7 @@ fn ensure_sorting( } (Some(required), None) => { // Ordering requirement is not met, we should add a `SortExec` to the plan. - add_sort_above(child, &required, None)?; + add_sort_above(child, &required, None); *sort_onwards = Some(ExecTree::new(child.clone(), idx, vec![])); } (None, Some(_)) => { @@ -492,9 +492,10 @@ fn ensure_sorting( { // This SortPreservingMergeExec is unnecessary, input already has a // single partition. + sort_onwards.truncate(1); return Ok(Transformed::Yes(PlanWithCorrespondingSort { - plan: children[0].clone(), - sort_onwards: vec![sort_onwards[0].clone()], + plan: children.swap_remove(0), + sort_onwards, })); } Ok(Transformed::Yes(PlanWithCorrespondingSort { @@ -600,7 +601,7 @@ fn analyze_window_sort_removal( .swap_remove(0) .unwrap_or_default(); // Satisfy the ordering requirement so that the window can run: - add_sort_above(&mut window_child, &reqs, None)?; + add_sort_above(&mut window_child, &reqs, None); let uses_bounded_memory = window_expr.iter().all(|e| e.uses_bounded_memory()); let new_window = if uses_bounded_memory { @@ -644,7 +645,7 @@ fn remove_corresponding_coalesce_in_sub_plan( && is_repartition(&new_plan) && is_repartition(parent) { - new_plan = new_plan.children()[0].clone() + new_plan = new_plan.children().swap_remove(0) } new_plan } else { @@ -684,7 +685,7 @@ fn remove_corresponding_sort_from_sub_plan( ) -> Result> { // A `SortExec` is always at the bottom of the tree. let mut updated_plan = if is_sort(&sort_onwards.plan) { - sort_onwards.plan.children()[0].clone() + sort_onwards.plan.children().swap_remove(0) } else { let plan = &sort_onwards.plan; let mut children = plan.children(); @@ -698,12 +699,12 @@ fn remove_corresponding_sort_from_sub_plan( } // Replace with variants that do not preserve order. if is_sort_preserving_merge(plan) { - children[0].clone() + children.swap_remove(0) } else if let Some(repartition) = plan.as_any().downcast_ref::() { Arc::new( RepartitionExec::try_new( - children[0].clone(), + children.swap_remove(0), repartition.partitioning().clone(), )? .with_preserve_order(false), @@ -725,7 +726,7 @@ fn remove_corresponding_sort_from_sub_plan( updated_plan, )); } else { - updated_plan = Arc::new(CoalescePartitionsExec::new(updated_plan.clone())); + updated_plan = Arc::new(CoalescePartitionsExec::new(updated_plan)); } } Ok(updated_plan) @@ -772,8 +773,7 @@ mod tests { use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use datafusion_common::Result; use datafusion_expr::JoinType; - use datafusion_physical_expr::expressions::Column; - use datafusion_physical_expr::expressions::{col, NotExpr}; + use datafusion_physical_expr::expressions::{col, Column, NotExpr}; fn create_test_schema() -> Result { let nullable_column = Field::new("nullable_col", DataType::Int32, true); diff --git a/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs b/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs index abfb598f3528..edab3a0dc481 100644 --- a/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs +++ b/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs @@ -70,14 +70,15 @@ impl OrderPreservationContext { // ordering, (or that can maintain ordering with the replacement of // its variant) let plan = item.plan; + let children = plan.children(); let ordering_onwards = item.ordering_onwards; - if plan.children().is_empty() { + if children.is_empty() { // Plan has no children, there is nothing to propagate. None } else if ordering_onwards[0].is_none() && ((is_repartition(&plan) && !plan.maintains_input_order()[0]) || (is_coalesce_partitions(&plan) - && plan.children()[0].output_ordering().is_some())) + && children[0].output_ordering().is_some())) { Some(ExecTree::new(plan, idx, vec![])) } else { @@ -174,19 +175,18 @@ fn get_updated_plan( // When a `RepartitionExec` doesn't preserve ordering, replace it with // a `SortPreservingRepartitionExec` if appropriate: if is_repartition(&plan) && !plan.maintains_input_order()[0] && is_spr_better { - let child = plan.children()[0].clone(); - plan = Arc::new( - RepartitionExec::try_new(child, plan.output_partitioning())? - .with_preserve_order(true), - ) as _ + let child = plan.children().swap_remove(0); + let repartition = RepartitionExec::try_new(child, plan.output_partitioning())?; + plan = Arc::new(repartition.with_preserve_order(true)) as _ } // When the input of a `CoalescePartitionsExec` has an ordering, replace it // with a `SortPreservingMergeExec` if appropriate: + let mut children = plan.children(); if is_coalesce_partitions(&plan) - && plan.children()[0].output_ordering().is_some() + && children[0].output_ordering().is_some() && is_spm_better { - let child = plan.children()[0].clone(); + let child = children.swap_remove(0); plan = Arc::new(SortPreservingMergeExec::new( child.output_ordering().unwrap_or(&[]).to_vec(), child, @@ -275,21 +275,21 @@ pub(crate) fn replace_with_order_preserving_variants( mod tests { use super::*; - use crate::prelude::SessionConfig; - use crate::datasource::file_format::file_compression_type::FileCompressionType; use crate::datasource::listing::PartitionedFile; use crate::datasource::physical_plan::{CsvExec, FileScanConfig}; use crate::physical_plan::coalesce_batches::CoalesceBatchesExec; use crate::physical_plan::coalesce_partitions::CoalescePartitionsExec; - use crate::physical_plan::filter::FilterExec; use crate::physical_plan::joins::{HashJoinExec, PartitionMode}; use crate::physical_plan::repartition::RepartitionExec; use crate::physical_plan::sorts::sort::SortExec; use crate::physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; use crate::physical_plan::{displayable, Partitioning}; + use crate::prelude::SessionConfig; + use arrow::compute::SortOptions; + use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; use datafusion_common::tree_node::TreeNode; use datafusion_common::{Result, Statistics}; use datafusion_execution::object_store::ObjectStoreUrl; @@ -297,9 +297,6 @@ mod tests { use datafusion_physical_expr::expressions::{self, col, Column}; use datafusion_physical_expr::PhysicalSortExpr; - use arrow::compute::SortOptions; - use arrow::datatypes::{DataType, Field, Schema, SchemaRef}; - /// Runs the `replace_with_order_preserving_variants` sub-rule and asserts the plan /// against the original and expected plans. /// diff --git a/datafusion/core/src/physical_optimizer/sort_pushdown.rs b/datafusion/core/src/physical_optimizer/sort_pushdown.rs index 2b0c1a9e95f4..3c49953637a7 100644 --- a/datafusion/core/src/physical_optimizer/sort_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/sort_pushdown.rs @@ -128,27 +128,29 @@ pub(crate) fn pushdown_sorts( const ERR_MSG: &str = "Expects parent requirement to contain something"; let err = || plan_datafusion_err!("{}", ERR_MSG); if let Some(sort_exec) = plan.as_any().downcast_ref::() { - let mut new_plan = plan.clone(); - if !plan + let new_plan = if !plan .schema_properties() .ordering_satisfy_requirement(parent_required) { // If the current plan is a SortExec, modify it to satisfy parent requirements: let parent_requirement = parent_required.ok_or_else(err)?; - new_plan = sort_exec.input().clone(); - add_sort_above(&mut new_plan, parent_requirement, sort_exec.fetch())?; + let mut new_plan = sort_exec.input().clone(); + add_sort_above(&mut new_plan, parent_requirement, sort_exec.fetch()); + new_plan + } else { + requirements.plan }; let required_ordering = new_plan .output_ordering() .map(PhysicalSortRequirement::from_sort_exprs); // Since new_plan is a SortExec, we can safely get the 0th index. - let child = &new_plan.children()[0]; + let child = new_plan.children().swap_remove(0); if let Some(adjusted) = - pushdown_requirement_to_children(child, required_ordering.as_deref())? + pushdown_requirement_to_children(&child, required_ordering.as_deref())? { // Can push down requirements Ok(Transformed::Yes(SortPushDown { - plan: child.clone(), + plan: child, required_ordering: None, adjusted_request_ordering: adjusted, })) @@ -171,15 +173,15 @@ pub(crate) fn pushdown_sorts( // Can not satisfy the parent requirements, check whether the requirements can be pushed down: if let Some(adjusted) = pushdown_requirement_to_children(plan, parent_required)? { Ok(Transformed::Yes(SortPushDown { - plan: plan.clone(), + plan: requirements.plan, required_ordering: None, adjusted_request_ordering: adjusted, })) } else { // Can not push down requirements, add new SortExec: - let mut new_plan = plan.clone(); + let mut new_plan = requirements.plan; let parent_requirement = parent_required.ok_or_else(err)?; - add_sort_above(&mut new_plan, parent_requirement, None)?; + add_sort_above(&mut new_plan, parent_requirement, None); Ok(Transformed::Yes(SortPushDown::init(new_plan))) } } @@ -195,7 +197,7 @@ fn pushdown_requirement_to_children( if is_window(plan) { let required_input_ordering = plan.required_input_ordering(); let request_child = required_input_ordering[0].as_deref(); - let child_plan = plan.children()[0].clone(); + let child_plan = plan.children().swap_remove(0); match determine_children_requirement(parent_required, request_child, child_plan) { RequirementsCompatibility::Satisfy => { Ok(Some(vec![request_child.map(|r| r.to_vec())])) @@ -207,7 +209,7 @@ fn pushdown_requirement_to_children( // 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 Ok(Some(vec![ - parent_required.map(|elem| elem.to_vec()); + parent_required.map(|item| item.to_vec()); plan.children().len() ])) } else if let Some(smj) = plan.as_any().downcast_ref::() { @@ -230,9 +232,8 @@ fn pushdown_requirement_to_children( smj.schema().fields.len() - smj.right().schema().fields.len(); let new_right_required = shift_right_required(parent_required.ok_or_else(err)?, right_offset)?; - let new_right_required_expr = PhysicalSortRequirement::to_sort_exprs( - new_right_required.iter().cloned(), - ); + let new_right_required_expr = + PhysicalSortRequirement::to_sort_exprs(new_right_required); try_pushdown_requirements_to_join( smj, parent_required, @@ -271,20 +272,20 @@ fn pushdown_requirement_to_children( } else { // Can push-down through SortPreservingMergeExec, because parent requirement is finer // than SortPreservingMergeExec output ordering. - Ok(Some(vec![parent_required.map(|elem| elem.to_vec())])) + Ok(Some(vec![parent_required.map(|item| item.to_vec())])) } } else { Ok(Some( maintains_input_order - .iter() + .into_iter() .map(|flag| { - if *flag { - parent_required.map(|elem| elem.to_vec()) + if flag { + parent_required.map(|item| item.to_vec()) } else { None } }) - .collect::>(), + .collect(), )) } // TODO: Add support for Projection push down @@ -328,30 +329,33 @@ fn try_pushdown_requirements_to_join( JoinSide::Left => (sort_expr.as_slice(), right_ordering), JoinSide::Right => (left_ordering, sort_expr.as_slice()), }; + let join_type = smj.join_type(); + let probe_side = SortMergeJoinExec::probe_side(&join_type); let new_output_ordering = calculate_join_output_ordering( new_left_ordering, new_right_ordering, - smj.join_type(), + join_type, smj.on(), smj.left().schema().fields.len(), &smj.maintains_input_order(), - Some(SortMergeJoinExec::probe_side(&smj.join_type())), + Some(probe_side), )?; let mut smj_oeq = smj.schema_properties(); // smj will have this ordering when its input changes. smj_oeq = smj_oeq.with_reorder(new_output_ordering.unwrap_or(vec![])); let should_pushdown = smj_oeq.ordering_satisfy_requirement(parent_required); Ok(should_pushdown.then(|| { - let required_input_ordering = smj.required_input_ordering(); + let mut required_input_ordering = smj.required_input_ordering(); let new_req = Some(PhysicalSortRequirement::from_sort_exprs(&sort_expr)); match push_side { JoinSide::Left => { - vec![new_req, required_input_ordering[1].clone()] + required_input_ordering[0] = new_req; } JoinSide::Right => { - vec![required_input_ordering[0].clone(), new_req] + required_input_ordering[1] = new_req; } } + required_input_ordering })) } diff --git a/datafusion/core/src/physical_optimizer/test_utils.rs b/datafusion/core/src/physical_optimizer/test_utils.rs index 2e2ba52b836f..159ee5089075 100644 --- a/datafusion/core/src/physical_optimizer/test_utils.rs +++ b/datafusion/core/src/physical_optimizer/test_utils.rs @@ -44,9 +44,9 @@ use datafusion_execution::object_store::ObjectStoreUrl; use datafusion_expr::{AggregateFunction, WindowFrame, WindowFunction}; use datafusion_physical_expr::expressions::col; use datafusion_physical_expr::{PhysicalExpr, PhysicalSortExpr}; +use datafusion_physical_plan::windows::PartitionSearchMode; use async_trait::async_trait; -use datafusion_physical_plan::windows::PartitionSearchMode; async fn register_current_csv( ctx: &SessionContext, diff --git a/datafusion/core/src/physical_optimizer/utils.rs b/datafusion/core/src/physical_optimizer/utils.rs index 0e180449112e..ebbecd84c688 100644 --- a/datafusion/core/src/physical_optimizer/utils.rs +++ b/datafusion/core/src/physical_optimizer/utils.rs @@ -21,7 +21,6 @@ use std::fmt; use std::fmt::Formatter; use std::sync::Arc; -use crate::error::Result; use crate::physical_plan::coalesce_partitions::CoalescePartitionsExec; use crate::physical_plan::limit::{GlobalLimitExec, LocalLimitExec}; use crate::physical_plan::repartition::RepartitionExec; @@ -102,7 +101,7 @@ pub fn add_sort_above( node: &mut Arc, sort_requirement: &[PhysicalSortRequirement], fetch: Option, -) -> Result<()> { +) { // If the ordering requirement is already satisfied, do not add a sort. if !node .schema_properties() @@ -117,7 +116,6 @@ pub fn add_sort_above( new_sort }) as _ } - Ok(()) } /// Checks whether the given operator is a limit; diff --git a/datafusion/core/tests/fuzz_cases/window_fuzz.rs b/datafusion/core/tests/fuzz_cases/window_fuzz.rs index 5a1b48edec12..a49a69147314 100644 --- a/datafusion/core/tests/fuzz_cases/window_fuzz.rs +++ b/datafusion/core/tests/fuzz_cases/window_fuzz.rs @@ -22,32 +22,31 @@ use arrow::compute::{concat_batches, SortOptions}; use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; use arrow::util::pretty::pretty_format_batches; -use hashbrown::HashMap; -use rand::rngs::StdRng; -use rand::{Rng, SeedableRng}; - use datafusion::physical_plan::memory::MemoryExec; use datafusion::physical_plan::sorts::sort::SortExec; use datafusion::physical_plan::windows::{ - create_window_expr, BoundedWindowAggExec, WindowAggExec, + create_window_expr, BoundedWindowAggExec, PartitionSearchMode, WindowAggExec, }; use datafusion::physical_plan::{collect, ExecutionPlan}; +use datafusion::prelude::{SessionConfig, SessionContext}; +use datafusion_common::{Result, ScalarValue}; +use datafusion_expr::type_coercion::aggregates::coerce_types; use datafusion_expr::{ AggregateFunction, BuiltInWindowFunction, WindowFrame, WindowFrameBound, WindowFrameUnits, WindowFunction, }; - -use datafusion::prelude::{SessionConfig, SessionContext}; -use datafusion_common::{Result, ScalarValue}; -use datafusion_expr::type_coercion::aggregates::coerce_types; use datafusion_physical_expr::expressions::{cast, col, lit}; use datafusion_physical_expr::{PhysicalExpr, PhysicalSortExpr}; -use datafusion_physical_plan::windows::PartitionSearchMode; use test_utils::add_empty_batches; +use hashbrown::HashMap; +use rand::rngs::StdRng; +use rand::{Rng, SeedableRng}; + #[cfg(test)] mod tests { use super::*; + use datafusion_physical_plan::windows::PartitionSearchMode::{ Linear, PartiallySorted, Sorted, }; diff --git a/datafusion/physical-expr/src/aggregate/first_last.rs b/datafusion/physical-expr/src/aggregate/first_last.rs index ce7a1daeec64..a4e0a6dc49a9 100644 --- a/datafusion/physical-expr/src/aggregate/first_last.rs +++ b/datafusion/physical-expr/src/aggregate/first_last.rs @@ -26,12 +26,9 @@ use crate::{ reverse_order_bys, AggregateExpr, LexOrdering, PhysicalExpr, PhysicalSortExpr, }; -use arrow::array::ArrayRef; -use arrow::compute; -use arrow::compute::{lexsort_to_indices, SortColumn}; +use arrow::array::{Array, ArrayRef, AsArray, BooleanArray}; +use arrow::compute::{self, lexsort_to_indices, SortColumn}; use arrow::datatypes::{DataType, Field}; -use arrow_array::cast::AsArray; -use arrow_array::{Array, BooleanArray}; use arrow_schema::SortOptions; use datafusion_common::utils::{compare_rows, get_arrayref_at_indices, get_row_at_idx}; use datafusion_common::{DataFusionError, Result, ScalarValue}; diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 5e0b80be365d..9c1548d16a0b 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -186,7 +186,7 @@ impl EquivalentGroups { exprs .iter() .map(|expr| self.normalize_expr(expr.clone())) - .collect::>() + .collect() } /// This function normalizes `sort_requirement` according to `EquivalenceClasses` in the `self`. @@ -1238,15 +1238,11 @@ pub fn schema_properties_helper( orderings: &[LexOrdering], ) -> SchemaProperties { let mut oep = SchemaProperties::new(schema); - if orderings.is_empty() { - // Return an empty `SchemaProperties`: - oep - } else { - oep.add_ordering_equivalent_group(OrderingEquivalentGroup::new( - orderings.to_vec(), - )); - oep + if !orderings.is_empty() { + let group = OrderingEquivalentGroup::new(orderings.to_vec()); + oep.add_ordering_equivalent_group(group); } + oep } /// This function constructs a duplicate-free `LexOrderingReq` by filtering out duplicate diff --git a/datafusion/physical-expr/src/partitioning.rs b/datafusion/physical-expr/src/partitioning.rs index 4e1f728d01fc..7a7435a7174d 100644 --- a/datafusion/physical-expr/src/partitioning.rs +++ b/datafusion/physical-expr/src/partitioning.rs @@ -93,16 +93,13 @@ impl Partitioning { .iter() .map(|e| eq_groups.normalize_expr(e.clone())) .collect::>(); - expr_list_eq_strict_order( + return expr_list_eq_strict_order( &normalized_required_exprs, &normalized_partition_exprs, - ) - } else { - fast_match + ); } - } else { - fast_match } + fast_match } _ => false, } @@ -158,16 +155,14 @@ impl Distribution { #[cfg(test)] mod tests { + use std::sync::Arc; + + use super::*; use crate::expressions::Column; - use super::*; - use arrow::datatypes::DataType; - use arrow::datatypes::Field; - use arrow::datatypes::Schema; + use arrow::datatypes::{DataType, Field, Schema}; use datafusion_common::Result; - use std::sync::Arc; - #[test] fn partitioning_satisfy_distribution() -> Result<()> { let schema = Arc::new(Schema::new(vec![ diff --git a/datafusion/physical-expr/src/physical_expr.rs b/datafusion/physical-expr/src/physical_expr.rs index f69be0c32ca8..291c58348e10 100644 --- a/datafusion/physical-expr/src/physical_expr.rs +++ b/datafusion/physical-expr/src/physical_expr.rs @@ -15,6 +15,11 @@ // specific language governing permissions and limitations // under the License. +use std::any::Any; +use std::fmt::{Debug, Display}; +use std::hash::{Hash, Hasher}; +use std::sync::Arc; + use crate::intervals::Interval; use crate::sort_properties::SortProperties; use crate::utils::scatter; @@ -28,10 +33,6 @@ use datafusion_common::{internal_err, not_impl_err, DataFusionError, Result}; use datafusion_expr::ColumnarValue; use itertools::izip; -use std::any::Any; -use std::fmt::{Debug, Display}; -use std::hash::{Hash, Hasher}; -use std::sync::Arc; /// Expression that can be evaluated against a RecordBatch /// A Physical expression knows its type, nullability and how to evaluate itself. @@ -55,13 +56,12 @@ pub trait PhysicalExpr: Send + Sync + Display + Debug + PartialEq { let tmp_batch = filter_record_batch(batch, selection)?; let tmp_result = self.evaluate(&tmp_batch)?; - // All values from the `selection` filter are true. + if batch.num_rows() == tmp_batch.num_rows() { - return Ok(tmp_result); - } - if let ColumnarValue::Array(a) = tmp_result { - let result = scatter(selection, a.as_ref())?; - Ok(ColumnarValue::Array(result)) + // All values from the `selection` filter are true. + Ok(tmp_result) + } else if let ColumnarValue::Array(a) = tmp_result { + scatter(selection, a.as_ref()).map(ColumnarValue::Array) } else { Ok(tmp_result) } From edd42e34fc3c4576cebab9b9d6388abfeef30913 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Fri, 20 Oct 2023 14:14:45 +0300 Subject: [PATCH 074/122] Review Part 2 --- .../physical-expr/src/aggregate/utils.rs | 12 +-- datafusion/physical-expr/src/equivalence.rs | 93 +++++++++---------- datafusion/physical-expr/src/partitioning.rs | 4 +- datafusion/physical-expr/src/physical_expr.rs | 8 +- datafusion/physical-expr/src/sort_expr.rs | 3 +- .../physical-expr/src/window/built_in.rs | 9 +- .../physical-plan/src/aggregates/mod.rs | 63 ++++++------- .../physical-plan/src/aggregates/order/mod.rs | 15 +-- .../physical-plan/src/aggregates/row_hash.rs | 60 ++++++------ .../physical-plan/src/coalesce_batches.rs | 10 +- datafusion/physical-plan/src/common.rs | 10 +- datafusion/physical-plan/src/filter.rs | 30 +++--- .../physical-plan/src/joins/cross_join.rs | 17 ++-- .../physical-plan/src/joins/hash_join.rs | 35 +++---- .../src/joins/sort_merge_join.rs | 3 +- .../src/joins/symmetric_hash_join.rs | 3 +- datafusion/physical-plan/src/joins/utils.rs | 21 ++--- datafusion/physical-plan/src/projection.rs | 36 ++++--- .../physical-plan/src/repartition/mod.rs | 7 +- .../src/sorts/sort_preserving_merge.rs | 7 +- datafusion/physical-plan/src/union.rs | 25 +++-- datafusion/physical-plan/src/windows/mod.rs | 19 ++-- 22 files changed, 226 insertions(+), 264 deletions(-) diff --git a/datafusion/physical-expr/src/aggregate/utils.rs b/datafusion/physical-expr/src/aggregate/utils.rs index 2f473f7608b0..9da78e22749e 100644 --- a/datafusion/physical-expr/src/aggregate/utils.rs +++ b/datafusion/physical-expr/src/aggregate/utils.rs @@ -168,14 +168,10 @@ pub fn adjust_output_array( /// for [`AggregateExpr`] aggregation expressions and allows comparing the equality /// between the trait objects. pub fn down_cast_any_ref(any: &dyn Any) -> &dyn Any { - if any.is::>() { - any.downcast_ref::>() - .unwrap() - .as_any() - } else if any.is::>() { - any.downcast_ref::>() - .unwrap() - .as_any() + if let Some(obj) = any.downcast_ref::>() { + obj.as_any() + } else if let Some(obj) = any.downcast_ref::>() { + obj.as_any() } else { any } diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 9c1548d16a0b..485ca15c707f 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -27,7 +27,7 @@ use crate::physical_expr::{deduplicate_physical_exprs, have_common_entries}; use crate::sort_properties::{ExprOrdering, SortProperties}; use arrow_schema::SortOptions; use datafusion_common::tree_node::{Transformed, TreeNode}; -use datafusion_common::{DataFusionError, JoinSide, JoinType, Result}; +use datafusion_common::{JoinSide, JoinType, Result}; use itertools::izip; use std::hash::Hash; use std::sync::Arc; @@ -381,25 +381,25 @@ impl EquivalentGroups { right_eq_classes: &EquivalentGroups, left_columns_len: usize, on: &[(Column, Column)], - ) -> Result { - let mut res = EquivalentGroups::empty(); + ) -> EquivalentGroups { + let mut result = EquivalentGroups::empty(); match join_type { JoinType::Inner | JoinType::Left | JoinType::Full | JoinType::Right => { - res.extend(self.clone()); + result.extend(self.clone()); let updated_eq_classes = right_eq_classes .iter() .map(|eq_class| { add_offset_to_exprs(eq_class.to_vec(), left_columns_len) }) - .collect::>>()?; + .collect(); - res.extend(EquivalentGroups::new(updated_eq_classes)); + result.extend(EquivalentGroups::new(updated_eq_classes)); } JoinType::LeftSemi | JoinType::LeftAnti => { - res.extend(self.clone()); + result.extend(self.clone()); } JoinType::RightSemi | JoinType::RightAnti => { - res.extend(right_eq_classes.clone()); + result.extend(right_eq_classes.clone()); } } // In the inner join, expressions in the on are equal at the resulting table. @@ -409,10 +409,10 @@ impl EquivalentGroups { let new_rhs = Arc::new(Column::new(rhs.name(), rhs.index() + left_columns_len)) as _; - res.add_equal_conditions((&new_lhs, &new_rhs)); + result.add_equal_conditions((&new_lhs, &new_rhs)); }); } - Ok(res) + result } } @@ -549,13 +549,13 @@ impl OrderingEquivalentGroup { } /// Adds `offset` value to the index of each expression inside `OrderingEquivalentGroup`. - pub fn add_offset(&self, offset: usize) -> Result { - let res = self - .inner - .iter() - .map(|ordering| add_offset_to_lex_ordering(ordering, offset)) - .collect::>>()?; - Ok(OrderingEquivalentGroup::new(res)) + pub fn add_offset(&self, offset: usize) -> OrderingEquivalentGroup { + OrderingEquivalentGroup::new( + self.inner + .iter() + .map(|ordering| add_offset_to_lex_ordering(ordering, offset)) + .collect(), + ) } /// Return finer ordering between lhs and rhs. @@ -643,8 +643,8 @@ impl SchemaProperties { } /// Get schema. - pub fn schema(&self) -> SchemaRef { - self.schema.clone() + pub fn schema(&self) -> &SchemaRef { + &self.schema } /// Return a reference to the ordering equivalent group @@ -1161,13 +1161,13 @@ pub fn join_schema_properties( maintains_input_order: &[bool], probe_side: Option, on: &[(Column, Column)], -) -> Result { +) -> SchemaProperties { let left_columns_len = left.schema.fields.len(); let mut new_properties = SchemaProperties::new(join_schema); let join_eq_groups = left.eq_groups() - .join(join_type, right.eq_groups(), left_columns_len, on)?; + .join(join_type, right.eq_groups(), left_columns_len, on); new_properties.add_equivalent_groups(join_eq_groups); // All joins have 2 children @@ -1178,9 +1178,7 @@ pub fn join_schema_properties( let right_oeq_class = right.oeq_group(); match (left_maintains, right_maintains) { (true, true) => { - return Err(DataFusionError::Plan( - "Cannot maintain ordering of both sides".to_string(), - )) + unreachable!("Cannot maintain ordering of both sides"); } (true, false) => { // In this special case, right side ordering can be prefixed with left side ordering. @@ -1189,7 +1187,7 @@ pub fn join_schema_properties( join_type, right_oeq_class, left_columns_len, - )?; + ); // Right side ordering equivalence properties should be prepended with // those of the left side while constructing output ordering equivalence @@ -1210,7 +1208,7 @@ pub fn join_schema_properties( join_type, right.oeq_group(), left_columns_len, - )?; + ); // In this special case, left side ordering can be prefixed with right side ordering. if let (Some(JoinSide::Right), JoinType::Inner) = (probe_side, join_type) { // Left side ordering equivalence properties should be prepended with @@ -1229,7 +1227,7 @@ pub fn join_schema_properties( } (false, false) => {} } - Ok(new_properties) + new_properties } /// Constructs a `SchemaProperties` struct from the given `orderings`. @@ -1279,19 +1277,19 @@ fn prune_sort_reqs_with_constants( fn add_offset_to_exprs( exprs: Vec>, offset: usize, -) -> Result>> { +) -> Vec> { exprs .into_iter() .map(|item| add_offset_to_expr(item, offset)) - .collect::>>() + .collect() } /// Adds the `offset` value to `Column` indices inside `expr`. This function is /// generally used during the update of the right table schema in join operations. -fn add_offset_to_expr( +pub fn add_offset_to_expr( expr: Arc, offset: usize, -) -> Result> { +) -> Arc { expr.transform_down(&|e| match e.as_any().downcast_ref::() { Some(col) => Ok(Transformed::Yes(Arc::new(Column::new( col.name(), @@ -1299,17 +1297,20 @@ fn add_offset_to_expr( )))), None => Ok(Transformed::No(e)), }) + .unwrap() + // Note that we can safely unwrap here since our transform always returns + // an `Ok` value. } /// Adds the `offset` value to `Column` indices inside `sort_expr.expr`. fn add_offset_to_sort_expr( sort_expr: &PhysicalSortExpr, offset: usize, -) -> Result { - Ok(PhysicalSortExpr { - expr: add_offset_to_expr(sort_expr.expr.clone(), offset)?, +) -> PhysicalSortExpr { + PhysicalSortExpr { + expr: add_offset_to_expr(sort_expr.expr.clone(), offset), options: sort_expr.options, - }) + } } /// Adds the `offset` value to `Column` indices for each `sort_expr.expr` @@ -1317,7 +1318,7 @@ fn add_offset_to_sort_expr( pub fn add_offset_to_lex_ordering( sort_exprs: LexOrderingRef, offset: usize, -) -> Result { +) -> LexOrdering { sort_exprs .iter() .map(|sort_expr| add_offset_to_sort_expr(sort_expr, offset)) @@ -1382,16 +1383,14 @@ fn get_updated_right_ordering_equivalent_group( join_type: &JoinType, right_oeq_group: &OrderingEquivalentGroup, left_columns_len: usize, -) -> Result { - match join_type { - // In these modes, indices of the right schema should be offset by - // the left table size. - JoinType::Inner | JoinType::Left | JoinType::Full | JoinType::Right => { - return right_oeq_group.add_offset(left_columns_len) - } - _ => {} - }; - Ok(right_oeq_group.clone()) +) -> OrderingEquivalentGroup { + if matches!( + join_type, + JoinType::Inner | JoinType::Left | JoinType::Full | JoinType::Right + ) { + return right_oeq_group.add_offset(left_columns_len); + } + right_oeq_group.clone() } #[cfg(test)] @@ -2130,7 +2129,7 @@ mod tests { &join_type, &right_oeq_class, left_columns_len, - )?; + ); join_schema_properties.add_ordering_equivalent_group(result); let result = join_schema_properties.oeq_group().clone(); diff --git a/datafusion/physical-expr/src/partitioning.rs b/datafusion/physical-expr/src/partitioning.rs index 7a7435a7174d..c9a38d7ba443 100644 --- a/datafusion/physical-expr/src/partitioning.rs +++ b/datafusion/physical-expr/src/partitioning.rs @@ -155,8 +155,8 @@ impl Distribution { #[cfg(test)] mod tests { - use std::sync::Arc; - + use std::sync::Arc; + use super::*; use crate::expressions::Column; diff --git a/datafusion/physical-expr/src/physical_expr.rs b/datafusion/physical-expr/src/physical_expr.rs index 291c58348e10..b57716782751 100644 --- a/datafusion/physical-expr/src/physical_expr.rs +++ b/datafusion/physical-expr/src/physical_expr.rs @@ -242,7 +242,7 @@ pub fn deduplicate_physical_exprs( unique_physical_exprs } -/// Check whether vectors in the arguments have common entries +/// Checks whether the given slices have any common entries. pub fn have_common_entries( lhs: &[Arc], rhs: &[Arc], @@ -250,7 +250,7 @@ pub fn have_common_entries( lhs.iter().any(|expr| physical_exprs_contains(rhs, expr)) } -/// Check whether physical exprs vectors are equal. +/// Checks whether the given physical expression slices are equal. #[allow(dead_code)] pub fn physical_exprs_equal( lhs: &[Arc], @@ -261,13 +261,15 @@ pub fn physical_exprs_equal( #[cfg(test)] mod tests { + use std::sync::Arc; + use crate::expressions::{Column, Literal}; use crate::physical_expr::{ deduplicate_physical_exprs, have_common_entries, physical_exprs_equal, }; use crate::{physical_exprs_contains, PhysicalExpr}; + use datafusion_common::{Result, ScalarValue}; - use std::sync::Arc; #[test] fn test_physical_exprs_contains() -> Result<()> { diff --git a/datafusion/physical-expr/src/sort_expr.rs b/datafusion/physical-expr/src/sort_expr.rs index 0581001f8513..5bf7d70c186a 100644 --- a/datafusion/physical-expr/src/sort_expr.rs +++ b/datafusion/physical-expr/src/sort_expr.rs @@ -26,8 +26,7 @@ use crate::PhysicalExpr; use arrow::compute::kernels::sort::{SortColumn, SortOptions}; use arrow::record_batch::RecordBatch; use arrow_schema::Schema; -use datafusion_common::plan_err; -use datafusion_common::{DataFusionError, Result}; +use datafusion_common::{plan_err, DataFusionError, Result}; use datafusion_expr::ColumnarValue; /// Represents Sort operation for a column in a RecordBatch diff --git a/datafusion/physical-expr/src/window/built_in.rs b/datafusion/physical-expr/src/window/built_in.rs index 95066a6af382..ce439e2b8cd1 100644 --- a/datafusion/physical-expr/src/window/built_in.rs +++ b/datafusion/physical-expr/src/window/built_in.rs @@ -21,20 +21,19 @@ use std::any::Any; use std::ops::Range; use std::sync::Arc; -use super::BuiltInWindowFunctionExpr; -use super::WindowExpr; +use super::{BuiltInWindowFunctionExpr, WindowExpr}; use crate::expressions::PhysicalSortExpr; use crate::window::window_expr::{get_orderby_values, WindowFn}; use crate::window::{PartitionBatches, PartitionWindowAggStates, WindowState}; use crate::{reverse_order_bys, PhysicalExpr, SchemaProperties}; + use arrow::array::{new_empty_array, ArrayRef}; use arrow::compute::SortOptions; use arrow::datatypes::Field; use arrow::record_batch::RecordBatch; use datafusion_common::utils::evaluate_partition_ranges; use datafusion_common::{Result, ScalarValue}; -use datafusion_expr::window_state::WindowAggState; -use datafusion_expr::window_state::WindowFrameContext; +use datafusion_expr::window_state::{WindowAggState, WindowFrameContext}; use datafusion_expr::WindowFrame; /// A window expr that takes the form of a [`BuiltInWindowFunctionExpr`]. @@ -75,7 +74,7 @@ impl BuiltInWindowExpr { /// by the ordering of `self.expr`. pub fn add_equal_orderings(&self, schema_properties: &mut SchemaProperties) { let schema = schema_properties.schema(); - if let Some(fn_res_ordering) = self.expr.get_result_ordering(&schema) { + if let Some(fn_res_ordering) = self.expr.get_result_ordering(schema) { if self.partition_by.is_empty() { // In the absence of a PARTITION BY, ordering of `self.expr` is global: schema_properties.add_new_orderings(&[vec![fn_res_ordering]]); diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index eba58ed36ff0..500c15278dd8 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -25,7 +25,11 @@ use crate::aggregates::{ no_grouping::AggregateStream, row_hash::GroupedHashAggregateStream, topk_stream::GroupedTopKAggregateStream, }; +use crate::common::calculate_projection_mapping; use crate::metrics::{ExecutionPlanMetricsSet, MetricsSet}; +use crate::windows::{ + get_ordered_partition_by_indices, get_window_mode, PartitionSearchMode, +}; use crate::{ DisplayFormatType, Distribution, ExecutionPlan, Partitioning, SendableRecordBatchStream, Statistics, @@ -40,6 +44,7 @@ use datafusion_execution::TaskContext; use datafusion_expr::Accumulator; use datafusion_physical_expr::{ aggregate::is_order_sensitive, + equivalence::collapse_lex_req, expressions::{Column, Max, Min}, physical_exprs_contains, project_out_expr, reverse_order_bys, AggregateExpr, LexOrdering, LexOrderingReq, PhysicalExpr, PhysicalSortExpr, PhysicalSortRequirement, @@ -55,12 +60,7 @@ mod row_hash; mod topk; mod topk_stream; -use crate::common::calculate_projection_mapping; -use crate::windows::{ - get_ordered_partition_by_indices, get_window_mode, PartitionSearchMode, -}; pub use datafusion_expr::AggregateFunction; -use datafusion_physical_expr::equivalence::collapse_lex_req; pub use datafusion_physical_expr::expressions::create_aggregate_expr; /// Hash aggregate modes @@ -217,18 +217,16 @@ impl PhysicalGroupBy { self.null_expr.is_empty() } - /// Calculate group by expressions according to input schema. + /// Calculate GROUP BY expressions according to input schema. pub fn input_exprs(&self) -> Vec> { self.expr .iter() .map(|(expr, _alias)| expr.clone()) - .collect::>() + .collect() } - /// This function returns grouping expressions as they occur in the output schema. + /// Return grouping expressions as they occur in the output schema. fn output_exprs(&self) -> Vec> { - // Update column indices. Since the group by columns come first in the output schema, their - // indices are simply 0..self.group_expr(len). self.expr .iter() .enumerate() @@ -296,10 +294,11 @@ pub struct AggregateExec { /// We need the input schema of partial aggregate to be able to deserialize aggregate /// expressions from protobuf for final aggregate. pub input_schema: SchemaRef, - /// The source_to_target_mapping used to normalize out expressions like Partitioning and PhysicalSortExpr - /// The key is the expression from the input schema and the value is the expression from the output schema. + /// The mapping used to normalize expressions like Partitioning and + /// PhysicalSortExpr. The key is the expression from the input schema + /// and the value is the expression from the output schema. source_to_target_mapping: Vec<(Arc, Arc)>, - /// Execution Metrics + /// Execution metrics metrics: ExecutionPlanMetricsSet, required_input_ordering: Option, partition_search_mode: PartitionSearchMode, @@ -337,8 +336,8 @@ fn get_finest_requirement( order_by_expr: &mut [Option], schema_properties: &SchemaProperties, ) -> Result> { - // Check at the beginning if all the requirements are satisfied by existing ordering - // If so return None, to indicate all of the requirements are already satisfied. + // First, we check if all the requirements are satisfied by the existing + // ordering. If so, we return `None` to indicate this. let mut all_satisfied = true; for (aggr_expr, fn_req) in aggr_expr.iter_mut().zip(order_by_expr.iter_mut()) { if schema_properties.ordering_satisfy(fn_req.as_deref()) { @@ -347,14 +346,14 @@ fn get_finest_requirement( if let Some(reverse) = aggr_expr.reverse_expr() { let reverse_req = fn_req.as_ref().map(|item| reverse_order_bys(item)); if schema_properties.ordering_satisfy(reverse_req.as_deref()) { - // We need to update `aggr_expr` with its reverse, since only its - // reverse requirement is compatible with existing requirements: + // We need to update `aggr_expr` with its reverse since only its + // reverse requirement is compatible with the existing requirements: *aggr_expr = reverse; *fn_req = reverse_req; continue; } } - // requirement is not satisfied + // Requirement is not satisfied: all_satisfied = false; } if all_satisfied { @@ -363,9 +362,7 @@ fn get_finest_requirement( } let mut finest_req = get_init_req(aggr_expr, order_by_expr); for (aggr_expr, fn_req) in aggr_expr.iter_mut().zip(order_by_expr.iter_mut()) { - let fn_req = if let Some(fn_req) = fn_req { - fn_req - } else { + let Some(fn_req) = fn_req else { continue; }; @@ -520,10 +517,11 @@ impl AggregateExec { &mut ordering_req, )?; - // get group by exprs + // Get GROUP BY expressions: let groupby_exprs = group_by.input_exprs(); - // If existing ordering satisfies a prefix of groupby expression, prefix requirement - // with this section. In this case, group by will work more efficient + // If existing ordering satisfies a prefix of the GROUP BY expressions, + // prefix requirements with this section. In this case, aggregation will + // work more efficiently. let indices = get_ordered_partition_by_indices(&groupby_exprs, &input); let mut new_requirement = indices .into_iter() @@ -541,11 +539,8 @@ impl AggregateExec { let source_to_target_mapping = calculate_projection_mapping(&group_by.expr, &input.schema())?; - let required_input_ordering = if new_requirement.is_empty() { - None - } else { - Some(new_requirement) - }; + let required_input_ordering = + (!new_requirement.is_empty()).then_some(new_requirement); let aggregate_oeq = input .schema_properties() @@ -1137,6 +1132,10 @@ pub(crate) fn evaluate_group_by( #[cfg(test)] mod tests { + use std::any::Any; + use std::sync::Arc; + use std::task::{Context, Poll}; + use super::*; use crate::aggregates::{ get_finest_requirement, AggregateExec, AggregateMode, PhysicalGroupBy, @@ -1161,6 +1160,7 @@ mod tests { assert_batches_eq, assert_batches_sorted_eq, internal_err, DataFusionError, Result, ScalarValue, }; + use datafusion_execution::config::SessionConfig; use datafusion_execution::runtime_env::{RuntimeConfig, RuntimeEnv}; use datafusion_physical_expr::expressions::{ lit, ApproxDistinct, Count, FirstValue, LastValue, Median, @@ -1169,11 +1169,6 @@ mod tests { AggregateExpr, PhysicalExpr, PhysicalSortExpr, SchemaProperties, }; - use std::any::Any; - use std::sync::Arc; - use std::task::{Context, Poll}; - - use datafusion_execution::config::SessionConfig; use futures::{FutureExt, Stream}; // Generate a schema which consists of 5 columns (a, b, c, d, e) diff --git a/datafusion/physical-plan/src/aggregates/order/mod.rs b/datafusion/physical-plan/src/aggregates/order/mod.rs index d905a3118af6..f72d2f06e459 100644 --- a/datafusion/physical-plan/src/aggregates/order/mod.rs +++ b/datafusion/physical-plan/src/aggregates/order/mod.rs @@ -45,15 +45,16 @@ impl GroupOrdering { mode: &PartitionSearchMode, ordering: &[PhysicalSortExpr], ) -> Result { - Ok(match mode { - PartitionSearchMode::Linear => GroupOrdering::None, + match mode { + PartitionSearchMode::Linear => Ok(GroupOrdering::None), PartitionSearchMode::PartiallySorted(order_indices) => { - let partial = - GroupOrderingPartial::try_new(input_schema, order_indices, ordering)?; - GroupOrdering::Partial(partial) + GroupOrderingPartial::try_new(input_schema, order_indices, ordering) + .map(GroupOrdering::Partial) } - PartitionSearchMode::Sorted => GroupOrdering::Full(GroupOrderingFull::new()), - }) + PartitionSearchMode::Sorted => { + Ok(GroupOrdering::Full(GroupOrderingFull::new())) + } + } } // How many groups be emitted, or None if no data can be emitted diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index 9739d95f7658..37b0f73a581c 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -17,17 +17,10 @@ //! Hash aggregation -use datafusion_physical_expr::{ - AggregateExpr, EmitTo, GroupsAccumulator, GroupsAccumulatorAdapter, PhysicalSortExpr, -}; -use log::debug; use std::sync::Arc; use std::task::{Context, Poll}; use std::vec; -use futures::ready; -use futures::stream::{Stream, StreamExt}; - use crate::aggregates::group_values::{new_group_values, GroupValues}; use crate::aggregates::order::GroupOrderingFull; use crate::aggregates::{ @@ -41,6 +34,7 @@ use crate::sorts::streaming_merge; use crate::stream::RecordBatchStreamAdapter; use crate::{aggregates, ExecutionPlan, PhysicalExpr}; use crate::{RecordBatchStream, SendableRecordBatchStream}; + use arrow::array::*; use arrow::{datatypes::SchemaRef, record_batch::RecordBatch}; use arrow_schema::SortOptions; @@ -50,7 +44,14 @@ use datafusion_execution::memory_pool::proxy::VecAllocExt; use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; use datafusion_execution::runtime_env::RuntimeEnv; use datafusion_execution::TaskContext; -use datafusion_physical_expr::expressions::col; +use datafusion_physical_expr::expressions::Column; +use datafusion_physical_expr::{ + AggregateExpr, EmitTo, GroupsAccumulator, GroupsAccumulatorAdapter, PhysicalSortExpr, +}; + +use futures::ready; +use futures::stream::{Stream, StreamExt}; +use log::debug; #[derive(Debug, Clone)] /// This object tracks the aggregation phase (input/output) @@ -321,8 +322,9 @@ impl GroupedHashAggregateStream { let spill_expr = group_schema .fields .into_iter() - .map(|field| PhysicalSortExpr { - expr: col(field.name(), &group_schema).unwrap(), + .enumerate() + .map(|(idx, field)| PhysicalSortExpr { + expr: Arc::new(Column::new(field.name().as_str(), idx)) as _, options: SortOptions::default(), }) .collect(); @@ -415,8 +417,7 @@ impl Stream for GroupedHashAggregateStream { let elapsed_compute = self.baseline_metrics.elapsed_compute().clone(); loop { - let exec_state = self.exec_state.clone(); - match exec_state { + match &self.exec_state { ExecutionState::ReadingInput => { match ready!(self.input.poll_next_unpin(cx)) { // new batch to aggregate @@ -450,14 +451,14 @@ impl Stream for GroupedHashAggregateStream { self.input_done = true; self.group_ordering.input_done(); let timer = elapsed_compute.timer(); - if self.spill_state.spills.is_empty() { + self.exec_state = if self.spill_state.spills.is_empty() { let batch = extract_ok!(self.emit(EmitTo::All, false)); - self.exec_state = ExecutionState::ProducingOutput(batch); + ExecutionState::ProducingOutput(batch) } else { // If spill files exist, stream-merge them. extract_ok!(self.update_merged_stream()); - self.exec_state = ExecutionState::ReadingInput; - } + ExecutionState::ReadingInput + }; timer.done(); } } @@ -465,19 +466,24 @@ impl Stream for GroupedHashAggregateStream { ExecutionState::ProducingOutput(batch) => { // slice off a part of the batch, if needed - let output_batch = if batch.num_rows() <= self.batch_size { - if self.input_done { - self.exec_state = ExecutionState::Done; - } else { - self.exec_state = ExecutionState::ReadingInput - } - batch + let output_batch; + let size = self.batch_size; + (self.exec_state, output_batch) = if batch.num_rows() <= size { + ( + if self.input_done { + ExecutionState::Done + } else { + ExecutionState::ReadingInput + }, + batch.clone(), + ) } else { // output first batch_size rows - let num_remaining = batch.num_rows() - self.batch_size; - let remaining = batch.slice(self.batch_size, num_remaining); - self.exec_state = ExecutionState::ProducingOutput(remaining); - batch.slice(0, self.batch_size) + let size = self.batch_size; + let num_remaining = batch.num_rows() - size; + let remaining = batch.slice(size, num_remaining); + let output = batch.slice(0, size); + (ExecutionState::ProducingOutput(remaining), output) }; return Poll::Ready(Some(Ok( output_batch.record_output(&self.baseline_metrics) diff --git a/datafusion/physical-plan/src/coalesce_batches.rs b/datafusion/physical-plan/src/coalesce_batches.rs index 63ac6d7a88ea..db11da77aa69 100644 --- a/datafusion/physical-plan/src/coalesce_batches.rs +++ b/datafusion/physical-plan/src/coalesce_batches.rs @@ -224,17 +224,17 @@ impl CoalesceBatchesStream { let _timer = cloned_time.timer(); match input_batch { Poll::Ready(x) => match x { - Some(Ok(ref batch)) => { + Some(Ok(batch)) => { if batch.num_rows() >= self.target_batch_size && self.buffer.is_empty() { - return Poll::Ready(Some(Ok(batch.clone()))); + return Poll::Ready(Some(Ok(batch))); } else if batch.num_rows() == 0 { // discard empty batches } else { // add to the buffered batches - self.buffer.push(batch.clone()); self.buffered_rows += batch.num_rows(); + self.buffer.push(batch); // check to see if we have enough batches yet if self.buffered_rows >= self.target_batch_size { // combine the batches and return @@ -296,14 +296,14 @@ pub fn concat_batches( batches.len(), row_count ); - let b = arrow::compute::concat_batches(schema, batches)?; - Ok(b) + arrow::compute::concat_batches(schema, batches) } #[cfg(test)] mod tests { use super::*; use crate::{memory::MemoryExec, repartition::RepartitionExec}; + use arrow::datatypes::{DataType, Field, Schema}; use arrow_array::UInt32Array; diff --git a/datafusion/physical-plan/src/common.rs b/datafusion/physical-plan/src/common.rs index 8359639954c3..823fdb71c8d1 100644 --- a/datafusion/physical-plan/src/common.rs +++ b/datafusion/physical-plan/src/common.rs @@ -31,13 +31,13 @@ use arrow::datatypes::Schema; use arrow::ipc::writer::{FileWriter, IpcWriteOptions}; use arrow::record_batch::RecordBatch; use datafusion_common::stats::Precision; +use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::{plan_err, DataFusionError, Result}; use datafusion_execution::memory_pool::MemoryReservation; +use datafusion_physical_expr::equivalence::ProjectionMapping; use datafusion_physical_expr::expressions::{BinaryExpr, Column}; use datafusion_physical_expr::{PhysicalExpr, PhysicalSortExpr}; -use datafusion_common::tree_node::{Transformed, TreeNode}; -use datafusion_physical_expr::equivalence::ProjectionMapping; use futures::{Future, StreamExt, TryStreamExt}; use parking_lot::Mutex; use pin_project_lite::pin_project; @@ -375,15 +375,15 @@ pub fn batch_byte_size(batch: &RecordBatch) -> usize { batch.get_array_memory_size() } -/// Constructs projection mapping between input and output +/// Constructs the mapping between a projection's input and output pub fn calculate_projection_mapping( expr: &[(Arc, String)], input_schema: &Arc, ) -> Result { - // construct a map from the input expressions to the output expression of the Projection + // Construct a map from the input expressions to the output expression of the projection: let mut source_to_target_mapping = vec![]; for (expr_idx, (expression, name)) in expr.iter().enumerate() { - let target_expr = Arc::new(Column::new(name, expr_idx)) as Arc; + let target_expr = Arc::new(Column::new(name, expr_idx)) as _; let source_expr = expression.clone().transform_down(&|e| match e .as_any() diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index b0d6fb107b25..2325775ec995 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -42,14 +42,13 @@ use datafusion_common::{plan_err, DataFusionError, Result}; use datafusion_execution::TaskContext; use datafusion_expr::Operator; use datafusion_physical_expr::expressions::BinaryExpr; +use datafusion_physical_expr::intervals::utils::check_support; +use datafusion_physical_expr::utils::collect_columns; use datafusion_physical_expr::{ analyze, split_conjunction, AnalysisContext, ExprBoundaries, PhysicalExpr, SchemaProperties, }; -use datafusion_physical_expr::intervals::utils::check_support; -use datafusion_physical_expr::utils::collect_columns; - use futures::stream::{Stream, StreamExt}; use log::trace; @@ -149,7 +148,7 @@ impl ExecutionPlan for FilterExec { let stats = self.statistics().unwrap(); // Combine the equal predicates with the input equivalence properties let mut filter_oeq = self.input.schema_properties(); - let (equal_pairs, _ne_pairs) = collect_columns_from_predicate(&self.predicate); + let (equal_pairs, _) = collect_columns_from_predicate(&self.predicate); for (lhs, rhs) in equal_pairs { let lhs_expr = Arc::new(lhs.clone()) as _; let rhs_expr = Arc::new(rhs.clone()) as _; @@ -159,8 +158,8 @@ impl ExecutionPlan for FilterExec { let constants = collect_columns(self.predicate()) .into_iter() .filter(|column| stats.column_statistics[column.index()].is_singleton()) - .map(|column| Arc::new(column) as Arc) - .collect::>(); + .map(|column| Arc::new(column) as _) + .collect(); filter_oeq.with_constants(constants) } @@ -168,10 +167,8 @@ impl ExecutionPlan for FilterExec { self: Arc, children: Vec>, ) -> Result> { - Ok(Arc::new(FilterExec::try_new( - self.predicate.clone(), - children[0].clone(), - )?)) + FilterExec::try_new(self.predicate.clone(), children[0].clone()) + .map(|e| Arc::new(e) as _) } fn execute( @@ -350,17 +347,16 @@ impl RecordBatchStream for FilterExecStream { /// Return the equals Column-Pairs and Non-equals Column-Pairs fn collect_columns_from_predicate(predicate: &Arc) -> EqualAndNonEqual { - let mut eq_predicate_columns: Vec<(&Column, &Column)> = Vec::new(); - let mut ne_predicate_columns: Vec<(&Column, &Column)> = Vec::new(); + let mut eq_predicate_columns = Vec::<(&Column, &Column)>::new(); + let mut ne_predicate_columns = Vec::<(&Column, &Column)>::new(); let predicates = split_conjunction(predicate); predicates.into_iter().for_each(|p| { if let Some(binary) = p.as_any().downcast_ref::() { - let left = binary.left(); - let right = binary.right(); - if left.as_any().is::() && right.as_any().is::() { - let left_column = left.as_any().downcast_ref::().unwrap(); - let right_column = right.as_any().downcast_ref::().unwrap(); + if let (Some(left_column), Some(right_column)) = ( + binary.left().as_any().downcast_ref::(), + binary.right().as_any().downcast_ref::(), + ) { match binary.op() { Operator::Eq => { eq_predicate_columns.push((left_column, right_column)) diff --git a/datafusion/physical-plan/src/joins/cross_join.rs b/datafusion/physical-plan/src/joins/cross_join.rs index dc404c6aaa9c..d4fd4895678f 100644 --- a/datafusion/physical-plan/src/joins/cross_join.rs +++ b/datafusion/physical-plan/src/joins/cross_join.rs @@ -41,8 +41,7 @@ use datafusion_physical_expr::equivalence::join_schema_properties; use datafusion_physical_expr::SchemaProperties; use async_trait::async_trait; -use futures::{ready, StreamExt}; -use futures::{Stream, TryStreamExt}; +use futures::{ready, Stream, StreamExt, TryStreamExt}; /// Data of the left side type JoinLeftData = (RecordBatch, MemoryReservation); @@ -105,12 +104,11 @@ async fn load_left_input( reservation: MemoryReservation, ) -> Result { // merge all left parts into a single stream - let merge = { - if left.output_partitioning().partition_count() != 1 { - Arc::new(CoalescePartitionsExec::new(left.clone())) - } else { - left.clone() - } + let left_schema = left.schema(); + let merge = if left.output_partitioning().partition_count() != 1 { + Arc::new(CoalescePartitionsExec::new(left)) + } else { + left }; let stream = merge.execute(0, context)?; @@ -135,7 +133,7 @@ async fn load_left_input( ) .await?; - let merged_batch = concat_batches(&left.schema(), &batches, num_rows)?; + let merged_batch = concat_batches(&left_schema, &batches, num_rows)?; Ok((merged_batch, reservation)) } @@ -227,7 +225,6 @@ impl ExecutionPlan for CrossJoinExec { None, &[], ) - .unwrap() } fn execute( diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index cf15b4f394dc..4774a2bf2521 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -56,6 +56,7 @@ use arrow::array::{ Array, ArrayRef, BooleanArray, BooleanBufferBuilder, PrimitiveArray, UInt32Array, UInt32BufferBuilder, UInt64Array, UInt64BufferBuilder, }; +use arrow::compute::kernels::cmp::{eq, not_distinct}; use arrow::compute::{and, take, FilterBuilder}; use arrow::datatypes::{Schema, SchemaRef}; use arrow::record_batch::RecordBatch; @@ -67,11 +68,10 @@ use datafusion_common::{ }; use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; use datafusion_execution::TaskContext; +use datafusion_physical_expr::equivalence::join_schema_properties; use datafusion_physical_expr::SchemaProperties; use ahash::RandomState; -use arrow::compute::kernels::cmp::{eq, not_distinct}; -use datafusion_physical_expr::equivalence::join_schema_properties; use futures::{ready, Stream, StreamExt, TryStreamExt}; type JoinLeftData = (JoinHashMap, RecordBatch, MemoryReservation); @@ -377,7 +377,6 @@ impl ExecutionPlan for HashJoinExec { Some(Self::probe_side()), self.on(), ) - .unwrap() } fn children(&self) -> Vec> { @@ -511,16 +510,10 @@ async fn collect_left_input( let (left_input, left_input_partition) = if let Some(partition) = partition { (left, partition) + } else if left.output_partitioning().partition_count() != 1 { + (Arc::new(CoalescePartitionsExec::new(left)) as _, 0) } else { - let merge = { - if left.output_partitioning().partition_count() != 1 { - Arc::new(CoalescePartitionsExec::new(left)) - } else { - left - } - }; - - (merge, 0) + (left, 0) }; // Depending on partition argument load single partition or whole left side in memory @@ -1051,24 +1044,22 @@ impl Stream for HashJoinStream { mod tests { use std::sync::Arc; - use arrow::array::{ArrayRef, Date32Array, Int32Array, UInt32Builder, UInt64Builder}; - use arrow::datatypes::{DataType, Field, Schema}; - - use datafusion_common::{assert_batches_sorted_eq, assert_contains, ScalarValue}; - use datafusion_expr::Operator; - use datafusion_physical_expr::expressions::Literal; - use hashbrown::raw::RawTable; - + use super::*; use crate::{ common, expressions::Column, hash_utils::create_hashes, joins::hash_join::build_equal_condition_join_indices, memory::MemoryExec, repartition::RepartitionExec, test::build_table_i32, test::exec::MockExec, }; + + use arrow::array::{ArrayRef, Date32Array, Int32Array, UInt32Builder, UInt64Builder}; + use arrow::datatypes::{DataType, Field, Schema}; + use datafusion_common::{assert_batches_sorted_eq, assert_contains, ScalarValue}; use datafusion_execution::config::SessionConfig; use datafusion_execution::runtime_env::{RuntimeConfig, RuntimeEnv}; - use datafusion_physical_expr::expressions::BinaryExpr; + use datafusion_expr::Operator; + use datafusion_physical_expr::expressions::{BinaryExpr, Literal}; - use super::*; + use hashbrown::raw::RawTable; fn build_table( a: (&str, &Vec), diff --git a/datafusion/physical-plan/src/joins/sort_merge_join.rs b/datafusion/physical-plan/src/joins/sort_merge_join.rs index 20da122a1003..3c0adedbb46b 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join.rs @@ -51,9 +51,9 @@ use datafusion_common::{ }; use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; use datafusion_execution::TaskContext; +use datafusion_physical_expr::equivalence::join_schema_properties; use datafusion_physical_expr::{PhysicalSortRequirement, SchemaProperties}; -use datafusion_physical_expr::equivalence::join_schema_properties; use futures::{Stream, StreamExt}; /// join execution plan executes partitions in parallel and combines them into a set of @@ -293,7 +293,6 @@ impl ExecutionPlan for SortMergeJoinExec { Some(Self::probe_side(&self.join_type)), self.on(), ) - .unwrap() } fn children(&self) -> Vec> { diff --git a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs index 60b39af92c24..da50ff8ff6ff 100644 --- a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs +++ b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs @@ -63,10 +63,10 @@ use datafusion_common::{ }; use datafusion_execution::memory_pool::MemoryConsumer; use datafusion_execution::TaskContext; +use datafusion_physical_expr::equivalence::join_schema_properties; use datafusion_physical_expr::intervals::ExprIntervalGraph; use ahash::RandomState; -use datafusion_physical_expr::equivalence::join_schema_properties; use futures::stream::{select, BoxStream}; use futures::{Stream, StreamExt}; use hashbrown::HashSet; @@ -442,7 +442,6 @@ impl ExecutionPlan for SymmetricHashJoinExec { None, self.on(), ) - .unwrap() } fn children(&self) -> Vec> { diff --git a/datafusion/physical-plan/src/joins/utils.rs b/datafusion/physical-plan/src/joins/utils.rs index eafe26d92658..910d582ad826 100644 --- a/datafusion/physical-plan/src/joins/utils.rs +++ b/datafusion/physical-plan/src/joins/utils.rs @@ -36,11 +36,11 @@ use arrow::datatypes::{Field, Schema, SchemaBuilder}; use arrow::record_batch::{RecordBatch, RecordBatchOptions}; use datafusion_common::cast::as_boolean_array; use datafusion_common::stats::Precision; -use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::{ exec_err, plan_datafusion_err, plan_err, DataFusionError, JoinSide, JoinType, Result, SharedResult, }; +use datafusion_physical_expr::equivalence::add_offset_to_expr; use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::intervals::{ExprIntervalGraph, Interval, IntervalBound}; use datafusion_physical_expr::utils::merge_vectors; @@ -92,8 +92,8 @@ fn check_join_set_is_valid( if !left_missing.is_empty() | !right_missing.is_empty() { return plan_err!( - "The left or right side of the join does not have all columns on \"on\": \nMissing on the left: {left_missing:?}\nMissing on the right: {right_missing:?}" - ); + "The left or right side of the join does not have all columns on \"on\": \nMissing on the left: {left_missing:?}\nMissing on the right: {right_missing:?}" + ); }; Ok(()) @@ -133,17 +133,8 @@ pub fn adjust_right_output_partitioning( Partitioning::Hash(exprs, size) => { let new_exprs = exprs .into_iter() - .map(|expr| { - expr.transform_down(&|e| match e.as_any().downcast_ref::() { - Some(col) => Ok(Transformed::Yes(Arc::new(Column::new( - col.name(), - left_columns_len + col.index(), - )))), - None => Ok(Transformed::No(e)), - }) - .unwrap() - }) - .collect::>(); + .map(|expr| add_offset_to_expr(expr, left_columns_len)) + .collect(); Partitioning::Hash(new_exprs, size) } } @@ -187,7 +178,7 @@ pub fn calculate_join_output_ordering( // In the case below, right ordering should be offseted with the left // side length, since we append the right table to the left table. JoinType::Inner | JoinType::Left | JoinType::Right | JoinType::Full => { - add_offset_to_lex_ordering(right_ordering, left_columns_len)? + add_offset_to_lex_ordering(right_ordering, left_columns_len) } _ => right_ordering.to_vec(), }; diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index bdd1f8abb9a3..49c19c256442 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -29,6 +29,7 @@ use std::task::{Context, Poll}; use super::expressions::{Column, PhysicalSortExpr}; use super::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; use super::{DisplayAs, RecordBatchStream, SendableRecordBatchStream, Statistics}; +use crate::common::calculate_projection_mapping; use crate::{ ColumnStatistics, DisplayFormatType, ExecutionPlan, Partitioning, PhysicalExpr, }; @@ -39,9 +40,8 @@ use datafusion_common::stats::Precision; use datafusion_common::Result; use datafusion_execution::TaskContext; use datafusion_physical_expr::expressions::Literal; - -use crate::common::calculate_projection_mapping; use datafusion_physical_expr::{project_out_expr, SchemaProperties}; + use futures::stream::{Stream, StreamExt}; use log::trace; @@ -56,8 +56,9 @@ pub struct ProjectionExec { input: Arc, /// The output ordering output_ordering: Option>, - /// The source_to_target_mapping used to normalize out expressions like Partitioning and PhysicalSortExpr - /// The key is the expression from the input schema and the value is the expression from the output schema + /// The mapping used to normalize expressions like Partitioning and + /// PhysicalSortExpr. The key is the expression from the input schema + /// and the value is the expression from the output schema. source_to_target_mapping: Vec<(Arc, Arc)>, /// Execution metrics metrics: ExecutionPlanMetricsSet, @@ -175,16 +176,14 @@ impl ExecutionPlan for ProjectionExec { fn output_partitioning(&self) -> Partitioning { // Output partition need to respect the alias let input_partition = self.input.output_partitioning(); - match input_partition { - Partitioning::Hash(exprs, part) => { - let normalized_exprs = exprs - .into_iter() - .map(|expr| project_out_expr(expr, &self.source_to_target_mapping)) - .collect::>(); - - Partitioning::Hash(normalized_exprs, part) - } - _ => input_partition, + if let Partitioning::Hash(exprs, part) = input_partition { + let normalized_exprs = exprs + .into_iter() + .map(|expr| project_out_expr(expr, &self.source_to_target_mapping)) + .collect(); + Partitioning::Hash(normalized_exprs, part) + } else { + input_partition } } @@ -205,12 +204,10 @@ impl ExecutionPlan for ProjectionExec { fn with_new_children( self: Arc, - children: Vec>, + mut children: Vec>, ) -> Result> { - Ok(Arc::new(ProjectionExec::try_new( - self.expr.clone(), - children[0].clone(), - )?)) + ProjectionExec::try_new(self.expr.clone(), children.swap_remove(0)) + .map(|p| Arc::new(p) as _) } fn benefits_from_input_partitioning(&self) -> Vec { @@ -366,6 +363,7 @@ mod tests { use crate::common::collect; use crate::expressions; use crate::test; + use arrow_schema::DataType; use datafusion_common::ScalarValue; diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index d763c153f3b6..a0bb614f3609 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -425,12 +425,11 @@ impl ExecutionPlan for RepartitionExec { fn with_new_children( self: Arc, - children: Vec>, + mut children: Vec>, ) -> Result> { let repartition = - RepartitionExec::try_new(children[0].clone(), self.partitioning.clone())? - .with_preserve_order(self.preserve_order); - Ok(Arc::new(repartition)) + RepartitionExec::try_new(children.swap_remove(0), self.partitioning.clone()); + repartition.map(|r| Arc::new(r.with_preserve_order(self.preserve_order)) as _) } /// Specifies whether this plan generates an infinite stream of records. diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index cd1911debd15..6e6e57023c3e 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -265,6 +265,8 @@ impl ExecutionPlan for SortPreservingMergeExec { #[cfg(test)] mod tests { + use std::iter::FromIterator; + use super::*; use crate::coalesce_partitions::CoalescePartitionsExec; use crate::expressions::col; @@ -275,8 +277,8 @@ mod tests { use crate::test::exec::{assert_strong_count_converges_to_zero, BlockingExec}; use crate::test::{self, assert_is_pending, make_partition}; use crate::{collect, common}; - use arrow::array::ArrayRef; - use arrow::array::{Int32Array, StringArray, TimestampNanosecondArray}; + + use arrow::array::{ArrayRef, Int32Array, StringArray, TimestampNanosecondArray}; use arrow::compute::SortOptions; use arrow::datatypes::{DataType, Field, Schema}; use arrow::record_batch::RecordBatch; @@ -284,7 +286,6 @@ mod tests { use datafusion_execution::config::SessionConfig; use futures::{FutureExt, StreamExt}; - use std::iter::FromIterator; #[tokio::test] async fn test_merge_interleave() { diff --git a/datafusion/physical-plan/src/union.rs b/datafusion/physical-plan/src/union.rs index 5b6f7e72a7cf..6a236ff242c0 100644 --- a/datafusion/physical-plan/src/union.rs +++ b/datafusion/physical-plan/src/union.rs @@ -40,8 +40,8 @@ use arrow::record_batch::RecordBatch; use datafusion_common::stats::Precision; use datafusion_common::{exec_err, internal_err, DFSchemaRef, DataFusionError, Result}; use datafusion_execution::TaskContext; - use datafusion_physical_expr::SchemaProperties; + use futures::Stream; use itertools::Itertools; use log::{debug, trace, warn}; @@ -225,7 +225,7 @@ impl ExecutionPlan for UnionExec { fn schema_properties(&self) -> SchemaProperties { // TODO: In some cases equivalent groups and constants - // can be preserved in union. Add support for these. + // can be preserved in union. Add support for these. let child_oeqs = self .inputs .iter() @@ -233,27 +233,26 @@ impl ExecutionPlan for UnionExec { .collect::>(); let mut union_oeq = SchemaProperties::new(self.schema()); // Iterate ordering equivalent group of first child - for elem in child_oeqs[0].oeq_group().iter() { + for item in child_oeqs[0].oeq_group().iter() { // Seed for the meet. - let mut meet = Some(elem.clone()); - child_oeqs.iter().for_each(|child_oeq| { + let mut meet = Some(item.clone()); + for child_oeq in child_oeqs.iter() { if let Some(meet_vec) = &meet { - let res = child_oeq + let result = child_oeq .oeq_group() .iter() .filter_map(|ordering| { child_oeq.get_meet_ordering(ordering, meet_vec) }) - .collect::>(); - if let Some(new_meet) = res.first() { - meet = Some(new_meet.to_vec()); + .next(); + if let Some(new_meet) = result { + meet.replace(new_meet); } else { - // If none of the child doesn't have a meet - // There is no meet. + // If none of the children has a meet, there is no meet. meet = None; } } - }); + } // All of the children have a common meet ordering. // This ordering can be propagated in Union. if let Some(meet) = meet { @@ -637,9 +636,9 @@ fn stats_union(mut left: Statistics, right: Statistics) -> Statistics { mod tests { use super::*; use crate::collect; + use crate::memory::MemoryExec; use crate::test; - use crate::memory::MemoryExec; use arrow::record_batch::RecordBatch; use arrow_schema::{DataType, SortOptions}; use datafusion_common::ScalarValue; diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index bc6950338c86..1bb06d1957d1 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -37,6 +37,7 @@ use datafusion_expr::{ window_function::{BuiltInWindowFunction, WindowFunction}, PartitionEvaluator, WindowFrame, WindowUDF, }; +use datafusion_physical_expr::equivalence::collapse_lex_req; use datafusion_physical_expr::{ reverse_order_bys, window::{BuiltInWindowFunctionExpr, SlidingAggregateWindowExpr}, @@ -47,7 +48,6 @@ mod bounded_window_agg_exec; mod window_agg_exec; pub use bounded_window_agg_exec::BoundedWindowAggExec; -use datafusion_physical_expr::equivalence::collapse_lex_req; pub use window_agg_exec::WindowAggExec; pub use datafusion_physical_expr::window::{ @@ -343,15 +343,14 @@ pub(crate) fn get_partition_by_sort_exprs( .collect::>(); // Make sure ordered section doesn't move over the partition by expression assert!(ordered_partition_by_indices.len() <= partition_by_exprs.len()); - let partition_by_sort_exprs = input + input .schema_properties() .get_lex_ordering(&ordered_partition_exprs) .ok_or_else(|| { DataFusionError::Execution( "Expects partition by expression to be ordered".to_string(), ) - })?; - Ok(partition_by_sort_exprs) + }) } pub(crate) fn window_ordering_equivalence( @@ -472,14 +471,14 @@ pub fn get_window_mode( if partitionby_exprs.is_empty() { partition_search_mode = PartitionSearchMode::Sorted; } else if let Some(indices) = input_oeq.set_satisfy(partitionby_exprs) { - let elem = indices + let item = indices .iter() .map(|&idx| PhysicalSortRequirement { expr: partitionby_exprs[idx].clone(), options: None, }) .collect::>(); - partition_by_reqs.extend(elem); + partition_by_reqs.extend(item); if indices.len() == partitionby_exprs.len() { partition_search_mode = PartitionSearchMode::Sorted; } else if !indices.is_empty() { @@ -498,8 +497,7 @@ pub fn get_window_mode( let req = [partition_by_reqs.clone(), order_by_reqs].concat(); let req = collapse_lex_req(req); if req.is_empty() { - // When requirement is empty, - // prefer None. Instead of Linear. + // When requirement is empty, prefer None instead of Linear. return Ok(None); } else if partition_by_oeq.ordering_satisfy_requirement_concrete(&req) { // Window can be run with existing ordering @@ -524,6 +522,7 @@ mod tests { use datafusion_execution::TaskContext; use futures::FutureExt; + use PartitionSearchMode::{Linear, PartiallySorted, Sorted}; fn create_test_schema() -> Result { @@ -1020,10 +1019,6 @@ mod tests { // ORDER BY b, a ASC NULLS FIRST (vec![], vec![("b", false, true), ("a", false, true)], None), ]; - // let test_cases = vec![ - // // PARTITION BY a, ORDER BY b ASC NULLS LAST - // (vec!["a"], vec![("b", false, false)], Some((false, Sorted))), - // ]; for (case_idx, test_case) in test_cases.iter().enumerate() { let (partition_by_columns, order_by_params, expected) = &test_case; let mut partition_by_exprs = vec![]; From 94626ee3cacfc861a1aa001aa9ffc37ce8c6e02f Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 20 Oct 2023 14:40:17 +0300 Subject: [PATCH 075/122] Change union meet implementation --- datafusion/physical-expr/src/equivalence.rs | 2 +- datafusion/physical-plan/src/union.rs | 210 +++++++++++--------- 2 files changed, 122 insertions(+), 90 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 5e0b80be365d..a3aa3f504036 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -479,7 +479,7 @@ impl OrderingEquivalentGroup { } /// Get length of the entries in the ordering equivalent group - fn len(&self) -> usize { + pub fn len(&self) -> usize { self.inner.len() } diff --git a/datafusion/physical-plan/src/union.rs b/datafusion/physical-plan/src/union.rs index 5b6f7e72a7cf..d29a0d47e4f9 100644 --- a/datafusion/physical-plan/src/union.rs +++ b/datafusion/physical-plan/src/union.rs @@ -232,34 +232,29 @@ impl ExecutionPlan for UnionExec { .map(|child| child.schema_properties()) .collect::>(); let mut union_oeq = SchemaProperties::new(self.schema()); + let mut existing_meets = child_oeqs[0] + .oeq_group() + .iter() + .map(|elem| elem.to_vec()) + .collect::>(); // Iterate ordering equivalent group of first child - for elem in child_oeqs[0].oeq_group().iter() { - // Seed for the meet. - let mut meet = Some(elem.clone()); - child_oeqs.iter().for_each(|child_oeq| { - if let Some(meet_vec) = &meet { - let res = child_oeq - .oeq_group() - .iter() - .filter_map(|ordering| { - child_oeq.get_meet_ordering(ordering, meet_vec) - }) - .collect::>(); - if let Some(new_meet) = res.first() { - meet = Some(new_meet.to_vec()); - } else { - // If none of the child doesn't have a meet - // There is no meet. - meet = None; - } - } - }); - // All of the children have a common meet ordering. - // This ordering can be propagated in Union. - if let Some(meet) = meet { - union_oeq.add_new_orderings(&[meet]) + for next_child_oeq in child_oeqs.iter().skip(1) { + let mut next_meets = vec![]; + for existing_meet in &existing_meets { + let new_meets = next_child_oeq + .oeq_group() + .iter() + .filter_map(|ordering| { + next_child_oeq.get_meet_ordering(ordering, existing_meet) + }) + .collect::>(); + next_meets.extend(new_meets); } + existing_meets = next_meets; } + // existing_meets contains the all of the valid orderings after union + union_oeq.add_new_orderings(&existing_meets); + union_oeq } @@ -644,6 +639,7 @@ mod tests { use arrow_schema::{DataType, SortOptions}; use datafusion_common::ScalarValue; use datafusion_physical_expr::expressions::col; + use datafusion_physical_expr::PhysicalExpr; // Generate a schema which consists of 7 columns (a, b, c, d, e, f, g) fn create_test_schema() -> Result { @@ -659,6 +655,19 @@ mod tests { Ok(schema) } + // Convert each tuple to PhysicalSortExpr + fn convert_to_sort_exprs( + in_data: &[(&Arc, SortOptions)], + ) -> Vec { + in_data + .iter() + .map(|(expr, options)| PhysicalSortExpr { + expr: (*expr).clone(), + options: *options, + }) + .collect::>() + } + #[tokio::test] async fn test_union_partitions() -> Result<()> { let task_ctx = Arc::new(TaskContext::default()); @@ -781,72 +790,95 @@ mod tests { let col_e = &col("e", &schema)?; let col_f = &col("f", &schema)?; let options = SortOptions::default(); - // [a ASC, b ASC, f ASC], [d ASC] - let orderings = vec![ - vec![ - PhysicalSortExpr { - expr: col_a.clone(), - options, - }, - PhysicalSortExpr { - expr: col_b.clone(), - options, - }, - PhysicalSortExpr { - expr: col_f.clone(), - options, - }, - ], - vec![PhysicalSortExpr { - expr: col_d.clone(), - options, - }], + let test_cases = vec![ + //-----------TEST CASE 1----------// + ( + // First child orderings + vec![ + // [a ASC, b ASC, f ASC] + vec![(col_a, options), (col_b, options), (col_f, options)], + ], + // Second child orderings + vec![ + // [a ASC, b ASC, c ASC] + vec![(col_a, options), (col_b, options), (col_c, options)], + // [a ASC, b ASC, f ASC] + vec![(col_a, options), (col_b, options), (col_f, options)], + ], + // Union output orderings + vec![ + // [a ASC, b ASC, f ASC] + vec![(col_a, options), (col_b, options), (col_f, options)], + ], + ), + //-----------TEST CASE 2----------// + ( + // First child orderings + vec![ + // [a ASC, b ASC, f ASC] + vec![(col_a, options), (col_b, options), (col_f, options)], + // d ASC + vec![(col_d, options)], + ], + // Second child orderings + vec![ + // [a ASC, b ASC, c ASC] + vec![(col_a, options), (col_b, options), (col_c, options)], + // [e ASC] + vec![(col_e, options)], + ], + // Union output orderings + vec![ + // [a ASC, b ASC] + vec![(col_a, options), (col_b, options)], + ], + ), ]; - let child1 = Arc::new( - MemoryExec::try_new(&[], schema.clone(), None)? - .with_sort_information(orderings), - ); - - // [a ASC, b ASC, c ASC], [e ASC] - let orderings = vec![ - vec![ - PhysicalSortExpr { - expr: col_a.clone(), - options, - }, - PhysicalSortExpr { - expr: col_b.clone(), - options, - }, - PhysicalSortExpr { - expr: col_c.clone(), - options, - }, - ], - vec![PhysicalSortExpr { - expr: col_e.clone(), - options, - }], - ]; - let child2 = Arc::new( - MemoryExec::try_new(&[], schema, None)?.with_sort_information(orderings), - ); - - let union = UnionExec::new(vec![child1, child2]); - // Expects union to have [a ASC, b ASC] (e.g meet of inout orderings) - let union_schema_properties = union.schema_properties(); - let union_orderings = union_schema_properties.oeq_group(); - println!("union_orderings:{:?}", union_orderings); - assert!(union_orderings.contains(&vec![ - PhysicalSortExpr { - expr: col_a.clone(), - options - }, - PhysicalSortExpr { - expr: col_b.clone(), - options + + for ( + test_idx, + (first_child_orderings, second_child_orderings, union_orderings), + ) in test_cases.iter().enumerate() + { + let first_orderings = first_child_orderings + .iter() + .map(|ordering| convert_to_sort_exprs(ordering)) + .collect::>(); + let second_orderings = second_child_orderings + .iter() + .map(|ordering| convert_to_sort_exprs(ordering)) + .collect::>(); + let union_expected_orderings = union_orderings + .iter() + .map(|ordering| convert_to_sort_exprs(ordering)) + .collect::>(); + let child1 = Arc::new( + MemoryExec::try_new(&[], schema.clone(), None)? + .with_sort_information(first_orderings), + ); + let child2 = Arc::new( + MemoryExec::try_new(&[], schema.clone(), None)? + .with_sort_information(second_orderings), + ); + + let union = UnionExec::new(vec![child1, child2]); + let union_schema_properties = union.schema_properties(); + let union_actual_orderings = union_schema_properties.oeq_group(); + println!("union_orderings:{:?}", union_actual_orderings); + let err_msg = format!( + "Error in test id: {:?}, test case: {:?}", + test_idx, test_cases[test_idx] + ); + assert_eq!( + union_actual_orderings.len(), + union_expected_orderings.len(), + "{}", + err_msg + ); + for expected in &union_expected_orderings { + assert!(union_actual_orderings.contains(expected), "{}", err_msg); } - ])); + } Ok(()) } } From 18d1d6efd44f01859dd611ee7f1798b7a088c0c5 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 20 Oct 2023 14:45:22 +0300 Subject: [PATCH 076/122] Update comments --- datafusion/physical-plan/src/union.rs | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/datafusion/physical-plan/src/union.rs b/datafusion/physical-plan/src/union.rs index 668f292c2e4b..54e5394a8711 100644 --- a/datafusion/physical-plan/src/union.rs +++ b/datafusion/physical-plan/src/union.rs @@ -232,13 +232,15 @@ impl ExecutionPlan for UnionExec { .map(|child| child.schema_properties()) .collect::>(); let mut union_oeq = SchemaProperties::new(self.schema()); + // Get first ordering equivalent group as seed group. let mut existing_meets = child_oeqs[0] .oeq_group() .iter() .map(|elem| elem.to_vec()) .collect::>(); - // Iterate ordering equivalent group of first child + // Iterate ordering equivalent group of other childs for next_child_oeq in child_oeqs.iter().skip(1) { + // Find the valid meet orderings of existing meet and new group. let mut next_meets = vec![]; for existing_meet in &existing_meets { let new_meets = next_child_oeq From 6a6136d4b24199357baa506998a4b8bcf0c9b878 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 20 Oct 2023 15:08:08 +0300 Subject: [PATCH 077/122] Remove redundant check --- datafusion/physical-plan/src/windows/mod.rs | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index 1bb06d1957d1..6489a2e2a9d0 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -496,10 +496,7 @@ pub fn get_window_mode( { let req = [partition_by_reqs.clone(), order_by_reqs].concat(); let req = collapse_lex_req(req); - if req.is_empty() { - // When requirement is empty, prefer None instead of Linear. - return Ok(None); - } else if partition_by_oeq.ordering_satisfy_requirement_concrete(&req) { + if partition_by_oeq.ordering_satisfy_requirement_concrete(&req) { // Window can be run with existing ordering return Ok(Some((should_swap, partition_search_mode))); } From 3d50b689deb9d95536f566698772c05eb50dac13 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 20 Oct 2023 15:12:43 +0300 Subject: [PATCH 078/122] Simplify project out_expr function --- datafusion/physical-expr/src/utils.rs | 10 +++------- 1 file changed, 3 insertions(+), 7 deletions(-) diff --git a/datafusion/physical-expr/src/utils.rs b/datafusion/physical-expr/src/utils.rs index 656dac388c43..3f9195733087 100644 --- a/datafusion/physical-expr/src/utils.rs +++ b/datafusion/physical-expr/src/utils.rs @@ -113,17 +113,13 @@ pub fn project_out_expr( .transform(&|expr| { // If expression is not valid after projection. Treat it is as UnknownColumn. let mut normalized_form = - Some(Arc::new(UnKnownColumn::new(&expr.to_string())) as _); + Arc::new(UnKnownColumn::new(&expr.to_string())) as _; for (source, target) in projection_map { if source.eq(&expr) { - normalized_form = Some(target.clone()) + normalized_form = target.clone() } } - Ok(if let Some(normalized_form) = normalized_form { - Transformed::Yes(normalized_form) - } else { - Transformed::No(expr) - }) + Ok(Transformed::Yes(normalized_form)) }) .unwrap_or(expr) } From e6ec769f6dada3ee4c850988d086e99f33e0f7b1 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 20 Oct 2023 16:10:00 +0300 Subject: [PATCH 079/122] Remove Option> API. --- .../enforce_distribution.rs | 2 +- .../src/physical_optimizer/enforce_sorting.rs | 4 +- .../replace_with_order_preserving_variants.rs | 2 +- .../src/physical_optimizer/sort_pushdown.rs | 60 ++++++++--------- .../core/src/physical_optimizer/utils.rs | 2 +- datafusion/physical-expr/src/equivalence.rs | 64 +++++-------------- .../physical-plan/src/aggregates/mod.rs | 4 +- datafusion/physical-plan/src/windows/mod.rs | 2 +- 8 files changed, 49 insertions(+), 91 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index 2fd0b5615e88..9c83743eebfd 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -1318,7 +1318,7 @@ fn ensure_distribution( // - using order preserving variant is not desirable. let ordering_satisfied = child .schema_properties() - .ordering_satisfy_requirement_concrete(required_input_ordering); + .ordering_satisfy_requirement(required_input_ordering); if !ordering_satisfied || !order_preserving_variants_desirable { replace_order_preserving_variants(&mut child, dist_onward)?; // If ordering requirements were satisfied before repartitioning, diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index b245efdb4522..424d3c73e452 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -452,7 +452,7 @@ fn ensure_sorting( (Some(required_ordering), Some(_)) => { if !child .schema_properties() - .ordering_satisfy_requirement_concrete(&required_ordering) + .ordering_satisfy_requirement(&required_ordering) { // Make sure we preserve the ordering requirements: update_child_to_remove_unnecessary_sort(child, sort_onwards, &plan)?; @@ -516,7 +516,7 @@ fn analyze_immediate_sort_removal( // If this sort is unnecessary, we should remove it: if sort_input .schema_properties() - .ordering_satisfy(sort_exec.output_ordering()) + .ordering_satisfy(sort_exec.output_ordering().unwrap_or(&[])) { // Since we know that a `SortExec` has exactly one child, // we can use the zero index safely: diff --git a/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs b/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs index edab3a0dc481..975c71ce37db 100644 --- a/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs +++ b/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs @@ -259,7 +259,7 @@ pub(crate) fn replace_with_order_preserving_variants( // If this sort is unnecessary, we should remove it and update the plan: if updated_sort_input .schema_properties() - .ordering_satisfy(plan.output_ordering()) + .ordering_satisfy(plan.output_ordering().unwrap_or(&[])) { return Ok(Transformed::Yes(OrderPreservationContext { plan: updated_sort_input, diff --git a/datafusion/core/src/physical_optimizer/sort_pushdown.rs b/datafusion/core/src/physical_optimizer/sort_pushdown.rs index 3c49953637a7..17dea3d2cdd3 100644 --- a/datafusion/core/src/physical_optimizer/sort_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/sort_pushdown.rs @@ -29,9 +29,7 @@ use crate::physical_plan::sorts::sort::SortExec; use crate::physical_plan::{with_new_children_if_necessary, ExecutionPlan}; use datafusion_common::tree_node::{Transformed, TreeNode, VisitRecursion}; -use datafusion_common::{ - plan_datafusion_err, plan_err, DataFusionError, JoinSide, Result, -}; +use datafusion_common::{plan_err, DataFusionError, JoinSide, Result}; use datafusion_expr::JoinType; use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::{PhysicalSortExpr, PhysicalSortRequirement}; @@ -124,29 +122,27 @@ pub(crate) fn pushdown_sorts( requirements: SortPushDown, ) -> Result> { let plan = &requirements.plan; - let parent_required = requirements.required_ordering.as_deref(); - const ERR_MSG: &str = "Expects parent requirement to contain something"; - let err = || plan_datafusion_err!("{}", ERR_MSG); + let parent_required = requirements.required_ordering.as_deref().unwrap_or(&[]); if let Some(sort_exec) = plan.as_any().downcast_ref::() { let new_plan = if !plan .schema_properties() .ordering_satisfy_requirement(parent_required) { // If the current plan is a SortExec, modify it to satisfy parent requirements: - let parent_requirement = parent_required.ok_or_else(err)?; let mut new_plan = sort_exec.input().clone(); - add_sort_above(&mut new_plan, parent_requirement, sort_exec.fetch()); + add_sort_above(&mut new_plan, parent_required, sort_exec.fetch()); new_plan } else { requirements.plan }; let required_ordering = new_plan .output_ordering() - .map(PhysicalSortRequirement::from_sort_exprs); + .map(PhysicalSortRequirement::from_sort_exprs) + .unwrap_or_default(); // Since new_plan is a SortExec, we can safely get the 0th index. let child = new_plan.children().swap_remove(0); if let Some(adjusted) = - pushdown_requirement_to_children(&child, required_ordering.as_deref())? + pushdown_requirement_to_children(&child, &required_ordering)? { // Can push down requirements Ok(Transformed::Yes(SortPushDown { @@ -180,8 +176,7 @@ pub(crate) fn pushdown_sorts( } else { // Can not push down requirements, add new SortExec: let mut new_plan = requirements.plan; - let parent_requirement = parent_required.ok_or_else(err)?; - add_sort_above(&mut new_plan, parent_requirement, None); + add_sort_above(&mut new_plan, parent_required, None); Ok(Transformed::Yes(SortPushDown::init(new_plan))) } } @@ -189,18 +184,17 @@ pub(crate) fn pushdown_sorts( fn pushdown_requirement_to_children( plan: &Arc, - parent_required: Option<&[PhysicalSortRequirement]>, + parent_required: &[PhysicalSortRequirement], ) -> Result>>>> { - const ERR_MSG: &str = "Expects parent requirement to contain something"; - let err = || plan_datafusion_err!("{}", ERR_MSG); let maintains_input_order = plan.maintains_input_order(); if is_window(plan) { let required_input_ordering = plan.required_input_ordering(); - let request_child = required_input_ordering[0].as_deref(); + let request_child = required_input_ordering[0].as_deref().unwrap_or(&[]); let child_plan = plan.children().swap_remove(0); match determine_children_requirement(parent_required, request_child, child_plan) { RequirementsCompatibility::Satisfy => { - Ok(Some(vec![request_child.map(|r| r.to_vec())])) + let req = (!request_child.is_empty()).then_some(request_child.to_vec()); + Ok(Some(vec![req])) } RequirementsCompatibility::Compatible(adjusted) => Ok(Some(vec![adjusted])), RequirementsCompatibility::NonCompatible => Ok(None), @@ -208,16 +202,13 @@ fn pushdown_requirement_to_children( } else if is_union(plan) { // UnionExec does not have real sort requirements for its input. Here we change the adjusted_request_ordering to UnionExec's output ordering and // propagate the sort requirements down to correct the unnecessary descendant SortExec under the UnionExec - Ok(Some(vec![ - parent_required.map(|item| item.to_vec()); - plan.children().len() - ])) + let req = (!parent_required.is_empty()).then_some(parent_required.to_vec()); + Ok(Some(vec![req; plan.children().len()])) } else if let Some(smj) = plan.as_any().downcast_ref::() { // If the current plan is SortMergeJoinExec let left_columns_len = smj.left().schema().fields().len(); - let parent_required_expr = PhysicalSortRequirement::to_sort_exprs( - parent_required.ok_or_else(err)?.iter().cloned(), - ); + let parent_required_expr = + PhysicalSortRequirement::to_sort_exprs(parent_required.iter().cloned()); let expr_source_side = expr_source_sides(&parent_required_expr, smj.join_type(), left_columns_len); match expr_source_side { @@ -231,7 +222,7 @@ fn pushdown_requirement_to_children( let right_offset = smj.schema().fields.len() - smj.right().schema().fields.len(); let new_right_required = - shift_right_required(parent_required.ok_or_else(err)?, right_offset)?; + shift_right_required(parent_required, right_offset)?; let new_right_required_expr = PhysicalSortRequirement::to_sort_exprs(new_right_required); try_pushdown_requirements_to_join( @@ -260,19 +251,20 @@ fn pushdown_requirement_to_children( // Pushing down is not beneficial Ok(None) } else if is_sort_preserving_merge(plan) { - let ordering_req = parent_required.unwrap_or(&[]); - let new_ordering = PhysicalSortRequirement::to_sort_exprs(ordering_req.to_vec()); + let new_ordering = + PhysicalSortRequirement::to_sort_exprs(parent_required.to_vec()); let mut spm_oeq = plan.schema_properties(); // Sort preserving merge will have new ordering, one requirement above is pushed down to its below. spm_oeq = spm_oeq.with_reorder(new_ordering); // Do not push-down through SortPreservingMergeExec when // ordering requirement invalidates requirement of sort preserving merge exec. - if !spm_oeq.ordering_satisfy(plan.output_ordering()) { + if !spm_oeq.ordering_satisfy(plan.output_ordering().unwrap_or(&[])) { Ok(None) } else { // Can push-down through SortPreservingMergeExec, because parent requirement is finer // than SortPreservingMergeExec output ordering. - Ok(Some(vec![parent_required.map(|item| item.to_vec())])) + let req = (!parent_required.is_empty()).then_some(parent_required.to_vec()); + Ok(Some(vec![req])) } } else { Ok(Some( @@ -280,7 +272,7 @@ fn pushdown_requirement_to_children( .into_iter() .map(|flag| { if flag { - parent_required.map(|item| item.to_vec()) + (!parent_required.is_empty()).then_some(parent_required.to_vec()) } else { None } @@ -296,8 +288,8 @@ fn pushdown_requirement_to_children( /// 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<&[PhysicalSortRequirement]>, - request_child: Option<&[PhysicalSortRequirement]>, + parent_required: &[PhysicalSortRequirement], + request_child: &[PhysicalSortRequirement], child_plan: Arc, ) -> RequirementsCompatibility { if child_plan @@ -311,7 +303,7 @@ fn determine_children_requirement( .requirements_compatible(parent_required, request_child) { // 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()); + let adjusted = (!parent_required.is_empty()).then_some(parent_required.to_vec()); RequirementsCompatibility::Compatible(adjusted) } else { RequirementsCompatibility::NonCompatible @@ -319,7 +311,7 @@ fn determine_children_requirement( } fn try_pushdown_requirements_to_join( smj: &SortMergeJoinExec, - parent_required: Option<&[PhysicalSortRequirement]>, + parent_required: &[PhysicalSortRequirement], sort_expr: Vec, push_side: JoinSide, ) -> Result>>>> { diff --git a/datafusion/core/src/physical_optimizer/utils.rs b/datafusion/core/src/physical_optimizer/utils.rs index ebbecd84c688..bff1ca8fc321 100644 --- a/datafusion/core/src/physical_optimizer/utils.rs +++ b/datafusion/core/src/physical_optimizer/utils.rs @@ -105,7 +105,7 @@ pub fn add_sort_above( // If the ordering requirement is already satisfied, do not add a sort. if !node .schema_properties() - .ordering_satisfy_requirement_concrete(sort_requirement) + .ordering_satisfy_requirement(sort_requirement) { let sort_expr = PhysicalSortRequirement::to_sort_exprs(sort_requirement.to_vec()); let new_sort = SortExec::new(sort_expr, node.clone()).with_fetch(fetch); diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index ea1854b5d0af..53fc3b34d440 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -256,6 +256,9 @@ impl EquivalentGroups { return Some(target.clone()); } // if equivalent group of source contains expr, expr can be projected + // Assume that projection mapping is (a as a1, a+c) + // and input table is (a, b, c), where a=b + // Expression b is projected as a1 also. else if let Some(group) = self.get_equivalent_group(source) { if physical_exprs_contains(&group, expr) { return Some(target.clone()); @@ -782,37 +785,17 @@ impl SchemaProperties { collapse_lex_req(normalized_sort_reqs) } - /// Checks whether given ordering requirements are satisfied by provided [PhysicalSortExpr]s. - pub fn ordering_satisfy(&self, required: Option<&[PhysicalSortExpr]>) -> bool { - match required { - None => true, - Some(required) => self.ordering_satisfy_concrete(required), - } - } - /// Checks whether the required [`PhysicalSortExpr`]s are satisfied by the /// any of the existing orderings. - pub fn ordering_satisfy_concrete(&self, required: &[PhysicalSortExpr]) -> bool { + pub fn ordering_satisfy(&self, required: &[PhysicalSortExpr]) -> bool { // Convert `PhysicalSortExpr`s to `PhysicalSortRequirement`s let sort_requirements = PhysicalSortRequirement::from_sort_exprs(required.iter()); - self.ordering_satisfy_requirement_concrete(&sort_requirements) + self.ordering_satisfy_requirement(&sort_requirements) } /// Checks whether the given [`PhysicalSortRequirement`]s are satisfied by the /// provided [`PhysicalSortExpr`]s. pub fn ordering_satisfy_requirement( - &self, - required: Option<&[PhysicalSortRequirement]>, - ) -> bool { - match required { - None => true, - Some(required) => self.ordering_satisfy_requirement_concrete(required), - } - } - - /// Checks whether the given [`PhysicalSortRequirement`]s are satisfied by the - /// provided [`PhysicalSortExpr`]s. - pub fn ordering_satisfy_requirement_concrete( &self, required: &[PhysicalSortRequirement], ) -> bool { @@ -822,22 +805,6 @@ impl SchemaProperties { /// Checks whether the given [`PhysicalSortRequirement`]s are equal or more /// specific than the provided [`PhysicalSortRequirement`]s. pub fn requirements_compatible( - &self, - provided: Option<&[PhysicalSortRequirement]>, - required: Option<&[PhysicalSortRequirement]>, - ) -> bool { - match (provided, required) { - (_, None) => true, - (None, Some(_)) => false, - (Some(provided), Some(required)) => { - self.requirements_compatible_concrete(provided, required) - } - } - } - - /// Checks whether the given [`PhysicalSortRequirement`]s are equal or more - /// specific than the provided [`PhysicalSortRequirement`]s. - fn requirements_compatible_concrete( &self, provided: &[PhysicalSortRequirement], required: &[PhysicalSortRequirement], @@ -988,7 +955,7 @@ impl SchemaProperties { &self, leading_requirement: &PhysicalSortRequirement, ) -> bool { - self.ordering_satisfy_requirement_concrete(&[leading_requirement.clone()]) + self.ordering_satisfy_requirement(&[leading_requirement.clone()]) } /// Projects `SchemaProperties` according to mapping given in `source_to_target_mapping`. @@ -1697,12 +1664,12 @@ mod tests { let empty_schema = &Arc::new(Schema::empty()); let mut schema_properties = SchemaProperties::new(empty_schema.clone()); schema_properties.oeq_group.push(finer.clone()); - assert!(schema_properties.ordering_satisfy(Some(&crude))); + assert!(schema_properties.ordering_satisfy(&crude)); // Crude ordering doesn't satisfy finer ordering. should return false let mut schema_properties = SchemaProperties::new(empty_schema.clone()); schema_properties.oeq_group.push(crude.clone()); - assert!(!schema_properties.ordering_satisfy(Some(&finer))); + assert!(!schema_properties.ordering_satisfy(&finer)); Ok(()) } @@ -1856,9 +1823,8 @@ mod tests { )?, expected ); - let required = Some(&required[..]); assert_eq!( - schema_properties.ordering_satisfy(required), + schema_properties.ordering_satisfy(&required), expected, "{err_msg}" ); @@ -1910,7 +1876,7 @@ mod tests { requirement, expected ); assert_eq!( - schema_properties.ordering_satisfy_concrete(&requirement), + schema_properties.ordering_satisfy(&requirement), expected, "{}", err_msg @@ -1997,28 +1963,28 @@ mod tests { options, }; - assert!(schema_properties.ordering_satisfy_concrete( + assert!(schema_properties.ordering_satisfy( // After normalization would be a ASC &[sort_req_c.clone(), sort_req_a.clone(), sort_req_e.clone(),], )); - assert!(!schema_properties.ordering_satisfy_concrete( + assert!(!schema_properties.ordering_satisfy( // After normalization would be a ASC, b ASC // which is not satisfied &[sort_req_c.clone(), sort_req_b.clone(),], )); - assert!(schema_properties.ordering_satisfy_concrete( + assert!(schema_properties.ordering_satisfy( // After normalization would be a ASC &[sort_req_c.clone(), sort_req_d.clone(),], )); - assert!(!schema_properties.ordering_satisfy_concrete( + assert!(!schema_properties.ordering_satisfy( // After normalization would be a ASC, b ASC // which is not satisfied &[sort_req_d.clone(), sort_req_f.clone(), sort_req_b.clone(),], )); - assert!(schema_properties.ordering_satisfy_concrete( + assert!(schema_properties.ordering_satisfy( // After normalization would be a ASC // which is satisfied &[sort_req_d.clone(), sort_req_f.clone()], diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 500c15278dd8..73aedd3f8384 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -340,12 +340,12 @@ fn get_finest_requirement( // ordering. If so, we return `None` to indicate this. let mut all_satisfied = true; for (aggr_expr, fn_req) in aggr_expr.iter_mut().zip(order_by_expr.iter_mut()) { - if schema_properties.ordering_satisfy(fn_req.as_deref()) { + if schema_properties.ordering_satisfy(fn_req.as_deref().unwrap_or(&[])) { continue; } if let Some(reverse) = aggr_expr.reverse_expr() { let reverse_req = fn_req.as_ref().map(|item| reverse_order_bys(item)); - if schema_properties.ordering_satisfy(reverse_req.as_deref()) { + if schema_properties.ordering_satisfy(reverse_req.as_deref().unwrap_or(&[])) { // We need to update `aggr_expr` with its reverse since only its // reverse requirement is compatible with the existing requirements: *aggr_expr = reverse; diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index 6489a2e2a9d0..62ee752d797a 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -496,7 +496,7 @@ pub fn get_window_mode( { let req = [partition_by_reqs.clone(), order_by_reqs].concat(); let req = collapse_lex_req(req); - if partition_by_oeq.ordering_satisfy_requirement_concrete(&req) { + if partition_by_oeq.ordering_satisfy_requirement(&req) { // Window can be run with existing ordering return Ok(Some((should_swap, partition_search_mode))); } From c8eecad3919dc089ee566e56e4bac14437997e64 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Mon, 23 Oct 2023 10:09:29 +0300 Subject: [PATCH 080/122] Do not use project_out_expr --- datafusion/physical-expr/src/equivalence.rs | 18 ++++++++++++ datafusion/physical-expr/src/lib.rs | 4 +-- datafusion/physical-expr/src/utils.rs | 28 +------------------ .../physical-plan/src/aggregates/mod.rs | 12 ++++++-- datafusion/physical-plan/src/projection.rs | 11 ++++++-- 5 files changed, 38 insertions(+), 35 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 53fc3b34d440..9c3f665dbb80 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -958,6 +958,24 @@ impl SchemaProperties { self.ordering_satisfy_requirement(&[leading_requirement.clone()]) } + /// Projects argument `expr` according to mapping inside `source_to_target_mapping`. + /// While doing so consider equalities also. + /// As an example assume `source_to_target_mapping` contains following mapping + /// a -> a1 + /// b -> b1 + /// Also assume that we know that a=c (they are equal) + /// This function projects + /// a+b to Some(a1+b1) + /// c+b to Some(a1+b1) + /// d to None. (meaning cannot be projected) + pub fn project_expr( + &self, + source_to_target_mapping: &ProjectionMapping, + expr: &Arc, + ) -> Option> { + self.eq_groups.project_expr(source_to_target_mapping, expr) + } + /// Projects `SchemaProperties` according to mapping given in `source_to_target_mapping`. pub fn project( &self, diff --git a/datafusion/physical-expr/src/lib.rs b/datafusion/physical-expr/src/lib.rs index 9f28aa6b5fc4..4348390c2a8e 100644 --- a/datafusion/physical-expr/src/lib.rs +++ b/datafusion/physical-expr/src/lib.rs @@ -66,6 +66,6 @@ pub use sort_expr::{ PhysicalSortRequirement, }; pub use utils::{ - expr_list_eq_any_order, expr_list_eq_strict_order, project_out_expr, - reverse_order_bys, split_conjunction, + expr_list_eq_any_order, expr_list_eq_strict_order, reverse_order_bys, + split_conjunction, }; diff --git a/datafusion/physical-expr/src/utils.rs b/datafusion/physical-expr/src/utils.rs index 3f9195733087..8df114f841c0 100644 --- a/datafusion/physical-expr/src/utils.rs +++ b/datafusion/physical-expr/src/utils.rs @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -use crate::expressions::{BinaryExpr, Column, UnKnownColumn}; +use crate::expressions::{BinaryExpr, Column}; use crate::{PhysicalExpr, PhysicalSortExpr}; use arrow::array::{make_array, Array, ArrayRef, BooleanArray, MutableArrayData}; @@ -27,7 +27,6 @@ use datafusion_common::tree_node::{ use datafusion_common::Result; use datafusion_expr::Operator; -use crate::equivalence::ProjectionMapping; use itertools::Itertools; use petgraph::graph::NodeIndex; use petgraph::stable_graph::StableGraph; @@ -99,31 +98,6 @@ fn split_conjunction_impl<'a>( } } -/// Normalize the output expressions based on projection_map. -/// -/// If there is a mapping in projection_map, replace the expression -/// in the output expressions with the expression after mapping. -/// Otherwise, replace the expression with a place holder of [UnKnownColumn] -/// -pub fn project_out_expr( - expr: Arc, - projection_map: &ProjectionMapping, -) -> Arc { - expr.clone() - .transform(&|expr| { - // If expression is not valid after projection. Treat it is as UnknownColumn. - let mut normalized_form = - Arc::new(UnKnownColumn::new(&expr.to_string())) as _; - for (source, target) in projection_map { - if source.eq(&expr) { - normalized_form = target.clone() - } - } - Ok(Transformed::Yes(normalized_form)) - }) - .unwrap_or(expr) -} - /// This function maps back requirement after ProjectionExec /// to the Executor for its input. // Specifically, `ProjectionExec` changes index of `Column`s in the schema of its input executor. diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 73aedd3f8384..c812a62e9545 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -46,8 +46,8 @@ use datafusion_physical_expr::{ aggregate::is_order_sensitive, equivalence::collapse_lex_req, expressions::{Column, Max, Min}, - physical_exprs_contains, project_out_expr, reverse_order_bys, AggregateExpr, - LexOrdering, LexOrderingReq, PhysicalExpr, PhysicalSortExpr, PhysicalSortRequirement, + physical_exprs_contains, reverse_order_bys, AggregateExpr, LexOrdering, + LexOrderingReq, PhysicalExpr, PhysicalSortExpr, PhysicalSortRequirement, SchemaProperties, }; @@ -62,6 +62,7 @@ mod topk_stream; pub use datafusion_expr::AggregateFunction; pub use datafusion_physical_expr::expressions::create_aggregate_expr; +use datafusion_physical_expr::expressions::UnKnownColumn; /// Hash aggregate modes #[derive(Debug, Copy, Clone, PartialEq, Eq)] @@ -745,12 +746,17 @@ impl ExecutionPlan for AggregateExec { AggregateMode::Partial | AggregateMode::Single => { // Partial and Single Aggregation will not change the output partitioning but need to respect the Alias let input_partition = self.input.output_partitioning(); + let input_schema_properties = self.input.schema_properties(); match input_partition { Partitioning::Hash(exprs, part) => { let normalized_exprs = exprs .into_iter() .map(|expr| { - project_out_expr(expr, &self.source_to_target_mapping) + input_schema_properties + .project_expr(&self.source_to_target_mapping, &expr) + .unwrap_or(Arc::new(UnKnownColumn::new( + &expr.to_string(), + ))) }) .collect::>(); Partitioning::Hash(normalized_exprs, part) diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index 49c19c256442..8fa80da387ba 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -39,8 +39,8 @@ use arrow::record_batch::{RecordBatch, RecordBatchOptions}; use datafusion_common::stats::Precision; use datafusion_common::Result; use datafusion_execution::TaskContext; -use datafusion_physical_expr::expressions::Literal; -use datafusion_physical_expr::{project_out_expr, SchemaProperties}; +use datafusion_physical_expr::expressions::{Literal, UnKnownColumn}; +use datafusion_physical_expr::SchemaProperties; use futures::stream::{Stream, StreamExt}; use log::trace; @@ -176,10 +176,15 @@ impl ExecutionPlan for ProjectionExec { fn output_partitioning(&self) -> Partitioning { // Output partition need to respect the alias let input_partition = self.input.output_partitioning(); + let input_schema_properties = self.input.schema_properties(); if let Partitioning::Hash(exprs, part) = input_partition { let normalized_exprs = exprs .into_iter() - .map(|expr| project_out_expr(expr, &self.source_to_target_mapping)) + .map(|expr| { + input_schema_properties + .project_expr(&self.source_to_target_mapping, &expr) + .unwrap_or(Arc::new(UnKnownColumn::new(&expr.to_string()))) + }) .collect(); Partitioning::Hash(normalized_exprs, part) } else { From 76f49212a9327768e75a9063f3b15bf45ba0a3cf Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Mon, 23 Oct 2023 10:14:00 +0300 Subject: [PATCH 081/122] Simplifications --- .../physical-plan/src/aggregates/mod.rs | 41 +++++++++---------- .../sqllogictest/test_files/tpch/q17.slt.part | 26 ++++++------ 2 files changed, 31 insertions(+), 36 deletions(-) diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index c812a62e9545..2db89cca8d93 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -742,30 +742,27 @@ impl ExecutionPlan for AggregateExec { /// Get the output partitioning of this plan fn output_partitioning(&self) -> Partitioning { - match &self.mode { - AggregateMode::Partial | AggregateMode::Single => { - // Partial and Single Aggregation will not change the output partitioning but need to respect the Alias - let input_partition = self.input.output_partitioning(); - let input_schema_properties = self.input.schema_properties(); - match input_partition { - Partitioning::Hash(exprs, part) => { - let normalized_exprs = exprs - .into_iter() - .map(|expr| { - input_schema_properties - .project_expr(&self.source_to_target_mapping, &expr) - .unwrap_or(Arc::new(UnKnownColumn::new( - &expr.to_string(), - ))) - }) - .collect::>(); - Partitioning::Hash(normalized_exprs, part) - } - _ => input_partition, - } + let input_partition = self.input.output_partitioning(); + if self.mode.is_first_stage() { + // First stage Aggregation will not change the output partitioning but need to respect the Alias + // (e.g mapping in the group by expression) + let input_schema_properties = self.input.schema_properties(); + if let Partitioning::Hash(exprs, part) = input_partition { + let normalized_exprs = exprs + .into_iter() + .map(|expr| { + input_schema_properties + .project_expr(&self.source_to_target_mapping, &expr) + .unwrap_or(Arc::new(UnKnownColumn::new(&expr.to_string()))) + }) + .collect(); + Partitioning::Hash(normalized_exprs, part) + } else { + input_partition } + } else { // Final Aggregation's output partitioning is the same as its real input - _ => self.input.output_partitioning(), + input_partition } } diff --git a/datafusion/sqllogictest/test_files/tpch/q17.slt.part b/datafusion/sqllogictest/test_files/tpch/q17.slt.part index 50661b9b10a8..4d4aa4b1395f 100644 --- a/datafusion/sqllogictest/test_files/tpch/q17.slt.part +++ b/datafusion/sqllogictest/test_files/tpch/q17.slt.part @@ -58,21 +58,19 @@ ProjectionExec: expr=[CAST(SUM(lineitem.l_extendedprice)@0 AS Float64) / 7 as av --------ProjectionExec: expr=[l_extendedprice@1 as l_extendedprice] ----------CoalesceBatchesExec: target_batch_size=8192 ------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(p_partkey@2, l_partkey@1)], filter=CAST(l_quantity@0 AS Decimal128(30, 15)) < Float64(0.2) * AVG(lineitem.l_quantity)@1 ---------------CoalesceBatchesExec: target_batch_size=8192 -----------------RepartitionExec: partitioning=Hash([p_partkey@2], 4), input_partitions=4 -------------------ProjectionExec: expr=[l_quantity@1 as l_quantity, l_extendedprice@2 as l_extendedprice, p_partkey@3 as p_partkey] +--------------ProjectionExec: expr=[l_quantity@1 as l_quantity, l_extendedprice@2 as l_extendedprice, p_partkey@3 as p_partkey] +----------------CoalesceBatchesExec: target_batch_size=8192 +------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)] --------------------CoalesceBatchesExec: target_batch_size=8192 -----------------------HashJoinExec: mode=Partitioned, join_type=Inner, on=[(l_partkey@0, p_partkey@0)] -------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 -----------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_partkey, l_quantity, l_extendedprice], has_header=false -------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 -----------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey] -------------------------------CoalesceBatchesExec: target_batch_size=8192 ---------------------------------FilterExec: p_brand@1 = Brand#23 AND p_container@2 = MED BOX -----------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 -------------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_brand, p_container], has_header=false +----------------------RepartitionExec: partitioning=Hash([l_partkey@0], 4), input_partitions=4 +------------------------CsvExec: file_groups={4 groups: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:0..18561749], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:18561749..37123498], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:37123498..55685247], [WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/lineitem.tbl:55685247..74246996]]}, projection=[l_partkey, l_quantity, l_extendedprice], has_header=false +--------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------RepartitionExec: partitioning=Hash([p_partkey@0], 4), input_partitions=4 +------------------------ProjectionExec: expr=[p_partkey@0 as p_partkey] +--------------------------CoalesceBatchesExec: target_batch_size=8192 +----------------------------FilterExec: p_brand@1 = Brand#23 AND p_container@2 = MED BOX +------------------------------RepartitionExec: partitioning=RoundRobinBatch(4), input_partitions=1 +--------------------------------CsvExec: file_groups={1 group: [[WORKSPACE_ROOT/datafusion/sqllogictest/test_files/tpch/data/part.tbl]]}, projection=[p_partkey, p_brand, p_container], has_header=false --------------ProjectionExec: expr=[CAST(0.2 * CAST(AVG(lineitem.l_quantity)@1 AS Float64) AS Decimal128(30, 15)) as Float64(0.2) * AVG(lineitem.l_quantity), l_partkey@0 as l_partkey] ----------------AggregateExec: mode=FinalPartitioned, gby=[l_partkey@0 as l_partkey], aggr=[AVG(lineitem.l_quantity)] ------------------CoalesceBatchesExec: target_batch_size=8192 From 36c65b88edccef555087bfaff65a4f5ccee341b1 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Mon, 23 Oct 2023 14:51:35 +0300 Subject: [PATCH 082/122] Review Part 3 --- .../enforce_distribution.rs | 14 +++---- .../src/physical_optimizer/sort_pushdown.rs | 28 +++++++++++--- datafusion/physical-expr/src/equivalence.rs | 21 +++++----- datafusion/physical-expr/src/lib.rs | 9 ++--- datafusion/physical-expr/src/partitioning.rs | 8 ++-- datafusion/physical-expr/src/physical_expr.rs | 1 - .../physical-expr/src/scalar_function.rs | 25 ++++++------ datafusion/physical-expr/src/utils.rs | 38 +++++-------------- .../physical-plan/src/aggregates/mod.rs | 21 +++++----- datafusion/physical-plan/src/filter.rs | 4 +- datafusion/physical-plan/src/projection.rs | 4 +- datafusion/physical-plan/src/union.rs | 14 ++----- 12 files changed, 88 insertions(+), 99 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index 9c83743eebfd..0bbaae827291 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -52,9 +52,7 @@ use datafusion_common::tree_node::{Transformed, TreeNode, VisitRecursion}; use datafusion_expr::logical_plan::JoinType; use datafusion_physical_expr::expressions::{Column, NoOp}; use datafusion_physical_expr::utils::map_columns_before_projection; -use datafusion_physical_expr::{ - expr_list_eq_strict_order, PhysicalExpr, SchemaProperties, -}; +use datafusion_physical_expr::{physical_exprs_equal, PhysicalExpr, SchemaProperties}; use datafusion_physical_plan::unbounded_output; use datafusion_physical_plan::windows::{get_best_fitting_window, BoundedWindowAggExec}; @@ -498,7 +496,7 @@ fn reorder_aggregate_keys( if parent_required.len() != output_exprs.len() || !agg_exec.group_by().null_expr().is_empty() - || expr_list_eq_strict_order(&output_exprs, parent_required) + || physical_exprs_equal(&output_exprs, parent_required) { Ok(PlanWithKeyRequirements::new(agg_plan)) } else { @@ -761,8 +759,8 @@ fn try_reorder( if join_keys.left_keys.len() != expected.len() { return None; } - if expr_list_eq_strict_order(expected, &join_keys.left_keys) - || expr_list_eq_strict_order(expected, &join_keys.right_keys) + if physical_exprs_equal(expected, &join_keys.left_keys) + || physical_exprs_equal(expected, &join_keys.right_keys) { return Some((join_keys, vec![])); } else if !equivalence_properties.eq_groups().is_empty() { @@ -786,8 +784,8 @@ fn try_reorder( .collect::>(); assert_eq!(join_keys.right_keys.len(), normalized_right_keys.len()); - if expr_list_eq_strict_order(&normalized_expected, &normalized_left_keys) - || expr_list_eq_strict_order(&normalized_expected, &normalized_right_keys) + if physical_exprs_equal(&normalized_expected, &normalized_left_keys) + || physical_exprs_equal(&normalized_expected, &normalized_right_keys) { return Some((join_keys, vec![])); } diff --git a/datafusion/core/src/physical_optimizer/sort_pushdown.rs b/datafusion/core/src/physical_optimizer/sort_pushdown.rs index 17dea3d2cdd3..48e9ba58d5b1 100644 --- a/datafusion/core/src/physical_optimizer/sort_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/sort_pushdown.rs @@ -193,7 +193,11 @@ fn pushdown_requirement_to_children( let child_plan = plan.children().swap_remove(0); match determine_children_requirement(parent_required, request_child, child_plan) { RequirementsCompatibility::Satisfy => { - let req = (!request_child.is_empty()).then_some(request_child.to_vec()); + let req = if request_child.is_empty() { + None + } else { + Some(request_child.to_vec()) + }; Ok(Some(vec![req])) } RequirementsCompatibility::Compatible(adjusted) => Ok(Some(vec![adjusted])), @@ -202,7 +206,11 @@ fn pushdown_requirement_to_children( } else if is_union(plan) { // UnionExec does not have real sort requirements for its input. Here we change the adjusted_request_ordering to UnionExec's output ordering and // propagate the sort requirements down to correct the unnecessary descendant SortExec under the UnionExec - let req = (!parent_required.is_empty()).then_some(parent_required.to_vec()); + let req = if parent_required.is_empty() { + None + } else { + Some(parent_required.to_vec()) + }; Ok(Some(vec![req; plan.children().len()])) } else if let Some(smj) = plan.as_any().downcast_ref::() { // If the current plan is SortMergeJoinExec @@ -263,7 +271,11 @@ fn pushdown_requirement_to_children( } else { // Can push-down through SortPreservingMergeExec, because parent requirement is finer // than SortPreservingMergeExec output ordering. - let req = (!parent_required.is_empty()).then_some(parent_required.to_vec()); + let req = if parent_required.is_empty() { + None + } else { + Some(parent_required.to_vec()) + }; Ok(Some(vec![req])) } } else { @@ -271,8 +283,8 @@ fn pushdown_requirement_to_children( maintains_input_order .into_iter() .map(|flag| { - if flag { - (!parent_required.is_empty()).then_some(parent_required.to_vec()) + if flag && !parent_required.is_empty() { + Some(parent_required.to_vec()) } else { None } @@ -303,7 +315,11 @@ fn determine_children_requirement( .requirements_compatible(parent_required, request_child) { // parent requirements are more specific, adjust the request child requirements and push down the new requirements - let adjusted = (!parent_required.is_empty()).then_some(parent_required.to_vec()); + let adjusted = if parent_required.is_empty() { + None + } else { + Some(parent_required.to_vec()) + }; RequirementsCompatibility::Compatible(adjusted) } else { RequirementsCompatibility::NonCompatible diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 9c3f665dbb80..f0d2b5fe296e 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -15,22 +15,23 @@ // specific language governing permissions and limitations // under the License. +use std::hash::Hash; +use std::sync::Arc; + use crate::expressions::Column; +use crate::physical_expr::{deduplicate_physical_exprs, have_common_entries}; +use crate::sort_properties::{ExprOrdering, SortProperties}; use crate::{ physical_exprs_contains, LexOrdering, LexOrderingRef, LexOrderingReq, PhysicalExpr, PhysicalSortExpr, PhysicalSortRequirement, }; use arrow::datatypes::SchemaRef; - -use crate::physical_expr::{deduplicate_physical_exprs, have_common_entries}; -use crate::sort_properties::{ExprOrdering, SortProperties}; use arrow_schema::SortOptions; use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::{JoinSide, JoinType, Result}; + use itertools::izip; -use std::hash::Hash; -use std::sync::Arc; /// EquivalentClass is a set of [`Arc`]s that are known /// to have the same value in all tuples in a relation. These are generated by equality predicates, @@ -1380,20 +1381,22 @@ fn get_updated_right_ordering_equivalent_group( #[cfg(test)] mod tests { + use std::sync::Arc; + use super::*; use crate::expressions::{col, lit, Column}; - use arrow::datatypes::{DataType, Field, Schema}; - use datafusion_common::Result; - use crate::physical_expr::physical_exprs_equal; + use arrow::compute::{lexsort_to_indices, SortColumn}; + use arrow::datatypes::{DataType, Field, Schema}; use arrow_array::{ArrayRef, RecordBatch, UInt32Array, UInt64Array}; use arrow_schema::{Fields, SortOptions}; + use datafusion_common::Result; + use itertools::Itertools; use rand::rngs::StdRng; use rand::seq::SliceRandom; use rand::{Rng, SeedableRng}; - use std::sync::Arc; // Generate a schema which consists of 7 columns (a, b, c, d, e, f, g) fn create_test_schema() -> Result { diff --git a/datafusion/physical-expr/src/lib.rs b/datafusion/physical-expr/src/lib.rs index 4348390c2a8e..02d666f5fa36 100644 --- a/datafusion/physical-expr/src/lib.rs +++ b/datafusion/physical-expr/src/lib.rs @@ -58,14 +58,13 @@ pub use equivalence::{ }; pub use partitioning::{Distribution, Partitioning}; -pub use physical_expr::{physical_exprs_contains, PhysicalExpr, PhysicalExprRef}; +pub use physical_expr::{ + physical_exprs_contains, physical_exprs_equal, PhysicalExpr, PhysicalExprRef, +}; pub use planner::create_physical_expr; pub use scalar_function::ScalarFunctionExpr; pub use sort_expr::{ LexOrdering, LexOrderingRef, LexOrderingReq, PhysicalSortExpr, PhysicalSortRequirement, }; -pub use utils::{ - expr_list_eq_any_order, expr_list_eq_strict_order, reverse_order_bys, - split_conjunction, -}; +pub use utils::{expr_list_eq_any_order, reverse_order_bys, split_conjunction}; diff --git a/datafusion/physical-expr/src/partitioning.rs b/datafusion/physical-expr/src/partitioning.rs index c9a38d7ba443..07cefbf57b12 100644 --- a/datafusion/physical-expr/src/partitioning.rs +++ b/datafusion/physical-expr/src/partitioning.rs @@ -20,7 +20,7 @@ use std::fmt; use std::sync::Arc; -use crate::{expr_list_eq_strict_order, PhysicalExpr, SchemaProperties}; +use crate::{physical_exprs_equal, PhysicalExpr, SchemaProperties}; /// Partitioning schemes supported by operators. #[derive(Debug, Clone)] @@ -78,7 +78,7 @@ impl Partitioning { // then we need to have the partition count and hash functions validation. Partitioning::Hash(partition_exprs, _) => { let fast_match = - expr_list_eq_strict_order(&required_exprs, partition_exprs); + physical_exprs_equal(&required_exprs, partition_exprs); // If the required exprs do not match, need to leverage the schema_properties provided by the child // and normalize both exprs based on the equivalent groups. if !fast_match { @@ -93,7 +93,7 @@ impl Partitioning { .iter() .map(|e| eq_groups.normalize_expr(e.clone())) .collect::>(); - return expr_list_eq_strict_order( + return physical_exprs_equal( &normalized_required_exprs, &normalized_partition_exprs, ); @@ -117,7 +117,7 @@ impl PartialEq for Partitioning { Partitioning::RoundRobinBatch(count2), ) if count1 == count2 => true, (Partitioning::Hash(exprs1, count1), Partitioning::Hash(exprs2, count2)) - if expr_list_eq_strict_order(exprs1, exprs2) && (count1 == count2) => + if physical_exprs_equal(exprs1, exprs2) && (count1 == count2) => { true } diff --git a/datafusion/physical-expr/src/physical_expr.rs b/datafusion/physical-expr/src/physical_expr.rs index b57716782751..35e73a919ec8 100644 --- a/datafusion/physical-expr/src/physical_expr.rs +++ b/datafusion/physical-expr/src/physical_expr.rs @@ -251,7 +251,6 @@ pub fn have_common_entries( } /// Checks whether the given physical expression slices are equal. -#[allow(dead_code)] pub fn physical_exprs_equal( lhs: &[Arc], rhs: &[Arc], diff --git a/datafusion/physical-expr/src/scalar_function.rs b/datafusion/physical-expr/src/scalar_function.rs index dc48baa23ab3..5acd5dcf2336 100644 --- a/datafusion/physical-expr/src/scalar_function.rs +++ b/datafusion/physical-expr/src/scalar_function.rs @@ -29,24 +29,23 @@ //! This module also has a set of coercion rules to improve user experience: if an argument i32 is passed //! to a function that supports f64, it is coerced to f64. +use std::any::Any; +use std::fmt::{self, Debug, Formatter}; +use std::hash::{Hash, Hasher}; +use std::sync::Arc; + use crate::functions::out_ordering; -use crate::physical_expr::down_cast_any_ref; +use crate::physical_expr::{down_cast_any_ref, physical_exprs_equal}; use crate::sort_properties::SortProperties; -use crate::utils::expr_list_eq_strict_order; use crate::PhysicalExpr; + use arrow::datatypes::{DataType, Schema}; use arrow::record_batch::RecordBatch; use datafusion_common::Result; -use datafusion_expr::expr_vec_fmt; -use datafusion_expr::BuiltinScalarFunction; -use datafusion_expr::ColumnarValue; -use datafusion_expr::FuncMonotonicity; -use datafusion_expr::ScalarFunctionImplementation; -use std::any::Any; -use std::fmt::Debug; -use std::fmt::{self, Formatter}; -use std::hash::{Hash, Hasher}; -use std::sync::Arc; +use datafusion_expr::{ + expr_vec_fmt, BuiltinScalarFunction, ColumnarValue, FuncMonotonicity, + ScalarFunctionImplementation, +}; /// Physical expression of a scalar function pub struct ScalarFunctionExpr { @@ -194,7 +193,7 @@ impl PartialEq for ScalarFunctionExpr { .downcast_ref::() .map(|x| { self.name == x.name - && expr_list_eq_strict_order(&self.args, &x.args) + && physical_exprs_equal(&self.args, &x.args) && self.return_type == x.return_type }) .unwrap_or(false) diff --git a/datafusion/physical-expr/src/utils.rs b/datafusion/physical-expr/src/utils.rs index 8df114f841c0..cc56d5568285 100644 --- a/datafusion/physical-expr/src/utils.rs +++ b/datafusion/physical-expr/src/utils.rs @@ -15,6 +15,10 @@ // specific language governing permissions and limitations // under the License. +use std::borrow::Borrow; +use std::collections::{HashMap, HashSet}; +use std::sync::Arc; + use crate::expressions::{BinaryExpr, Column}; use crate::{PhysicalExpr, PhysicalSortExpr}; @@ -30,10 +34,6 @@ use datafusion_expr::Operator; use itertools::Itertools; use petgraph::graph::NodeIndex; use petgraph::stable_graph::StableGraph; -use std::borrow::Borrow; -use std::collections::HashMap; -use std::collections::HashSet; -use std::sync::Arc; /// Compare the two expr lists are equal no matter the order. /// For example two InListExpr can be considered to be equals no matter the order: @@ -59,14 +59,6 @@ pub fn expr_list_eq_any_order( } } -/// Strictly compare the two expr lists are equal in the given order. -pub fn expr_list_eq_strict_order( - list1: &[Arc], - list2: &[Arc], -) -> bool { - list1.len() == list2.len() && list1.iter().zip(list2.iter()).all(|(e1, e2)| e1.eq(e2)) -} - /// Assume the predicate is in the form of CNF, split the predicate to a Vec of PhysicalExprs. /// /// For example, split "a1 = a2 AND b1 <= b2 AND c1 != c2" into ["a1 = a2", "b1 <= b2", "c1 != c2"] @@ -400,7 +392,7 @@ mod tests { use super::*; use crate::equivalence::SchemaProperties; use crate::expressions::{binary, cast, col, in_list, lit, Column, Literal}; - use crate::PhysicalSortExpr; + use crate::{physical_exprs_equal, PhysicalSortExpr}; use arrow::compute::SortOptions; use arrow_array::Int32Array; @@ -558,14 +550,8 @@ mod tests { assert!(!expr_list_eq_any_order(list1.as_slice(), list2.as_slice())); assert!(!expr_list_eq_any_order(list2.as_slice(), list1.as_slice())); - assert!(!expr_list_eq_strict_order( - list1.as_slice(), - list2.as_slice() - )); - assert!(!expr_list_eq_strict_order( - list2.as_slice(), - list1.as_slice() - )); + assert!(!physical_exprs_equal(list1.as_slice(), list2.as_slice())); + assert!(!physical_exprs_equal(list2.as_slice(), list1.as_slice())); let list3: Vec> = vec![ Arc::new(Column::new("a", 0)), @@ -586,14 +572,8 @@ mod tests { assert!(expr_list_eq_any_order(list3.as_slice(), list3.as_slice())); assert!(expr_list_eq_any_order(list4.as_slice(), list4.as_slice())); - assert!(!expr_list_eq_strict_order( - list3.as_slice(), - list4.as_slice() - )); - assert!(!expr_list_eq_strict_order( - list4.as_slice(), - list3.as_slice() - )); + assert!(!physical_exprs_equal(list3.as_slice(), list4.as_slice())); + assert!(!physical_exprs_equal(list4.as_slice(), list3.as_slice())); assert!(expr_list_eq_any_order(list3.as_slice(), list3.as_slice())); assert!(expr_list_eq_any_order(list4.as_slice(), list4.as_slice())); diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 2db89cca8d93..060b077d9032 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -45,7 +45,7 @@ use datafusion_expr::Accumulator; use datafusion_physical_expr::{ aggregate::is_order_sensitive, equivalence::collapse_lex_req, - expressions::{Column, Max, Min}, + expressions::{Column, Max, Min, UnKnownColumn}, physical_exprs_contains, reverse_order_bys, AggregateExpr, LexOrdering, LexOrderingReq, PhysicalExpr, PhysicalSortExpr, PhysicalSortRequirement, SchemaProperties, @@ -62,7 +62,6 @@ mod topk_stream; pub use datafusion_expr::AggregateFunction; pub use datafusion_physical_expr::expressions::create_aggregate_expr; -use datafusion_physical_expr::expressions::UnKnownColumn; /// Hash aggregate modes #[derive(Debug, Copy, Clone, PartialEq, Eq)] @@ -744,8 +743,9 @@ impl ExecutionPlan for AggregateExec { fn output_partitioning(&self) -> Partitioning { let input_partition = self.input.output_partitioning(); if self.mode.is_first_stage() { - // First stage Aggregation will not change the output partitioning but need to respect the Alias - // (e.g mapping in the group by expression) + // First stage aggregation will not change the output partitioning, + // but needs to respect aliases (e.g. mapping in the GROUP BY + // expression). let input_schema_properties = self.input.schema_properties(); if let Partitioning::Hash(exprs, part) = input_partition { let normalized_exprs = exprs @@ -753,17 +753,16 @@ impl ExecutionPlan for AggregateExec { .map(|expr| { input_schema_properties .project_expr(&self.source_to_target_mapping, &expr) - .unwrap_or(Arc::new(UnKnownColumn::new(&expr.to_string()))) + .unwrap_or_else(|| { + Arc::new(UnKnownColumn::new(&expr.to_string())) + }) }) .collect(); - Partitioning::Hash(normalized_exprs, part) - } else { - input_partition + return Partitioning::Hash(normalized_exprs, part); } - } else { - // Final Aggregation's output partitioning is the same as its real input - input_partition } + // Final Aggregation's output partitioning is the same as its real input + input_partition } /// Specifies whether this plan generates an infinite stream of records. diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index 2325775ec995..525fd6e18ed5 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -165,9 +165,9 @@ impl ExecutionPlan for FilterExec { fn with_new_children( self: Arc, - children: Vec>, + mut children: Vec>, ) -> Result> { - FilterExec::try_new(self.predicate.clone(), children[0].clone()) + FilterExec::try_new(self.predicate.clone(), children.swap_remove(0)) .map(|e| Arc::new(e) as _) } diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index 8fa80da387ba..0722a8787032 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -183,7 +183,9 @@ impl ExecutionPlan for ProjectionExec { .map(|expr| { input_schema_properties .project_expr(&self.source_to_target_mapping, &expr) - .unwrap_or(Arc::new(UnKnownColumn::new(&expr.to_string()))) + .unwrap_or_else(|| { + Arc::new(UnKnownColumn::new(&expr.to_string())) + }) }) .collect(); Partitioning::Hash(normalized_exprs, part) diff --git a/datafusion/physical-plan/src/union.rs b/datafusion/physical-plan/src/union.rs index 54e5394a8711..628f366118f6 100644 --- a/datafusion/physical-plan/src/union.rs +++ b/datafusion/physical-plan/src/union.rs @@ -239,24 +239,18 @@ impl ExecutionPlan for UnionExec { .map(|elem| elem.to_vec()) .collect::>(); // Iterate ordering equivalent group of other childs - for next_child_oeq in child_oeqs.iter().skip(1) { + for next_child_oeq in &child_oeqs[1..] { // Find the valid meet orderings of existing meet and new group. let mut next_meets = vec![]; for existing_meet in &existing_meets { - let new_meets = next_child_oeq - .oeq_group() - .iter() - .filter_map(|ordering| { - next_child_oeq.get_meet_ordering(ordering, existing_meet) - }) - .collect::>(); - next_meets.extend(new_meets); + next_meets.extend(next_child_oeq.oeq_group().iter().filter_map( + |ordering| next_child_oeq.get_meet_ordering(ordering, existing_meet), + )); } existing_meets = next_meets; } // existing_meets contains the all of the valid orderings after union union_oeq.add_new_orderings(&existing_meets); - union_oeq } From 1d30d89b1d6e102d97ff40bd665be61a4ab95fe4 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Mon, 23 Oct 2023 17:43:33 +0300 Subject: [PATCH 083/122] Review Part 4 --- datafusion/physical-expr/src/equivalence.rs | 233 +++++++++--------- datafusion/physical-expr/src/physical_expr.rs | 36 +-- 2 files changed, 139 insertions(+), 130 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index f0d2b5fe296e..000c1e04f6ca 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -33,137 +33,139 @@ use datafusion_common::{JoinSide, JoinType, Result}; use itertools::izip; -/// EquivalentClass is a set of [`Arc`]s that are known -/// to have the same value in all tuples in a relation. These are generated by equality predicates, -/// typically equi-join conditions and equality conditions in filters. +/// An `EquivalenceClass` is a set of [`Arc`]s that are known +/// to have the same value for all tuples in a relation. These are generated by +/// equality predicates, typically equi-join conditions and equality conditions +/// in filters. +pub type EquivalenceClass = Vec>; + +/// An `EquivalenceGroups` is a collection of `EquivalenceClass`es where each +/// class represents a distinct equivalence class in a relation. #[derive(Debug, Clone)] -pub struct EquivalentGroups { - inner: Vec>>, +pub struct EquivalenceGroup { + classes: Vec, } -impl EquivalentGroups { - /// Creates an empty ordering equivalent group +impl EquivalenceGroup { + /// Creates an empty equivalence group. fn empty() -> Self { - EquivalentGroups { inner: vec![] } + EquivalenceGroup { classes: vec![] } } - /// Creates ordering equivalent groups from given vectors - /// Each vector corresponds to a group - fn new(entries: Vec>>) -> Self { - let mut res = EquivalentGroups { inner: entries }; - res.remove_redundant_entries(); - res + /// Creates an equivalence group from the given equivalence classes. + fn new(classes: Vec) -> Self { + let mut result = EquivalenceGroup { classes }; + result.remove_redundant_entries(); + result } - /// Get how many equivalent groups there are. + /// Returns how many equivalence classes there are in this group. fn len(&self) -> usize { - self.inner.len() + self.classes.len() } - /// Check whether equivalent groups is empty + /// Checks whether this equivalence group is empty. pub fn is_empty(&self) -> bool { self.len() == 0 } - /// Iterate over inner vector. + /// Returns an iterator over the equivalence classes in this group. fn iter(&self) -> impl Iterator>> { - self.inner.iter() + self.classes.iter() } - /// Adds tuple argument to the equivalent groups - /// It is known that first and second entry in the tuple will have same values in the table. - /// This can arise after filter(a=b), alias(a, a as b), etc. - pub fn add_equal_conditions( + /// Adds the equality `left` = `right` to this equivalence group. + /// New equality conditions often arise after steps like `Filter(a = b)`, + /// `Alias(a, a as b)` etc. + fn add_equal_conditions( &mut self, - new_conditions: (&Arc, &Arc), + left: &Arc, + right: &Arc, ) { - let (first, second) = new_conditions; - let mut first_group = None; - let mut second_group = None; - for (group_idx, eq_class) in self.inner.iter().enumerate() { - if physical_exprs_contains(eq_class, first) { - first_group = Some(group_idx); + let mut first_class = None; + let mut second_class = None; + for (idx, eq_class) in self.classes.iter().enumerate() { + if physical_exprs_contains(eq_class, left) { + first_class = Some(idx); } - if physical_exprs_contains(eq_class, second) { - second_group = Some(group_idx); + if physical_exprs_contains(eq_class, right) { + second_class = Some(idx); } } - match (first_group, second_group) { - (Some(first_group_idx), Some(second_group_idx)) => { - // We should bridge these groups - if first_group_idx != second_group_idx { - let other_class = self.inner[second_group_idx].clone(); - // TODO: Use group at the lower index during merging - // This would improve performance during remove. - self.inner[first_group_idx].extend(other_class); - self.inner.remove(second_group_idx); + match (first_class, second_class) { + (Some(mut first_idx), Some(mut second_idx)) => { + // If the given left and right sides belong to different classes, + // we should unify/bridge these classes. + if first_idx != second_idx { + if first_idx > second_idx { + (first_idx, second_idx) = (second_idx, first_idx); + } + let other_class = self.classes.swap_remove(second_idx); + self.classes[first_idx].extend(other_class); } } (Some(group_idx), None) => { - // Extend existing group with new entry - self.inner[group_idx].push(second.clone()); + // Right side is new, extend left side's class: + self.classes[group_idx].push(right.clone()); } (None, Some(group_idx)) => { - // Extend existing group with new entry - self.inner[group_idx].push(first.clone()); + // Left side is new, extend right side's class: + self.classes[group_idx].push(left.clone()); } (None, None) => { - // None of the expressions, is among existing groups - // Create a new group. - self.inner.push(vec![first.clone(), second.clone()]); + // None of the expressions is among existing classes. + // Create a new equivalence class and extend the group. + self.classes.push(vec![left.clone(), right.clone()]); } } } - /// Remove redundant entries from the state. + /// Removes redundant entries from this group. fn remove_redundant_entries(&mut self) { - // Remove duplicate entries from each group. - self.inner = self - .inner - .iter() - .filter_map(|eq_group| { - let unique_eq_group = deduplicate_physical_exprs(eq_group); - // Keep groups that have at least 2 entries - (unique_eq_group.len() > 1).then_some(unique_eq_group) - }) - .collect(); - // Bridge groups that have common expressions - self.bridge_groups() - } + // Remove duplicate entries from each equivalence class: + self.classes.retain_mut(|class| { + // Keep groups that have at least two entries: + deduplicate_physical_exprs(class); + class.len() > 1 + }); + // Unify/bridge groups that have common expressions: + self.bridge_classes() + } + + /// This utility function unifies/bridges classes that have common expressions. + fn bridge_classes(&mut self) { + let mut i = 0; + while i < self.classes.len() { + let mut j = i + 1; + let start_size = self.classes[i].len(); + while j < self.classes.len() { + if have_common_entries(&self.classes[i], &self.classes[j]) { + let extension = self.classes.swap_remove(j); + self.classes[i].extend(extension); + } else { + j += 1; + } + } - /// This utils bridges groups that have common expressions - fn bridge_groups(&mut self) { - let mut out_groups = vec![]; - for group in &self.inner { - if out_groups.is_empty() { - out_groups.push(group.clone()); - } else { - let mut bridged_group = group.clone(); - // Delete groups in the `out_groups` that have common entry with `group`. - // Append deleted groups to the `bridged_group` - out_groups.retain(|distinct_group| { - let have_common = have_common_entries(distinct_group, group); - if have_common { - bridged_group.extend(distinct_group.clone()); - } - !have_common - }); - // before inserting make sure that entries are deduplicated - let bridged_group = deduplicate_physical_exprs(&bridged_group); - out_groups.push(bridged_group); + if self.classes[i].len() > start_size { + deduplicate_physical_exprs(&mut self.classes[i]); + if self.classes[i].len() > start_size { + continue; + } } + i += 1; } - self.inner = out_groups; } - /// Extend equivalent group with other equivalent groups - fn extend(&mut self, other: EquivalentGroups) { - self.inner.extend(other.inner); + /// Extends this equivalence group with the `other`` equivalence group. + fn extend(&mut self, other: EquivalenceGroup) { + self.classes.extend(other.classes); self.remove_redundant_entries(); } - /// Normalizes physical expression according to `EquivalentClass`es inside `self.classes`. - /// expression is replaced with `EquivalentClass::head` expression if it is among `EquivalentClass::others`. + /// Normalizes the given physical expression according to this group. + /// The expression is replaced with the first expression in the equivalence + /// it matches with. pub fn normalize_expr(&self, expr: Arc) -> Arc { expr.clone() .transform(&|expr| { @@ -177,9 +179,8 @@ impl EquivalentGroups { .unwrap_or(expr) } - /// This function applies the \[`normalize_expr`] - /// function for all expression in `exprs` and returns a vector of - /// normalized physical expressions. + /// This function applies the `normalize_expr` function for all expressions + /// in `exprs` and returns the corresponding normalized physical expressions. pub fn normalize_exprs( &self, exprs: &[Arc], @@ -313,11 +314,11 @@ impl EquivalentGroups { } } - /// Projects EquivalentGroups according to projection mapping described in `source_to_target_mapping`. + /// Projects EquivalenceGroups according to projection mapping described in `source_to_target_mapping`. pub fn project( &self, source_to_target_mapping: &ProjectionMapping, - ) -> EquivalentGroups { + ) -> EquivalenceGroup { let mut projected_eq_groups = vec![]; for eq_class in self.iter() { let new_eq_class = eq_class @@ -333,7 +334,7 @@ impl EquivalentGroups { projected_eq_groups.extend(new_eq_groups); // Return projected equivalent groups - EquivalentGroups::new(projected_eq_groups) + EquivalenceGroup::new(projected_eq_groups) } /// Construct equivalent groups according to projection mapping. @@ -378,15 +379,15 @@ impl EquivalentGroups { None } - /// Combine EquivalentGroups of the given join children. + /// Combine EquivalenceGroups of the given join children. pub fn join( &self, join_type: &JoinType, - right_eq_classes: &EquivalentGroups, + right_eq_classes: &EquivalenceGroup, left_columns_len: usize, on: &[(Column, Column)], - ) -> EquivalentGroups { - let mut result = EquivalentGroups::empty(); + ) -> EquivalenceGroup { + let mut result = EquivalenceGroup::empty(); match join_type { JoinType::Inner | JoinType::Left | JoinType::Full | JoinType::Right => { result.extend(self.clone()); @@ -397,7 +398,7 @@ impl EquivalentGroups { }) .collect(); - result.extend(EquivalentGroups::new(updated_eq_classes)); + result.extend(EquivalenceGroup::new(updated_eq_classes)); } JoinType::LeftSemi | JoinType::LeftAnti => { result.extend(self.clone()); @@ -413,7 +414,7 @@ impl EquivalentGroups { let new_rhs = Arc::new(Column::new(rhs.name(), rhs.index() + left_columns_len)) as _; - result.add_equal_conditions((&new_lhs, &new_rhs)); + result.add_equal_conditions(&new_lhs, &new_rhs); }); } result @@ -625,7 +626,7 @@ impl OrderingEquivalentGroup { #[derive(Debug, Clone)] pub struct SchemaProperties { /// Keeps track of expressions that have equivalent value. - eq_groups: EquivalentGroups, + eq_groups: EquivalenceGroup, /// Keeps track of valid ordering that satisfied table. oeq_group: OrderingEquivalentGroup, /// Keeps track of expressions that have constant value. @@ -639,7 +640,7 @@ impl SchemaProperties { /// Create an empty `SchemaProperties` pub fn new(schema: SchemaRef) -> Self { Self { - eq_groups: EquivalentGroups::empty(), + eq_groups: EquivalenceGroup::empty(), oeq_group: OrderingEquivalentGroup::empty(), constants: vec![], schema, @@ -657,7 +658,7 @@ impl SchemaProperties { } /// Return a reference to the equivalent groups - pub fn eq_groups(&self) -> &EquivalentGroups { + pub fn eq_groups(&self) -> &EquivalenceGroup { &self.eq_groups } @@ -704,7 +705,7 @@ impl SchemaProperties { } /// Add new equivalent group to state. - pub fn add_equivalent_groups(&mut self, other_eq_group: EquivalentGroups) { + pub fn add_equivalent_groups(&mut self, other_eq_group: EquivalenceGroup) { self.eq_groups.extend(other_eq_group); } @@ -714,7 +715,8 @@ impl SchemaProperties { &mut self, new_conditions: (&Arc, &Arc), ) { - self.eq_groups.add_equal_conditions(new_conditions); + self.eq_groups + .add_equal_conditions(new_conditions.0, new_conditions.1); } /// Add physical expression that have constant value to the `self.constants` @@ -1581,7 +1583,7 @@ mod tests { let new_condition = (&col_b_expr, &col_a_expr); schema_properties.add_equal_conditions(new_condition); assert_eq!(schema_properties.eq_groups().len(), 1); - let eq_groups = &schema_properties.eq_groups().inner[0]; + let eq_groups = &schema_properties.eq_groups().classes[0]; assert_eq!(eq_groups.len(), 2); assert!(physical_exprs_contains(eq_groups, &col_a_expr)); assert!(physical_exprs_contains(eq_groups, &col_b_expr)); @@ -1589,7 +1591,7 @@ mod tests { let new_condition = (&col_b_expr, &col_c_expr); schema_properties.add_equal_conditions(new_condition); assert_eq!(schema_properties.eq_groups().len(), 1); - let eq_groups = &schema_properties.eq_groups().inner[0]; + let eq_groups = &schema_properties.eq_groups().classes[0]; assert_eq!(eq_groups.len(), 3); assert!(physical_exprs_contains(eq_groups, &col_a_expr)); assert!(physical_exprs_contains(eq_groups, &col_b_expr)); @@ -1605,7 +1607,7 @@ mod tests { let new_condition = (&col_x_expr, &col_a_expr); schema_properties.add_equal_conditions(new_condition); assert_eq!(schema_properties.eq_groups().len(), 1); - let eq_groups = &schema_properties.eq_groups().inner[0]; + let eq_groups = &schema_properties.eq_groups().classes[0]; assert_eq!(eq_groups.len(), 5); assert!(physical_exprs_contains(eq_groups, &col_a_expr)); assert!(physical_exprs_contains(eq_groups, &col_b_expr)); @@ -1655,7 +1657,7 @@ mod tests { input_properties.project(&source_to_target_mapping, out_schema); assert_eq!(out_properties.eq_groups().len(), 1); - let eq_class = &out_properties.eq_groups().inner[0]; + let eq_class = &out_properties.eq_groups().classes[0]; assert_eq!(eq_class.len(), 4); assert!(physical_exprs_contains(eq_class, &col_a1_expr)); assert!(physical_exprs_contains(eq_class, &col_a2_expr)); @@ -2026,13 +2028,14 @@ mod tests { // where there is no common entry between any groups. // Since we do check for vector equality, this version should be used during comparison in the test. let expected = vec![ + vec![lit(1), lit(2), lit(3), lit(5), lit(4), lit(6), lit(7)], vec![lit(11), lit(12), lit(9)], - vec![lit(7), lit(6), lit(5), lit(2), lit(4), lit(1), lit(3)], ]; - let mut eq_groups = EquivalentGroups::new(entries); - eq_groups.bridge_groups(); - let eq_groups = eq_groups.inner; + let mut eq_groups = EquivalenceGroup::new(entries); + eq_groups.bridge_classes(); + + let eq_groups = eq_groups.classes; assert_eq!(eq_groups.len(), expected.len()); assert_eq!(eq_groups.len(), 2); @@ -2053,10 +2056,10 @@ mod tests { // where there is no common entry between any groups. // Since we do check for vector equality, this version should be used during comparison in the test. let expected = vec![vec![lit(1), lit(2)], vec![lit(4), lit(5), lit(6)]]; - let mut eq_groups = EquivalentGroups::new(entries); + let mut eq_groups = EquivalenceGroup::new(entries); eq_groups.remove_redundant_entries(); - let eq_groups = eq_groups.inner; + let eq_groups = eq_groups.classes; assert_eq!(eq_groups.len(), expected.len()); assert_eq!(eq_groups.len(), 2); diff --git a/datafusion/physical-expr/src/physical_expr.rs b/datafusion/physical-expr/src/physical_expr.rs index 35e73a919ec8..36e3dbbe8e48 100644 --- a/datafusion/physical-expr/src/physical_expr.rs +++ b/datafusion/physical-expr/src/physical_expr.rs @@ -217,8 +217,8 @@ pub fn down_cast_any_ref(any: &dyn Any) -> &dyn Any { } } -/// It is similar to contains method of vector. -/// Finds whether `expr` is among `physical_exprs`. +/// This function is similar to the `contains` method of `Vec`. It finds +/// whether `expr` is among `physical_exprs`. pub fn physical_exprs_contains( physical_exprs: &[Arc], expr: &Arc, @@ -228,18 +228,24 @@ pub fn physical_exprs_contains( .any(|physical_expr| physical_expr.eq(expr)) } -/// This util removes duplicates from the `physical_exprs` vector in its argument. -/// Once we can use `HashSet` with `Arc` use it instead. -pub fn deduplicate_physical_exprs( - physical_exprs: &[Arc], -) -> Vec> { - let mut unique_physical_exprs = vec![]; - for expr in physical_exprs { - if !physical_exprs_contains(&unique_physical_exprs, expr) { - unique_physical_exprs.push(expr.clone()); +/// This utility function removes duplicates from the given `physical_exprs` +/// vector. Once we can use `HashSet`s with `Arc`, this +/// function should use a `HashSet` to reduce computational complexity. +/// +/// Note that this function does not necessarily preserve its input ordering. +pub fn deduplicate_physical_exprs(physical_exprs: &mut Vec>) { + let mut i = 0; + while i < physical_exprs.len() { + let mut j = i + 1; + while j < physical_exprs.len() { + if physical_exprs[i].eq(&physical_exprs[j]) { + physical_exprs.swap_remove(j); + } else { + j += 1; + } } + i += 1; } - unique_physical_exprs } /// Checks whether the given slices have any common entries. @@ -320,7 +326,7 @@ mod tests { let col_b_expr = Arc::new(Column::new("b", 1)) as Arc; // lit(true), lit(false), lit(4), lit(2), Col(a), Col(b) - let physical_exprs: Vec> = vec![ + let mut physical_exprs: Vec> = vec![ lit_true.clone(), lit_false.clone(), lit4.clone(), @@ -341,8 +347,8 @@ mod tests { col_b_expr.clone(), ]; // expected contains unique versions of the physical_exprs - let result = deduplicate_physical_exprs(&physical_exprs); - physical_exprs_equal(&result, &expected); + deduplicate_physical_exprs(&mut physical_exprs); + physical_exprs_equal(&physical_exprs, &expected); Ok(()) } From f5d3d98f78370fe579ceb4226cac6aea226122aa Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Mon, 23 Oct 2023 22:05:25 +0300 Subject: [PATCH 084/122] Review Part 5 --- datafusion/physical-expr/src/equivalence.rs | 116 ++++++++++---------- 1 file changed, 55 insertions(+), 61 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 000c1e04f6ca..eec4e94d9517 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -134,26 +134,25 @@ impl EquivalenceGroup { /// This utility function unifies/bridges classes that have common expressions. fn bridge_classes(&mut self) { - let mut i = 0; - while i < self.classes.len() { - let mut j = i + 1; - let start_size = self.classes[i].len(); - while j < self.classes.len() { - if have_common_entries(&self.classes[i], &self.classes[j]) { - let extension = self.classes.swap_remove(j); - self.classes[i].extend(extension); + let mut idx = 0; + while idx < self.classes.len() { + let mut next_idx = idx + 1; + let start_size = self.classes[idx].len(); + while next_idx < self.classes.len() { + if have_common_entries(&self.classes[idx], &self.classes[next_idx]) { + let extension = self.classes.swap_remove(next_idx); + self.classes[idx].extend(extension); } else { - j += 1; + next_idx += 1; } } - - if self.classes[i].len() > start_size { - deduplicate_physical_exprs(&mut self.classes[i]); - if self.classes[i].len() > start_size { + if self.classes[idx].len() > start_size { + deduplicate_physical_exprs(&mut self.classes[idx]); + if self.classes[idx].len() > start_size { continue; } } - i += 1; + idx += 1; } } @@ -165,7 +164,7 @@ impl EquivalenceGroup { /// Normalizes the given physical expression according to this group. /// The expression is replaced with the first expression in the equivalence - /// it matches with. + /// class it matches with (if any). pub fn normalize_expr(&self, expr: Arc) -> Arc { expr.clone() .transform(&|expr| { @@ -191,9 +190,11 @@ impl EquivalenceGroup { .collect() } - /// This function normalizes `sort_requirement` according to `EquivalenceClasses` in the `self`. - /// If the given sort requirement doesn't belong to equivalence set inside - /// `self`, it returns `sort_requirement` as is. + /// Normalizes the given sort requirement according to this group. + /// The underlying physical expression is replaced with the first expression + /// in the equivalence class it matches with (if any). If the underlying + /// expression does not belong to any equivalence class in this group, returns + /// the given sort requirement as is. pub fn normalize_sort_requirement( &self, mut sort_requirement: PhysicalSortRequirement, @@ -202,9 +203,11 @@ impl EquivalenceGroup { sort_requirement } - /// This function normalizes `sort_requirement` according to `EquivalenceClasses` in the `self`. - /// If the given sort requirement doesn't belong to equivalence set inside - /// `self`, it returns `sort_requirement` as is. + /// Normalizes the given sort expression according to this group. + /// The underlying physical expression is replaced with the first expression + /// in the equivalence class it matches with (if any). If the underlying + /// expression does not belong to any equivalence class in this group, returns + /// the sort expression as is. pub fn normalize_sort_expr( &self, mut sort_expr: PhysicalSortExpr, @@ -213,9 +216,9 @@ impl EquivalenceGroup { sort_expr } - /// This function applies the \[`normalize_sort_requirement`] - /// function for all sort requirements in `sort_reqs` and returns a vector of - /// normalized sort expressions. + /// This function applies the `normalize_sort_requirement` function for all + /// requirements in `sort_reqs` and returns the corresponding normalized + /// sort requirements. pub fn normalize_sort_requirements( &self, sort_reqs: &[PhysicalSortRequirement], @@ -227,24 +230,24 @@ impl EquivalenceGroup { collapse_lex_req(normalized_sort_reqs) } - /// Similar to the \[`normalize_sort_requirements`] this function normalizes - /// sort expressions in `sort_exprs` and returns a vector of - /// normalized sort expressions. + /// This function applies the `normalize_sort_expr` function for all sort + /// expressions in `sort_exprs` and returns the corresponding normalized + /// sort expressions. pub fn normalize_sort_exprs( &self, sort_exprs: &[PhysicalSortExpr], ) -> Vec { - // Convert `PhysicalSortExpr`s to `PhysicalSortRequirement`s + // Convert `PhysicalSortExpr`s to `PhysicalSortRequirement`s: let sort_requirements = PhysicalSortRequirement::from_sort_exprs(sort_exprs.iter()); let normalized_sort_requirement = self.normalize_sort_requirements(&sort_requirements); - // Convert back `PhysicalSortRequirement`s to `PhysicalSortExpr`s + // Convert back `PhysicalSortRequirement`s to `PhysicalSortExpr`s: PhysicalSortRequirement::to_sort_exprs(normalized_sort_requirement) } - /// Projects given expression according to mapping in the `source_to_target_mapping`. - /// If expression is not valid after projection returns `None`. + /// Projects `expr` according to the mapping `source_to_target_mapping`. + /// If the resulting expression is invalid after projection, returns `None`. fn project_expr( &self, source_to_target_mapping: &ProjectionMapping, @@ -253,33 +256,29 @@ impl EquivalenceGroup { let children = expr.children(); if children.is_empty() { for (source, target) in source_to_target_mapping.iter() { - // if source matches expr, expr can be projected - if source.eq(expr) { + // If we match the source, or an equivalent expression to source, + // then we can project. For example, if we have the mapping + // (a as a1, a + c) and the equivalence class (a, b), expression + // b also projects to a1. + if source.eq(expr) + || self + .get_equivalent_group(source) + .map_or(false, |group| physical_exprs_contains(&group, expr)) + { return Some(target.clone()); } - // if equivalent group of source contains expr, expr can be projected - // Assume that projection mapping is (a as a1, a+c) - // and input table is (a, b, c), where a=b - // Expression b is projected as a1 also. - else if let Some(group) = self.get_equivalent_group(source) { - if physical_exprs_contains(&group, expr) { - return Some(target.clone()); - } - } } - // After projection, expression is not valid. - None } - // All of the childrens can be projected + // Project a non-leaf expression by projecting its children. else if let Some(children) = children .into_iter() .map(|child| self.project_expr(source_to_target_mapping, &child)) .collect::>>() { - Some(expr.clone().with_new_children(children).unwrap()) - } else { - None + return Some(expr.clone().with_new_children(children).unwrap()); } + // Arriving here implies the expression was invalid after projection. + None } /// Projects given ordering according to mapping in the `source_to_target_mapping`. @@ -289,12 +288,12 @@ impl EquivalenceGroup { source_to_target_mapping: &ProjectionMapping, ordering: &[PhysicalSortExpr], ) -> Option> { - let mut res = vec![]; + let mut result = vec![]; for order in ordering { if let Some(new_expr) = self.project_expr(source_to_target_mapping, &order.expr) { - res.push(PhysicalSortExpr { + result.push(PhysicalSortExpr { expr: new_expr, options: order.options, }) @@ -307,11 +306,7 @@ impl EquivalenceGroup { break; } } - if res.is_empty() { - None - } else { - Some(res) - } + (!result.is_empty()).then_some(result) } /// Projects EquivalenceGroups according to projection mapping described in `source_to_target_mapping`. @@ -1231,16 +1226,15 @@ pub fn schema_properties_helper( oep } -/// This function constructs a duplicate-free `LexOrderingReq` by filtering out duplicate -/// entries that have same physical expression inside the given vector `input`. -/// `vec![a Some(Asc), a Some(Desc)]` is collapsed to the `vec![a Some(Asc)]`. Since -/// when same expression is already seen before, following expressions are redundant. +/// This function constructs a duplicate-free `LexOrderingReq` by filtering out +/// duplicate entries that have same physical expression inside. For example, +/// `vec![a Some(Asc), a Some(Desc)]` collapses to `vec![a Some(Asc)]`. pub fn collapse_lex_req(input: LexOrderingReq) -> LexOrderingReq { let mut output = vec![]; for item in input { - if output + if !output .iter() - .all(|elem: &PhysicalSortRequirement| !elem.expr.eq(&item.expr)) + .any(|req: &PhysicalSortRequirement| req.expr.eq(&item.expr)) { output.push(item); } From a1087a10c8e97c83c3b02a9e3654d0029eee6e96 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Tue, 24 Oct 2023 00:41:29 +0300 Subject: [PATCH 085/122] Review Part 6 --- datafusion/physical-expr/src/equivalence.rs | 149 +++++++++----------- 1 file changed, 66 insertions(+), 83 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index eec4e94d9517..9349c1319da4 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -84,11 +84,11 @@ impl EquivalenceGroup { ) { let mut first_class = None; let mut second_class = None; - for (idx, eq_class) in self.classes.iter().enumerate() { - if physical_exprs_contains(eq_class, left) { + for (idx, cls) in self.classes.iter().enumerate() { + if physical_exprs_contains(cls, left) { first_class = Some(idx); } - if physical_exprs_contains(eq_class, right) { + if physical_exprs_contains(cls, right) { second_class = Some(idx); } } @@ -123,10 +123,10 @@ impl EquivalenceGroup { /// Removes redundant entries from this group. fn remove_redundant_entries(&mut self) { // Remove duplicate entries from each equivalence class: - self.classes.retain_mut(|class| { + self.classes.retain_mut(|cls| { // Keep groups that have at least two entries: - deduplicate_physical_exprs(class); - class.len() > 1 + deduplicate_physical_exprs(cls); + cls.len() > 1 }); // Unify/bridge groups that have common expressions: self.bridge_classes() @@ -156,7 +156,7 @@ impl EquivalenceGroup { } } - /// Extends this equivalence group with the `other`` equivalence group. + /// Extends this equivalence group with the `other` equivalence group. fn extend(&mut self, other: EquivalenceGroup) { self.classes.extend(other.classes); self.remove_redundant_entries(); @@ -168,9 +168,9 @@ impl EquivalenceGroup { pub fn normalize_expr(&self, expr: Arc) -> Arc { expr.clone() .transform(&|expr| { - for class in self.iter() { - if physical_exprs_contains(class, &expr) { - return Ok(Transformed::Yes(class[0].clone())); + for cls in self.iter() { + if physical_exprs_contains(cls, &expr) { + return Ok(Transformed::Yes(cls[0].clone())); } } Ok(Transformed::No(expr)) @@ -246,23 +246,23 @@ impl EquivalenceGroup { PhysicalSortRequirement::to_sort_exprs(normalized_sort_requirement) } - /// Projects `expr` according to the mapping `source_to_target_mapping`. + /// Projects `expr` according to the given projection mapping. /// If the resulting expression is invalid after projection, returns `None`. fn project_expr( &self, - source_to_target_mapping: &ProjectionMapping, + mapping: &ProjectionMapping, expr: &Arc, ) -> Option> { let children = expr.children(); if children.is_empty() { - for (source, target) in source_to_target_mapping.iter() { + for (source, target) in mapping.iter() { // If we match the source, or an equivalent expression to source, // then we can project. For example, if we have the mapping // (a as a1, a + c) and the equivalence class (a, b), expression // b also projects to a1. if source.eq(expr) || self - .get_equivalent_group(source) + .get_equivalence_class(source) .map_or(false, |group| physical_exprs_contains(&group, expr)) { return Some(target.clone()); @@ -272,7 +272,7 @@ impl EquivalenceGroup { // Project a non-leaf expression by projecting its children. else if let Some(children) = children .into_iter() - .map(|child| self.project_expr(source_to_target_mapping, &child)) + .map(|child| self.project_expr(mapping, &child)) .collect::>>() { return Some(expr.clone().with_new_children(children).unwrap()); @@ -281,100 +281,83 @@ impl EquivalenceGroup { None } - /// Projects given ordering according to mapping in the `source_to_target_mapping`. - /// If ordering is not valid after projection returns `None`. + /// Projects `ordering` according to the given projection mapping. + /// If the resulting ordering is invalid after projection, returns `None`. fn project_ordering( &self, - source_to_target_mapping: &ProjectionMapping, + mapping: &ProjectionMapping, ordering: &[PhysicalSortExpr], ) -> Option> { - let mut result = vec![]; - for order in ordering { - if let Some(new_expr) = - self.project_expr(source_to_target_mapping, &order.expr) - { - result.push(PhysicalSortExpr { - expr: new_expr, - options: order.options, - }) - } else { - // Expression is not valid, rest of the ordering shouldn't be projected also. - // e.g if input ordering is [a ASC, b ASC, c ASC], and column b is not valid - // after projection - // we should return projected ordering as [a ASC] not as [a ASC, c ASC] even if - // column c is valid after projection. - break; - } - } + // If any sort expression is invalid after projection, rest of the + // ordering shouldn't be projected either. For example, if input ordering + // is [a ASC, b ASC, c ASC], and column b is not valid after projection, + // the result should be [a ASC], not [a ASC, c ASC], even if column c is + // valid after projection. + let result = ordering + .iter() + .map_while(|sort_expr| { + self.project_expr(mapping, &sort_expr.expr) + .map(|expr| PhysicalSortExpr { + expr, + options: sort_expr.options, + }) + }) + .collect::>(); (!result.is_empty()).then_some(result) } - /// Projects EquivalenceGroups according to projection mapping described in `source_to_target_mapping`. - pub fn project( - &self, - source_to_target_mapping: &ProjectionMapping, - ) -> EquivalenceGroup { - let mut projected_eq_groups = vec![]; - for eq_class in self.iter() { - let new_eq_class = eq_class + /// Projects this equivalence group according to the given projection mapping. + pub fn project(&self, mapping: &ProjectionMapping) -> EquivalenceGroup { + let projected_classes = self.iter().filter_map(|cls| { + let new_class = cls .iter() - .filter_map(|expr| self.project_expr(source_to_target_mapping, expr)) + .filter_map(|expr| self.project_expr(mapping, expr)) .collect::>(); - if new_eq_class.len() > 1 { - projected_eq_groups.push(new_eq_class.clone()); - } - } - let new_eq_groups = - Self::calculate_new_projection_equivalent_groups(source_to_target_mapping); - projected_eq_groups.extend(new_eq_groups); - - // Return projected equivalent groups - EquivalenceGroup::new(projected_eq_groups) - } - - /// Construct equivalent groups according to projection mapping. - /// In the result, each inner vector contains equivalents sets. Outer vector corresponds to - /// distinct equivalent groups - fn calculate_new_projection_equivalent_groups( - source_to_target_mapping: &ProjectionMapping, - ) -> Vec>> { + (new_class.len() > 1).then_some(new_class) + }); // TODO: Convert below algorithm to the version that use HashMap. - // once `Arc` can be stored in `HashMap`. - let mut res = vec![]; - for (source, target) in source_to_target_mapping { - if res.is_empty() { - res.push((source, vec![target.clone()])); + // once `Arc` can be stored in `HashMap`. + let mut new_classes = vec![]; + for (source, target) in mapping { + if new_classes.is_empty() { + new_classes.push((source, vec![target.clone()])); } - if let Some(idx) = res.iter_mut().position(|(key, _values)| key.eq(source)) { - let (_, values) = &mut res[idx]; + if let Some(idx) = new_classes.iter_mut().position(|(key, _)| key.eq(source)) + { + let (_, values) = &mut new_classes[idx]; if !physical_exprs_contains(values, target) { values.push(target.clone()); } } } - - // Filter out groups with single entry, there is nothing - // else equal to these expressions. Hence tracking them is meaningless - res.into_iter() - .filter_map(|(_key, values)| (values.len() > 1).then_some(values)) - .collect() + // Only add equivalence classes with at least two members as singleton + // equivalence classes are meaningless. + EquivalenceGroup::new( + projected_classes + .chain( + new_classes + .into_iter() + .filter_map(|(_, values)| (values.len() > 1).then_some(values)), + ) + .collect(), + ) } - /// Returns the equivalent group that contains `expr` - /// If none of the groups contains `expr`, returns None. - fn get_equivalent_group( + /// Returns the equivalence class that contains `expr`. + /// If none of the equivalence classes contains `expr`, returns `None`. + fn get_equivalence_class( &self, expr: &Arc, ) -> Option>> { - for eq_class in self.iter() { - if physical_exprs_contains(eq_class, expr) { - return Some(eq_class.to_vec()); + for cls in self.iter() { + if physical_exprs_contains(cls, expr) { + return Some(cls.to_vec()); } } None } - /// Combine EquivalenceGroups of the given join children. + /// Combine equivalence groups of the given join children. pub fn join( &self, join_type: &JoinType, From 111bb1567e113673e6c69524b5d518dc28902353 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Tue, 24 Oct 2023 00:57:11 +0300 Subject: [PATCH 086/122] Review Part 7 --- datafusion/physical-expr/src/equivalence.rs | 35 ++++++++++----------- 1 file changed, 17 insertions(+), 18 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 9349c1319da4..ea83783ac0df 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -39,6 +39,10 @@ use itertools::izip; /// in filters. pub type EquivalenceClass = Vec>; +/// Stores the mapping between source expressions and target expressions for a +/// projection. Indices in the vector corresponds to the indices after projection. +pub type ProjectionMapping = Vec<(Arc, Arc)>; + /// An `EquivalenceGroups` is a collection of `EquivalenceClass`es where each /// class represents a distinct equivalence class in a relation. #[derive(Debug, Clone)] @@ -322,9 +326,9 @@ impl EquivalenceGroup { if new_classes.is_empty() { new_classes.push((source, vec![target.clone()])); } - if let Some(idx) = new_classes.iter_mut().position(|(key, _)| key.eq(source)) + if let Some((_, values)) = + new_classes.iter_mut().find(|(key, _)| key.eq(source)) { - let (_, values) = &mut new_classes[idx]; if !physical_exprs_contains(values, target) { values.push(target.clone()); } @@ -361,48 +365,43 @@ impl EquivalenceGroup { pub fn join( &self, join_type: &JoinType, - right_eq_classes: &EquivalenceGroup, - left_columns_len: usize, + right_equivalences: &EquivalenceGroup, + left_column_count: usize, on: &[(Column, Column)], ) -> EquivalenceGroup { let mut result = EquivalenceGroup::empty(); match join_type { JoinType::Inner | JoinType::Left | JoinType::Full | JoinType::Right => { result.extend(self.clone()); - let updated_eq_classes = right_eq_classes + let updated_eq_classes = right_equivalences .iter() .map(|eq_class| { - add_offset_to_exprs(eq_class.to_vec(), left_columns_len) + add_offset_to_exprs(eq_class.to_vec(), left_column_count) }) .collect(); - result.extend(EquivalenceGroup::new(updated_eq_classes)); } JoinType::LeftSemi | JoinType::LeftAnti => { result.extend(self.clone()); } JoinType::RightSemi | JoinType::RightAnti => { - result.extend(right_eq_classes.clone()); + result.extend(right_equivalences.clone()); } } - // In the inner join, expressions in the on are equal at the resulting table. + // In we have an inner join, expressions in the "on" condition are equal + // at the resulting table. if *join_type == JoinType::Inner { - on.iter().for_each(|(lhs, rhs)| { + for (lhs, rhs) in on.iter() { + let index = rhs.index() + left_column_count; let new_lhs = Arc::new(lhs.clone()) as _; - let new_rhs = - Arc::new(Column::new(rhs.name(), rhs.index() + left_columns_len)) - as _; + let new_rhs = Arc::new(Column::new(rhs.name(), index)) as _; result.add_equal_conditions(&new_lhs, &new_rhs); - }); + } } result } } -/// Stores the mapping between source expression and target expression during projection -/// Indices in the vector corresponds to index after projection. -pub type ProjectionMapping = Vec<(Arc, Arc)>; - /// `LexOrdering` stores the lexicographical ordering for a schema. /// OrderingEquivalentClass keeps track of different alternative orderings than can /// describe the schema. From ac34dcc0d5124c305c61b24b780b86c991c24b13 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Tue, 24 Oct 2023 01:19:27 +0300 Subject: [PATCH 087/122] Review Part 8 --- datafusion/physical-expr/src/equivalence.rs | 122 ++++++++++---------- 1 file changed, 61 insertions(+), 61 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index ea83783ac0df..57b5d2fda046 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -375,9 +375,7 @@ impl EquivalenceGroup { result.extend(self.clone()); let updated_eq_classes = right_equivalences .iter() - .map(|eq_class| { - add_offset_to_exprs(eq_class.to_vec(), left_column_count) - }) + .map(|cls| add_offset_to_exprs(cls.to_vec(), left_column_count)) .collect(); result.extend(EquivalenceGroup::new(updated_eq_classes)); } @@ -402,80 +400,82 @@ impl EquivalenceGroup { } } -/// `LexOrdering` stores the lexicographical ordering for a schema. -/// OrderingEquivalentClass keeps track of different alternative orderings than can -/// describe the schema. -/// For instance, for the table below +/// An `OrderingEquivalenceGroup` object keeps track of different alternative +/// orderings than can describe a schema. For example, consider the following table: +/// +/// ```text /// |a|b|c|d| /// |1|4|3|1| /// |2|3|3|2| /// |3|1|2|2| /// |3|2|1|3| -/// both `vec![a ASC, b ASC]` and `vec![c DESC, d ASC]` describe the ordering of the table. -/// For this case, we say that `vec![a ASC, b ASC]`, and `vec![c DESC, d ASC]` are ordering equivalent. +/// ``` +/// +/// Here, both `vec![a ASC, b ASC]` and `vec![c DESC, d ASC]` describe the table +/// ordering. In this case, we say that these orderings are equivalent. #[derive(Debug, Clone, Eq, PartialEq, Hash)] -pub struct OrderingEquivalentGroup { - inner: Vec, +pub struct OrderingEquivalenceClass { + orderings: Vec, } -impl OrderingEquivalentGroup { - /// Creates new empty ordering equivalent group +impl OrderingEquivalenceClass { + /// Creates new empty ordering equivalent class. fn empty() -> Self { - OrderingEquivalentGroup { inner: vec![] } + OrderingEquivalenceClass { orderings: vec![] } } - /// Creates new ordering equivalent from given vector - pub fn new(entries: Vec) -> Self { - let mut res = OrderingEquivalentGroup { inner: entries }; - // Make sure ordering equivalences doesn't contain something redundant - res.remove_redundant_entries(); - res + /// Creates new ordering equivalence class from the given orderings. + pub fn new(orderings: Vec) -> Self { + let mut result = OrderingEquivalenceClass { orderings }; + result.remove_redundant_entries(); + result } - /// Check whether ordering is in the state. - pub fn contains(&self, other: &LexOrdering) -> bool { - self.inner.contains(other) + /// Checks whether `ordering` is a member of this equivalence class. + pub fn contains(&self, ordering: &LexOrdering) -> bool { + self.orderings.contains(ordering) } - /// Pushes new ordering to the state. - fn push(&mut self, other: LexOrdering) { - if !self.contains(&other) { - self.inner.push(other); + /// Adds `ordering` to this equivalence class. + fn push(&mut self, ordering: LexOrdering) { + if !self.contains(&ordering) { + self.orderings.push(ordering); } - // Make sure that after new entry there is no redundant - // entry in the state. + // Make sure that there are no redundant orderings: self.remove_redundant_entries(); } - /// Check whether ordering equivalent group is empty + /// Checks whether this ordering equivalence class is empty. pub fn is_empty(&self) -> bool { self.len() == 0 } + /// Returns an iterator over the equivalent orderings in this class. pub fn iter(&self) -> impl Iterator { - self.inner.iter() + self.orderings.iter() } + /// Returns an iterator over the equivalent orderings in this class. fn into_iter(self) -> impl Iterator { - self.inner.into_iter() + self.orderings.into_iter() } - /// Get length of the entries in the ordering equivalent group + /// Returns how many equivalent orderings there are in this class. pub fn len(&self) -> usize { - self.inner.len() + self.orderings.len() } - /// Extend ordering equivalent group with other group - pub fn extend(&mut self, other: OrderingEquivalentGroup) { + /// Extend this ordering equivalence class with the `other` class. + pub fn extend(&mut self, other: OrderingEquivalenceClass) { for ordering in other.iter() { if !self.contains(ordering) { - self.inner.push(ordering.clone()) + self.orderings.push(ordering.clone()) } } self.remove_redundant_entries(); } - /// Adds new ordering into the ordering equivalent group. + /// Adds new orderings into this ordering equivalence class. pub fn add_new_orderings(&mut self, orderings: &[LexOrdering]) { for ordering in orderings.iter() { self.push(ordering.clone()); @@ -502,22 +502,22 @@ impl OrderingEquivalentGroup { res.push(ordering.clone()); } } - self.inner = res; + self.orderings = res; } /// Get first ordering entry in the ordering equivalences /// This is one of the many valid orderings (if available) pub fn output_ordering(&self) -> Option> { - self.inner.first().cloned() + self.orderings.first().cloned() } // Append other as postfix to existing ordering equivalences pub fn join_postfix( &self, - other: &OrderingEquivalentGroup, - ) -> OrderingEquivalentGroup { + other: &OrderingEquivalenceClass, + ) -> OrderingEquivalenceClass { if other.is_empty() { - return OrderingEquivalentGroup::new(self.inner.clone()); + return OrderingEquivalenceClass::new(self.orderings.clone()); } let mut res = vec![]; for ordering in self.iter() { @@ -527,13 +527,13 @@ impl OrderingEquivalentGroup { res.push(new_ordering) } } - OrderingEquivalentGroup::new(res) + OrderingEquivalenceClass::new(res) } /// Adds `offset` value to the index of each expression inside `OrderingEquivalentGroup`. - pub fn add_offset(&self, offset: usize) -> OrderingEquivalentGroup { - OrderingEquivalentGroup::new( - self.inner + pub fn add_offset(&self, offset: usize) -> OrderingEquivalenceClass { + OrderingEquivalenceClass::new( + self.orderings .iter() .map(|ordering| add_offset_to_lex_ordering(ordering, offset)) .collect(), @@ -605,7 +605,7 @@ pub struct SchemaProperties { /// Keeps track of expressions that have equivalent value. eq_groups: EquivalenceGroup, /// Keeps track of valid ordering that satisfied table. - oeq_group: OrderingEquivalentGroup, + oeq_group: OrderingEquivalenceClass, /// Keeps track of expressions that have constant value. /// TODO: We do not need to track constants separately, they can be tracked /// inside `eq_groups` as `Literal` expressions. @@ -618,7 +618,7 @@ impl SchemaProperties { pub fn new(schema: SchemaRef) -> Self { Self { eq_groups: EquivalenceGroup::empty(), - oeq_group: OrderingEquivalentGroup::empty(), + oeq_group: OrderingEquivalenceClass::empty(), constants: vec![], schema, } @@ -630,7 +630,7 @@ impl SchemaProperties { } /// Return a reference to the ordering equivalent group - pub fn oeq_group(&self) -> &OrderingEquivalentGroup { + pub fn oeq_group(&self) -> &OrderingEquivalenceClass { &self.oeq_group } @@ -642,7 +642,7 @@ impl SchemaProperties { /// Return the normalized version of the ordering equivalent group /// Where constants, duplicates are removed and expressions are normalized /// according to equivalent groups. - pub fn normalized_oeq_group(&self) -> OrderingEquivalentGroup { + pub fn normalized_oeq_group(&self) -> OrderingEquivalenceClass { // Construct a new ordering group that is normalized // With equivalences, and constants are removed let normalized_orderings = self @@ -650,7 +650,7 @@ impl SchemaProperties { .iter() .map(|ordering| self.normalize_sort_exprs(ordering)) .collect::>(); - OrderingEquivalentGroup::new(normalized_orderings) + OrderingEquivalenceClass::new(normalized_orderings) } /// Add SchemaProperties of the other to the state. @@ -662,13 +662,13 @@ impl SchemaProperties { /// Empties the `oeq_group` inside self, When existing orderings are invalidated. pub fn with_empty_ordering_equivalence(mut self) -> Self { - self.oeq_group = OrderingEquivalentGroup::empty(); + self.oeq_group = OrderingEquivalenceClass::empty(); self } /// Extends `SchemaProperties` by adding ordering inside the `other` /// to the `self.oeq_class`. - pub fn add_ordering_equivalent_group(&mut self, other: OrderingEquivalentGroup) { + pub fn add_ordering_equivalent_group(&mut self, other: OrderingEquivalenceClass) { for ordering in other.into_iter() { if !self.oeq_group.contains(&ordering) { self.oeq_group.push(ordering); @@ -715,7 +715,7 @@ impl SchemaProperties { // Reset ordering equivalent group with the new ordering. // Constants, and equivalent groups are still valid after re-sort. // Hence only `oeq_group` is overwritten. - self.oeq_group = OrderingEquivalentGroup::new(vec![sort_expr]); + self.oeq_group = OrderingEquivalenceClass::new(vec![sort_expr]); self } @@ -979,7 +979,7 @@ impl SchemaProperties { // if empty, no need to track projected_orderings. if !projected_orderings.is_empty() { projected_properties.oeq_group = - OrderingEquivalentGroup::new(projected_orderings); + OrderingEquivalenceClass::new(projected_orderings); } for (source, target) in source_to_target_mapping { @@ -1202,7 +1202,7 @@ pub fn schema_properties_helper( ) -> SchemaProperties { let mut oep = SchemaProperties::new(schema); if !orderings.is_empty() { - let group = OrderingEquivalentGroup::new(orderings.to_vec()); + let group = OrderingEquivalenceClass::new(orderings.to_vec()); oep.add_ordering_equivalent_group(group); } oep @@ -1345,9 +1345,9 @@ fn update_ordering( /// it can thereafter safely be used for ordering equivalence normalization. fn get_updated_right_ordering_equivalent_group( join_type: &JoinType, - right_oeq_group: &OrderingEquivalentGroup, + right_oeq_group: &OrderingEquivalenceClass, left_columns_len: usize, -) -> OrderingEquivalentGroup { +) -> OrderingEquivalenceClass { if matches!( join_type, JoinType::Inner | JoinType::Left | JoinType::Full | JoinType::Right @@ -2049,7 +2049,7 @@ mod tests { let join_type = JoinType::Inner; let options = SortOptions::default(); - let right_oeq_class = OrderingEquivalentGroup::new(vec![ + let right_oeq_class = OrderingEquivalenceClass::new(vec![ vec![ PhysicalSortExpr { expr: Arc::new(Column::new("x", 0)), @@ -2099,7 +2099,7 @@ mod tests { join_schema_properties.add_ordering_equivalent_group(result); let result = join_schema_properties.oeq_group().clone(); - let expected = OrderingEquivalentGroup::new(vec![ + let expected = OrderingEquivalenceClass::new(vec![ vec![ PhysicalSortExpr { expr: col_x_expr, From 3721f747f39ced41a43d4d4ee0f43a7f6eaa3ebd Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 24 Oct 2023 09:39:58 +0300 Subject: [PATCH 088/122] Update comments --- datafusion/common/src/join_type.rs | 2 +- datafusion/physical-expr/src/equivalence.rs | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/datafusion/common/src/join_type.rs b/datafusion/common/src/join_type.rs index fd0bdd8ac1eb..0a00a57ba45f 100644 --- a/datafusion/common/src/join_type.rs +++ b/datafusion/common/src/join_type.rs @@ -106,7 +106,7 @@ impl Display for JoinSide { } /// Join side. -/// Used in `ColumnIndex` type to distinguish which side the index is for. +/// Stores the referred table side during calculations #[derive(Debug, Clone, Copy, PartialEq)] pub enum JoinSide { /// Left side of the join diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 57b5d2fda046..6d9918ab6232 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -43,7 +43,7 @@ pub type EquivalenceClass = Vec>; /// projection. Indices in the vector corresponds to the indices after projection. pub type ProjectionMapping = Vec<(Arc, Arc)>; -/// An `EquivalenceGroups` is a collection of `EquivalenceClass`es where each +/// An `EquivalenceGroup` is a collection of `EquivalenceClass`es where each /// class represents a distinct equivalence class in a relation. #[derive(Debug, Clone)] pub struct EquivalenceGroup { @@ -400,7 +400,7 @@ impl EquivalenceGroup { } } -/// An `OrderingEquivalenceGroup` object keeps track of different alternative +/// An `OrderingEquivalenceClass` object keeps track of different alternative /// orderings than can describe a schema. For example, consider the following table: /// /// ```text From c1f9e172cafc2c3280a20b371c43a5102d39dcc1 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 24 Oct 2023 14:53:48 +0300 Subject: [PATCH 089/122] Add new unit tests, simplifications --- .../enforce_distribution.rs | 4 +- datafusion/physical-expr/src/equivalence.rs | 488 +++++++++++------- datafusion/physical-expr/src/partitioning.rs | 2 +- datafusion/physical-expr/src/physical_expr.rs | 68 +-- datafusion/physical-expr/src/utils.rs | 10 +- .../physical-plan/src/aggregates/mod.rs | 4 +- .../physical-plan/src/aggregates/row_hash.rs | 3 +- datafusion/physical-plan/src/filter.rs | 2 +- datafusion/physical-plan/src/memory.rs | 2 +- datafusion/physical-plan/src/projection.rs | 2 +- datafusion/physical-plan/src/union.rs | 6 +- 11 files changed, 347 insertions(+), 244 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index 0bbaae827291..cccd6c2cde65 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -752,7 +752,7 @@ fn try_reorder( expected: &[Arc], equivalence_properties: &SchemaProperties, ) -> Option<(JoinKeyPairs, Vec)> { - let eq_groups = equivalence_properties.eq_groups(); + let eq_groups = equivalence_properties.eq_group(); let mut normalized_expected = vec![]; let mut normalized_left_keys = vec![]; let mut normalized_right_keys = vec![]; @@ -763,7 +763,7 @@ fn try_reorder( || physical_exprs_equal(expected, &join_keys.right_keys) { return Some((join_keys, vec![])); - } else if !equivalence_properties.eq_groups().is_empty() { + } else if !equivalence_properties.eq_group().is_empty() { normalized_expected = expected .iter() .map(|e| eq_groups.normalize_expr(e.clone())) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 6d9918ab6232..239372b99c99 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -101,9 +101,12 @@ impl EquivalenceGroup { // If the given left and right sides belong to different classes, // we should unify/bridge these classes. if first_idx != second_idx { + // By convention make sure second_idx is larger than first_idx. if first_idx > second_idx { (first_idx, second_idx) = (second_idx, first_idx); } + // Remove second_idx from self.classes then merge its values with class at first_idx. + // Convention above makes sure that first_idx is still valid after second_idx removal. let other_class = self.classes.swap_remove(second_idx); self.classes[first_idx].extend(other_class); } @@ -186,11 +189,11 @@ impl EquivalenceGroup { /// in `exprs` and returns the corresponding normalized physical expressions. pub fn normalize_exprs( &self, - exprs: &[Arc], + exprs: Vec>, ) -> Vec> { exprs - .iter() - .map(|expr| self.normalize_expr(expr.clone())) + .into_iter() + .map(|expr| self.normalize_expr(expr)) .collect() } @@ -267,7 +270,7 @@ impl EquivalenceGroup { if source.eq(expr) || self .get_equivalence_class(source) - .map_or(false, |group| physical_exprs_contains(&group, expr)) + .map_or(false, |group| physical_exprs_contains(group, expr)) { return Some(target.clone()); } @@ -352,13 +355,10 @@ impl EquivalenceGroup { fn get_equivalence_class( &self, expr: &Arc, - ) -> Option>> { - for cls in self.iter() { - if physical_exprs_contains(cls, expr) { - return Some(cls.to_vec()); - } - } - None + ) -> Option<&[Arc]> { + self.iter() + .map(|cls| cls.as_slice()) + .find(|cls| physical_exprs_contains(cls, expr)) } /// Combine equivalence groups of the given join children. @@ -440,9 +440,9 @@ impl OrderingEquivalenceClass { fn push(&mut self, ordering: LexOrdering) { if !self.contains(&ordering) { self.orderings.push(ordering); + // Make sure that there are no redundant orderings: + self.remove_redundant_entries(); } - // Make sure that there are no redundant orderings: - self.remove_redundant_entries(); } /// Checks whether this ordering equivalence class is empty. @@ -472,7 +472,6 @@ impl OrderingEquivalenceClass { self.orderings.push(ordering.clone()) } } - self.remove_redundant_entries(); } /// Adds new orderings into this ordering equivalence class. @@ -480,7 +479,6 @@ impl OrderingEquivalenceClass { for ordering in orderings.iter() { self.push(ordering.clone()); } - self.remove_redundant_entries(); } /// Removes redundant orderings from the state. @@ -494,7 +492,7 @@ impl OrderingEquivalenceClass { let mut is_inside = false; for item in &mut res { if let Some(finer) = Self::get_finer_strict(item, ordering) { - *item = finer; + *item = finer.to_vec(); is_inside = true; } } @@ -541,15 +539,15 @@ impl OrderingEquivalenceClass { } /// Return finer ordering between lhs and rhs. - fn get_finer_strict( - lhs: &[PhysicalSortExpr], - rhs: &[PhysicalSortExpr], - ) -> Option> { + fn get_finer_strict<'a>( + lhs: &'a [PhysicalSortExpr], + rhs: &'a [PhysicalSortExpr], + ) -> Option<&'a [PhysicalSortExpr]> { if izip!(lhs.iter(), rhs.iter()).all(|(lhs, rhs)| lhs.eq(rhs)) { if lhs.len() > rhs.len() { - return Some(lhs.to_vec()); + return Some(lhs); } else { - return Some(rhs.to_vec()); + return Some(rhs); } } None @@ -603,9 +601,9 @@ impl OrderingEquivalenceClass { #[derive(Debug, Clone)] pub struct SchemaProperties { /// Keeps track of expressions that have equivalent value. - eq_groups: EquivalenceGroup, + eq_group: EquivalenceGroup, /// Keeps track of valid ordering that satisfied table. - oeq_group: OrderingEquivalenceClass, + oeq_class: OrderingEquivalenceClass, /// Keeps track of expressions that have constant value. /// TODO: We do not need to track constants separately, they can be tracked /// inside `eq_groups` as `Literal` expressions. @@ -617,8 +615,8 @@ impl SchemaProperties { /// Create an empty `SchemaProperties` pub fn new(schema: SchemaRef) -> Self { Self { - eq_groups: EquivalenceGroup::empty(), - oeq_group: OrderingEquivalenceClass::empty(), + eq_group: EquivalenceGroup::empty(), + oeq_class: OrderingEquivalenceClass::empty(), constants: vec![], schema, } @@ -629,24 +627,24 @@ impl SchemaProperties { &self.schema } - /// Return a reference to the ordering equivalent group - pub fn oeq_group(&self) -> &OrderingEquivalenceClass { - &self.oeq_group + /// Return a reference to the ordering equivalence class + pub fn oeq_class(&self) -> &OrderingEquivalenceClass { + &self.oeq_class } - /// Return a reference to the equivalent groups - pub fn eq_groups(&self) -> &EquivalenceGroup { - &self.eq_groups + /// Return a reference to the equivalent group + pub fn eq_group(&self) -> &EquivalenceGroup { + &self.eq_group } - /// Return the normalized version of the ordering equivalent group + /// Return the normalized version of the ordering equivalence class /// Where constants, duplicates are removed and expressions are normalized - /// according to equivalent groups. - pub fn normalized_oeq_group(&self) -> OrderingEquivalenceClass { + /// according to equivalent group. + pub fn normalized_oeq_class(&self) -> OrderingEquivalenceClass { // Construct a new ordering group that is normalized // With equivalences, and constants are removed let normalized_orderings = self - .oeq_group + .oeq_class .iter() .map(|ordering| self.normalize_sort_exprs(ordering)) .collect::>(); @@ -655,14 +653,14 @@ impl SchemaProperties { /// Add SchemaProperties of the other to the state. pub fn extend(mut self, other: SchemaProperties) -> Self { - self.eq_groups.extend(other.eq_groups); - self.oeq_group.extend(other.oeq_group); + self.eq_group.extend(other.eq_group); + self.oeq_class.extend(other.oeq_class); self.with_constants(other.constants) } - /// Empties the `oeq_group` inside self, When existing orderings are invalidated. + /// Empties the `oeq_class` inside self, When existing orderings are invalidated. pub fn with_empty_ordering_equivalence(mut self) -> Self { - self.oeq_group = OrderingEquivalenceClass::empty(); + self.oeq_class = OrderingEquivalenceClass::empty(); self } @@ -670,35 +668,35 @@ impl SchemaProperties { /// to the `self.oeq_class`. pub fn add_ordering_equivalent_group(&mut self, other: OrderingEquivalenceClass) { for ordering in other.into_iter() { - if !self.oeq_group.contains(&ordering) { - self.oeq_group.push(ordering); + if !self.oeq_class.contains(&ordering) { + self.oeq_class.push(ordering); } } } - /// Adds new ordering into the ordering equivalent group. + /// Adds new ordering into the ordering equivalent class. pub fn add_new_orderings(&mut self, orderings: &[LexOrdering]) { - self.oeq_group.add_new_orderings(orderings); + self.oeq_class.add_new_orderings(orderings); } - /// Add new equivalent group to state. + /// Add new equivalence group to state. pub fn add_equivalent_groups(&mut self, other_eq_group: EquivalenceGroup) { - self.eq_groups.extend(other_eq_group); + self.eq_group.extend(other_eq_group); } /// Adds new equality group into the equivalent groups. /// If equalities are new, otherwise extends corresponding group. pub fn add_equal_conditions( &mut self, - new_conditions: (&Arc, &Arc), + left: &Arc, + right: &Arc, ) { - self.eq_groups - .add_equal_conditions(new_conditions.0, new_conditions.1); + self.eq_group.add_equal_conditions(left, right); } /// Add physical expression that have constant value to the `self.constants` pub fn with_constants(mut self, constants: Vec>) -> Self { - let constants = self.eq_groups.normalize_exprs(&constants); + let constants = self.eq_group.normalize_exprs(constants); constants.into_iter().for_each(|constant| { if !physical_exprs_contains(&self.constants, &constant) { self.constants.push(constant); @@ -715,7 +713,7 @@ impl SchemaProperties { // Reset ordering equivalent group with the new ordering. // Constants, and equivalent groups are still valid after re-sort. // Hence only `oeq_group` is overwritten. - self.oeq_group = OrderingEquivalenceClass::new(vec![sort_expr]); + self.oeq_class = OrderingEquivalenceClass::new(vec![sort_expr]); self } @@ -757,8 +755,8 @@ impl SchemaProperties { &self, sort_reqs: &[PhysicalSortRequirement], ) -> Vec { - let normalized_sort_reqs = self.eq_groups.normalize_sort_requirements(sort_reqs); - let constants_normalized = self.eq_groups.normalize_exprs(&self.constants); + let normalized_sort_reqs = self.eq_group.normalize_sort_requirements(sort_reqs); + let constants_normalized = self.eq_group.normalize_exprs(self.constants.clone()); let normalized_sort_reqs = prune_sort_reqs_with_constants(&normalized_sort_reqs, &constants_normalized); // Prune redundant sections in the requirement. @@ -869,9 +867,9 @@ impl SchemaProperties { let lhs = self.normalize_sort_exprs(req1); let rhs = self.normalize_sort_exprs(req2); let mut meet = vec![]; - for (lhs, rhs) in izip!(lhs.iter(), rhs.iter()) { - if lhs.eq(rhs) { - meet.push(lhs.clone()); + for (lhs, rhs) in izip!(lhs.into_iter(), rhs.into_iter()) { + if lhs.eq(&rhs) { + meet.push(lhs); } else { break; } @@ -894,7 +892,7 @@ impl SchemaProperties { return normalized_sort_req; } - for ordering in self.normalized_oeq_group().iter() { + for ordering in self.normalized_oeq_class().iter() { let match_indices = ordering .iter() .map(|elem| { @@ -953,7 +951,7 @@ impl SchemaProperties { source_to_target_mapping: &ProjectionMapping, expr: &Arc, ) -> Option> { - self.eq_groups.project_expr(source_to_target_mapping, expr) + self.eq_group.project_expr(source_to_target_mapping, expr) } /// Projects `SchemaProperties` according to mapping given in `source_to_target_mapping`. @@ -964,21 +962,21 @@ impl SchemaProperties { ) -> SchemaProperties { let mut projected_properties = SchemaProperties::new(output_schema); - let projected_eq_groups = self.eq_groups.project(source_to_target_mapping); - projected_properties.eq_groups = projected_eq_groups; + let projected_eq_groups = self.eq_group.project(source_to_target_mapping); + projected_properties.eq_group = projected_eq_groups; let projected_orderings = self - .oeq_group + .oeq_class .iter() .filter_map(|order| { - self.eq_groups + self.eq_group .project_ordering(source_to_target_mapping, order) }) .collect::>(); // if empty, no need to track projected_orderings. if !projected_orderings.is_empty() { - projected_properties.oeq_group = + projected_properties.oeq_class = OrderingEquivalenceClass::new(projected_orderings); } @@ -993,7 +991,7 @@ impl SchemaProperties { options: sort_options, }; // Push new ordering to the state. - projected_properties.oeq_group.push(vec![sort_expr]); + projected_properties.oeq_class.push(vec![sort_expr]); } } // Remove redundant entries from ordering group if any. @@ -1004,22 +1002,12 @@ impl SchemaProperties { /// Check whether any permutation of the argument has a prefix with existing ordering. /// Return indices that describes ordering and their ordering information. pub fn set_satisfy(&self, exprs: &[Arc]) -> Option> { - let normalized_exprs = self.eq_groups.normalize_exprs(exprs); - if let Some(ordered_section) = self.get_lex_ordering_section(&normalized_exprs) { - let mut indices = vec![]; - for sort_expr in ordered_section { - if let Some(position) = - exprs.iter().position(|expr| sort_expr.expr.eq(expr)) - { - indices.push(position); - } else { - panic!("expects to find all of the ordered section inside normalized_exprs"); - } - } - Some(indices) - } else { - None - } + let ordered_section = self.get_lex_ordering_section(exprs); + // Get indices of the ordered_section inside exprs + ordered_section + .into_iter() + .map(|sort_expr| exprs.iter().position(|expr| sort_expr.expr.eq(expr))) + .collect::>>() } /// Check whether one of the permutation of the exprs satisfies existing ordering. @@ -1046,48 +1034,25 @@ impl SchemaProperties { &self, exprs: &[Arc], ) -> Option> { - let mut normalized_exprs = self.eq_groups.normalize_exprs(exprs); - let mut ordered_exprs: Vec = vec![]; - for ordering in self.normalized_oeq_group().iter() { - for sort_expr in ordering { - if let Some(idx) = normalized_exprs.iter().position(|normalized_expr| { - sort_expr.satisfy_with_schema( - &PhysicalSortRequirement { - expr: normalized_expr.clone(), - options: None, - }, - &self.schema, - ) - }) { - ordered_exprs.push(PhysicalSortExpr { - expr: normalized_exprs[idx].clone(), - options: sort_expr.options, - }); - normalized_exprs.remove(idx); - } else { - // Should find in consecutive chunks - break; - } - } - } - if normalized_exprs.is_empty() { - Some(ordered_exprs) - } else { - None + let ordered_section = self.get_lex_ordering_section(exprs); + if ordered_section.len() == exprs.len() { + return Some(ordered_section); } + None } /// Get ordering of the expressions in the argument - /// Assumes arguments define lexicographical ordering. - /// None, represents none of the existing ordering satisfy - /// lexicographical ordering of the exprs. + /// If a subset of the arguments defines ordering, that section is returned + /// Even if not all of the expression are ordered. pub fn get_lex_ordering_section( &self, exprs: &[Arc], - ) -> Option> { - let mut normalized_exprs = self.eq_groups.normalize_exprs(exprs); - let mut ordered_exprs: Vec = vec![]; - for ordering in self.normalized_oeq_group().iter() { + ) -> Vec { + let normalized_exprs = self.eq_group.normalize_exprs(exprs.to_vec()); + // After normalization size of the vector shouldn't change. + assert_eq!(normalized_exprs.len(), exprs.len()); + let mut ordered_exprs: Vec<(usize, SortOptions)> = vec![]; + for ordering in self.normalized_oeq_class().iter() { for sort_expr in ordering { if let Some(idx) = normalized_exprs.iter().position(|normalized_expr| { sort_expr.satisfy_with_schema( @@ -1098,22 +1063,26 @@ impl SchemaProperties { &self.schema, ) }) { - ordered_exprs.push(PhysicalSortExpr { - expr: normalized_exprs[idx].clone(), - options: sort_expr.options, - }); - normalized_exprs.remove(idx); + // We did not insert this entry before. + if ordered_exprs + .iter() + .all(|(existing_idx, _opt)| *existing_idx != idx) + { + ordered_exprs.push((idx, sort_expr.options)); + } } else { // Should find in consecutive chunks break; } } } - if !ordered_exprs.is_empty() { - Some(ordered_exprs) - } else { - None - } + ordered_exprs + .into_iter() + .map(|(idx, options)| PhysicalSortExpr { + expr: exprs[idx].clone(), + options, + }) + .collect::>() } } @@ -1131,16 +1100,16 @@ pub fn join_schema_properties( let mut new_properties = SchemaProperties::new(join_schema); let join_eq_groups = - left.eq_groups() - .join(join_type, right.eq_groups(), left_columns_len, on); + left.eq_group() + .join(join_type, right.eq_group(), left_columns_len, on); new_properties.add_equivalent_groups(join_eq_groups); // All joins have 2 children assert_eq!(maintains_input_order.len(), 2); let left_maintains = maintains_input_order[0]; let right_maintains = maintains_input_order[1]; - let left_oeq_class = left.oeq_group(); - let right_oeq_class = right.oeq_group(); + let left_oeq_class = left.oeq_class(); + let right_oeq_class = right.oeq_class(); match (left_maintains, right_maintains) { (true, true) => { unreachable!("Cannot maintain ordering of both sides"); @@ -1171,7 +1140,7 @@ pub fn join_schema_properties( (false, true) => { let updated_right_oeq = get_updated_right_ordering_equivalent_group( join_type, - right.oeq_group(), + right.oeq_class(), left_columns_len, ); // In this special case, left side ordering can be prefixed with right side ordering. @@ -1306,8 +1275,8 @@ fn update_ordering( mut node: ExprOrdering, ordering_equal_properties: &SchemaProperties, ) -> Result> { - let eq_groups = &ordering_equal_properties.eq_groups; - let oeq_group = &ordering_equal_properties.oeq_group; + let eq_groups = &ordering_equal_properties.eq_group; + let oeq_group = &ordering_equal_properties.oeq_class; if let Some(children_sort_options) = &node.children_states { // We have an intermediate (non-leaf) node, account for its children: node.state = Some(node.expr.get_ordering(children_sort_options)); @@ -1376,7 +1345,7 @@ mod tests { use rand::seq::SliceRandom; use rand::{Rng, SeedableRng}; - // Generate a schema which consists of 7 columns (a, b, c, d, e, f, g) + // Generate a schema which consists of 8 columns (a, b, c, d, e, f, g, h) fn create_test_schema() -> Result { let a = Field::new("a", DataType::Int32, true); let b = Field::new("b", DataType::Int32, true); @@ -1385,13 +1354,14 @@ mod tests { let e = Field::new("e", DataType::Int32, true); let f = Field::new("f", DataType::Int32, true); let g = Field::new("g", DataType::Int32, true); - let schema = Arc::new(Schema::new(vec![a, b, c, d, e, f, g])); + let h = Field::new("h", DataType::Int32, true); + let schema = Arc::new(Schema::new(vec![a, b, c, d, e, f, g, h])); Ok(schema) } /// Construct a schema with following properties - /// Schema satisfied following orderings: + /// Schema satisfies following orderings: /// [a ASC], [d ASC, b ASC], [e DESC, f ASC, g ASC] /// and /// Column [a=c] (e.g they are aliases). @@ -1405,7 +1375,7 @@ mod tests { let col_f_expr = &col("f", &test_schema)?; let col_g_expr = &col("g", &test_schema)?; let mut schema_properties = SchemaProperties::new(test_schema.clone()); - schema_properties.add_equal_conditions((col_a_expr, col_c_expr)); + schema_properties.add_equal_conditions(col_a_expr, col_c_expr); let option1 = SortOptions { descending: false, @@ -1480,7 +1450,7 @@ mod tests { let mut schema_properties = SchemaProperties::new(test_schema.clone()); // Define equivalent columns and constant columns - schema_properties.add_equal_conditions((&col_exprs[0], &col_exprs[5])); + schema_properties.add_equal_conditions(&col_exprs[0], &col_exprs[5]); schema_properties = schema_properties.with_constants(vec![col_exprs[4].clone()]); // Randomly order columns for sorting @@ -1510,6 +1480,19 @@ mod tests { Ok((test_schema, schema_properties)) } + /// Checks whether the given physical expression slices are equal. + pub fn physical_exprs_set_equal( + lhs: &[Arc], + rhs: &[Arc], + ) -> bool { + if lhs.len() != rhs.len() { + false + } else { + lhs.iter() + .all(|lhs_entry| physical_exprs_contains(rhs, lhs_entry)) + } + } + // Convert each tuple to PhysicalSortRequirement fn convert_to_sort_reqs( in_data: &[(&Arc, Option)], @@ -1552,38 +1535,33 @@ mod tests { let col_x_expr = Arc::new(Column::new("x", 3)) as Arc; let col_y_expr = Arc::new(Column::new("y", 4)) as Arc; - let new_condition = (&col_a_expr, &col_b_expr); - schema_properties.add_equal_conditions(new_condition); - assert_eq!(schema_properties.eq_groups().len(), 1); + schema_properties.add_equal_conditions(&col_a_expr, &col_b_expr); + assert_eq!(schema_properties.eq_group().len(), 1); - let new_condition = (&col_b_expr, &col_a_expr); - schema_properties.add_equal_conditions(new_condition); - assert_eq!(schema_properties.eq_groups().len(), 1); - let eq_groups = &schema_properties.eq_groups().classes[0]; + schema_properties.add_equal_conditions(&col_b_expr, &col_a_expr); + assert_eq!(schema_properties.eq_group().len(), 1); + let eq_groups = &schema_properties.eq_group().classes[0]; assert_eq!(eq_groups.len(), 2); assert!(physical_exprs_contains(eq_groups, &col_a_expr)); assert!(physical_exprs_contains(eq_groups, &col_b_expr)); - let new_condition = (&col_b_expr, &col_c_expr); - schema_properties.add_equal_conditions(new_condition); - assert_eq!(schema_properties.eq_groups().len(), 1); - let eq_groups = &schema_properties.eq_groups().classes[0]; + schema_properties.add_equal_conditions(&col_b_expr, &col_c_expr); + assert_eq!(schema_properties.eq_group().len(), 1); + let eq_groups = &schema_properties.eq_group().classes[0]; assert_eq!(eq_groups.len(), 3); assert!(physical_exprs_contains(eq_groups, &col_a_expr)); assert!(physical_exprs_contains(eq_groups, &col_b_expr)); assert!(physical_exprs_contains(eq_groups, &col_c_expr)); // This is a new set of equality. Hence equivalent class count should be 2. - let new_condition = (&col_x_expr, &col_y_expr); - schema_properties.add_equal_conditions(new_condition); - assert_eq!(schema_properties.eq_groups().len(), 2); + schema_properties.add_equal_conditions(&col_x_expr, &col_y_expr); + assert_eq!(schema_properties.eq_group().len(), 2); // This equality bridges distinct equality sets. // Hence equivalent class count should decrease from 2 to 1. - let new_condition = (&col_x_expr, &col_a_expr); - schema_properties.add_equal_conditions(new_condition); - assert_eq!(schema_properties.eq_groups().len(), 1); - let eq_groups = &schema_properties.eq_groups().classes[0]; + schema_properties.add_equal_conditions(&col_x_expr, &col_a_expr); + assert_eq!(schema_properties.eq_group().len(), 1); + let eq_groups = &schema_properties.eq_group().classes[0]; assert_eq!(eq_groups.len(), 5); assert!(physical_exprs_contains(eq_groups, &col_a_expr)); assert!(physical_exprs_contains(eq_groups, &col_b_expr)); @@ -1607,10 +1585,8 @@ mod tests { let col_b_expr = Arc::new(Column::new("b", 1)) as Arc; let col_c_expr = Arc::new(Column::new("c", 2)) as Arc; - let new_condition = (&col_a_expr, &col_b_expr); - input_properties.add_equal_conditions(new_condition); - let new_condition = (&col_b_expr, &col_c_expr); - input_properties.add_equal_conditions(new_condition); + input_properties.add_equal_conditions(&col_a_expr, &col_b_expr); + input_properties.add_equal_conditions(&col_b_expr, &col_c_expr); let out_schema = Arc::new(Schema::new(vec![ Field::new("a1", DataType::Int64, true), @@ -1632,8 +1608,8 @@ mod tests { let out_properties = input_properties.project(&source_to_target_mapping, out_schema); - assert_eq!(out_properties.eq_groups().len(), 1); - let eq_class = &out_properties.eq_groups().classes[0]; + assert_eq!(out_properties.eq_group().len(), 1); + let eq_class = &out_properties.eq_group().classes[0]; assert_eq!(eq_class.len(), 4); assert!(physical_exprs_contains(eq_class, &col_a1_expr)); assert!(physical_exprs_contains(eq_class, &col_a2_expr)); @@ -1662,12 +1638,12 @@ mod tests { // finer ordering satisfies, crude ordering shoul return true let empty_schema = &Arc::new(Schema::empty()); let mut schema_properties = SchemaProperties::new(empty_schema.clone()); - schema_properties.oeq_group.push(finer.clone()); + schema_properties.oeq_class.push(finer.clone()); assert!(schema_properties.ordering_satisfy(&crude)); // Crude ordering doesn't satisfy finer ordering. should return false let mut schema_properties = SchemaProperties::new(empty_schema.clone()); - schema_properties.oeq_group.push(crude.clone()); + schema_properties.oeq_class.push(crude.clone()); assert!(!schema_properties.ordering_satisfy(&finer)); Ok(()) } @@ -1902,7 +1878,7 @@ mod tests { }; // Column a and c are aliases. let mut schema_properties = SchemaProperties::new(test_schema); - schema_properties.add_equal_conditions((&col_a, &col_c)); + schema_properties.add_equal_conditions(&col_a, &col_c); // Column a and e are ordering equivalent (e.g global ordering of the table can be described both as a ASC and e ASC.) schema_properties.add_new_orderings(&[ @@ -1994,29 +1970,46 @@ mod tests { #[test] fn test_bridge_groups() -> Result<()> { - let entries = vec![ - vec![lit(1), lit(2), lit(3)], - vec![lit(2), lit(4), lit(5)], - vec![lit(11), lit(12), lit(9)], - vec![lit(7), lit(6), lit(5)], - ]; - // Expected is a bit weird. However, what we care is they expected contains distinct groups. - // where there is no common entry between any groups. - // Since we do check for vector equality, this version should be used during comparison in the test. - let expected = vec![ - vec![lit(1), lit(2), lit(3), lit(5), lit(4), lit(6), lit(7)], - vec![lit(11), lit(12), lit(9)], + // First entry in the tuple is argument, second entry is the bridged result + let test_cases = vec![ + // ------- TEST CASE 1 -----------// + ( + vec![vec![1, 2, 3], vec![2, 4, 5], vec![11, 12, 9], vec![7, 6, 5]], + // Expected is compared with set equality. Order of the specific results may change. + vec![vec![1, 2, 3, 4, 5, 6, 7], vec![9, 11, 12]], + ), + // ------- TEST CASE 2 -----------// + ( + vec![vec![1, 2, 3], vec![3, 4, 5], vec![9, 8, 7], vec![7, 6, 5]], + // Expected + vec![vec![1, 2, 3, 4, 5, 6, 7, 8, 9]], + ), ]; - - let mut eq_groups = EquivalenceGroup::new(entries); - eq_groups.bridge_classes(); - - let eq_groups = eq_groups.classes; - assert_eq!(eq_groups.len(), expected.len()); - assert_eq!(eq_groups.len(), 2); - - assert!(physical_exprs_equal(&eq_groups[0], &expected[0])); - assert!(physical_exprs_equal(&eq_groups[1], &expected[1])); + for (entries, expected) in test_cases { + let entries = entries + .into_iter() + .map(|entry| entry.into_iter().map(lit).collect::>()) + .collect::>(); + let expected = expected + .into_iter() + .map(|entry| entry.into_iter().map(lit).collect::>()) + .collect::>(); + let mut eq_groups = EquivalenceGroup::new(entries.clone()); + eq_groups.bridge_classes(); + let eq_groups = eq_groups.classes; + let err_msg = format!( + "error in test entries: {:?}, expected: {:?}, actual:{:?}", + entries, expected, eq_groups + ); + assert_eq!(eq_groups.len(), expected.len(), "{}", err_msg); + for idx in 0..eq_groups.len() { + assert!( + physical_exprs_set_equal(&eq_groups[idx], &expected[idx]), + "{}", + err_msg + ); + } + } Ok(()) } @@ -2088,8 +2081,8 @@ mod tests { let col_w_expr = col("w", &schema)?; let mut join_schema_properties = SchemaProperties::new(Arc::new(schema)); - join_schema_properties.add_equal_conditions((&col_a_expr, &col_x_expr)); - join_schema_properties.add_equal_conditions((&col_d_expr, &col_w_expr)); + join_schema_properties.add_equal_conditions(&col_a_expr, &col_x_expr); + join_schema_properties.add_equal_conditions(&col_d_expr, &col_w_expr); let result = get_updated_right_ordering_equivalent_group( &join_type, @@ -2097,7 +2090,7 @@ mod tests { left_columns_len, ); join_schema_properties.add_ordering_equivalent_group(result); - let result = join_schema_properties.oeq_group().clone(); + let result = join_schema_properties.oeq_class().clone(); let expected = OrderingEquivalenceClass::new(vec![ vec![ @@ -2233,7 +2226,7 @@ mod tests { } // Fill columns based on ordering equivalences - for ordering in schema_properties.oeq_group.iter() { + for ordering in schema_properties.oeq_class.iter() { let (sort_columns, indices): (Vec<_>, Vec<_>) = ordering .iter() .map(|PhysicalSortExpr { expr, options }| { @@ -2257,7 +2250,7 @@ mod tests { } // Fill columns based on equivalence groups - for eq_group in schema_properties.eq_groups.iter() { + for eq_group in schema_properties.eq_group.iter() { let representative_array = get_representative_arr(eq_group, &schema_vec, schema.clone()) .unwrap_or_else(|| generate_random_array(n_elem, n_distinct)); @@ -2298,7 +2291,7 @@ mod tests { // Cannot normalize column b (&col_b_expr, &col_b_expr), ]; - let eq_groups = schema_properties.eq_groups(); + let eq_groups = schema_properties.eq_group(); for (expr, expected_eq) in expressions { assert!( expected_eq.eq(&eq_groups.normalize_expr(expr.clone())), @@ -2544,4 +2537,115 @@ mod tests { Ok(()) } + + #[test] + fn test_get_lex_ordering() -> Result<()> { + // Schema satisfies following orderings: + // [a ASC], [d ASC, b ASC], [e DESC, f ASC, g ASC] + // and + // Column [a=c] (e.g they are aliases). + let (test_schema, schema_props) = create_test_params()?; + let col_a = &col("a", &test_schema)?; + let col_b = &col("b", &test_schema)?; + let col_c = &col("c", &test_schema)?; + let col_d = &col("d", &test_schema)?; + let col_e = &col("e", &test_schema)?; + let option_asc = SortOptions { + descending: false, + nulls_first: false, + }; + let option_desc = SortOptions { + descending: true, + nulls_first: true, + }; + let test_cases = vec![ + // TEST CASE 1 + (vec![col_a], Some(vec![(col_a, option_asc)])), + // TEST CASE 2 + (vec![col_c], Some(vec![(col_c, option_asc)])), + // TEST CASE 3 + ( + vec![col_d, col_e, col_b], + Some(vec![ + (col_d, option_asc), + (col_b, option_asc), + (col_e, option_desc), + ]), + ), + // TEST CASE 4 + (vec![col_b], None), + // TEST CASE 5 + (vec![col_b, col_a], None), + ]; + for (exprs, expected) in test_cases { + let exprs = exprs.into_iter().cloned().collect::>(); + let expected = expected.map(|expected| convert_to_sort_exprs(&expected)); + let actual = schema_props.get_lex_ordering(&exprs); + assert_eq!(actual, expected); + } + + Ok(()) + } + + #[test] + fn test_get_lex_ordering_section() -> Result<()> { + // Schema satisfies following orderings: + // [a ASC], [d ASC, b ASC], [e DESC, f ASC, g ASC] + // and + // Column [a=c] (e.g they are aliases). + // At below we add [d ASC, h DESC] also, for test purposes + let (test_schema, mut schema_props) = create_test_params()?; + let col_a = &col("a", &test_schema)?; + let col_b = &col("b", &test_schema)?; + let col_c = &col("c", &test_schema)?; + let col_d = &col("d", &test_schema)?; + let col_e = &col("e", &test_schema)?; + let col_h = &col("h", &test_schema)?; + + let option_asc = SortOptions { + descending: false, + nulls_first: false, + }; + let option_desc = SortOptions { + descending: true, + nulls_first: true, + }; + schema_props.add_new_orderings(&[vec![ + PhysicalSortExpr { + expr: col_d.clone(), + options: option_asc, + }, + PhysicalSortExpr { + expr: col_h.clone(), + options: option_desc, + }, + ]]); + let test_cases = vec![ + // TEST CASE 1 + (vec![col_a], vec![(col_a, option_asc)]), + // TEST CASE 2 + (vec![col_c], vec![(col_c, option_asc)]), + // TEST CASE 3 + ( + vec![col_d, col_e, col_b], + vec![ + (col_d, option_asc), + (col_b, option_asc), + (col_e, option_desc), + ], + ), + // TEST CASE 4 + (vec![col_b], vec![]), + // TEST CASE 5 + (vec![col_d], vec![(col_d, option_asc)]), + ]; + for (exprs, expected) in test_cases { + let exprs = exprs.into_iter().cloned().collect::>(); + let expected = convert_to_sort_exprs(&expected); + let actual = schema_props.get_lex_ordering_section(&exprs); + assert_eq!(actual, expected); + } + + Ok(()) + } } diff --git a/datafusion/physical-expr/src/partitioning.rs b/datafusion/physical-expr/src/partitioning.rs index 07cefbf57b12..6ca299a341d0 100644 --- a/datafusion/physical-expr/src/partitioning.rs +++ b/datafusion/physical-expr/src/partitioning.rs @@ -83,7 +83,7 @@ impl Partitioning { // and normalize both exprs based on the equivalent groups. if !fast_match { let schema_properties = schema_properties(); - let eq_groups = schema_properties.eq_groups(); + let eq_groups = schema_properties.eq_group(); if !eq_groups.is_empty() { let normalized_required_exprs = required_exprs .iter() diff --git a/datafusion/physical-expr/src/physical_expr.rs b/datafusion/physical-expr/src/physical_expr.rs index 36e3dbbe8e48..df96ac6d594f 100644 --- a/datafusion/physical-expr/src/physical_expr.rs +++ b/datafusion/physical-expr/src/physical_expr.rs @@ -314,41 +314,41 @@ mod tests { #[test] fn test_deduplicate_physical_exprs() -> Result<()> { - let lit_true = Arc::new(Literal::new(ScalarValue::Boolean(Some(true)))) - as Arc; - let lit_false = Arc::new(Literal::new(ScalarValue::Boolean(Some(false)))) - as Arc; - let lit4 = - Arc::new(Literal::new(ScalarValue::Int32(Some(4)))) as Arc; - let lit2 = - Arc::new(Literal::new(ScalarValue::Int32(Some(2)))) as Arc; - let col_a_expr = Arc::new(Column::new("a", 0)) as Arc; - let col_b_expr = Arc::new(Column::new("b", 1)) as Arc; - - // lit(true), lit(false), lit(4), lit(2), Col(a), Col(b) - let mut physical_exprs: Vec> = vec![ - lit_true.clone(), - lit_false.clone(), - lit4.clone(), - lit2.clone(), - col_a_expr.clone(), - col_a_expr.clone(), - col_b_expr.clone(), - lit_true.clone(), - lit2.clone(), + let lit_true = &(Arc::new(Literal::new(ScalarValue::Boolean(Some(true)))) + as Arc); + let lit_false = &(Arc::new(Literal::new(ScalarValue::Boolean(Some(false)))) + as Arc); + let lit4 = &(Arc::new(Literal::new(ScalarValue::Int32(Some(4)))) + as Arc); + let lit2 = &(Arc::new(Literal::new(ScalarValue::Int32(Some(2)))) + as Arc); + let col_a_expr = &(Arc::new(Column::new("a", 0)) as Arc); + let col_b_expr = &(Arc::new(Column::new("b", 1)) as Arc); + + // First vector in the tuple is arguments, second one is the expected value. + let test_cases = vec![ + // ---------- TEST CASE 1----------// + ( + vec![ + lit_true, lit_false, lit4, lit2, col_a_expr, col_a_expr, col_b_expr, + lit_true, lit2, + ], + // lit(true), lit(false), lit(4), lit(2), Col(a), Col(b) + vec![lit_true, lit_false, lit4, lit2, col_a_expr, col_b_expr], + ), + // ---------- TEST CASE 2----------// + ( + vec![lit_true, lit_true, lit_false, lit4], + vec![lit_true, lit4, lit_false], + ), ]; - - let expected = vec![ - lit_true.clone(), - lit_false.clone(), - lit4.clone(), - lit2.clone(), - col_a_expr.clone(), - col_b_expr.clone(), - ]; - // expected contains unique versions of the physical_exprs - deduplicate_physical_exprs(&mut physical_exprs); - physical_exprs_equal(&physical_exprs, &expected); + for (physical_exprs, expected) in test_cases { + let mut physical_exprs = + physical_exprs.into_iter().cloned().collect::>(); + let expected = expected.into_iter().cloned().collect::>(); + deduplicate_physical_exprs(&mut physical_exprs); + assert!(physical_exprs_equal(&physical_exprs, &expected)); + } Ok(()) } diff --git a/datafusion/physical-expr/src/utils.rs b/datafusion/physical-expr/src/utils.rs index cc56d5568285..85c04df6b42c 100644 --- a/datafusion/physical-expr/src/utils.rs +++ b/datafusion/physical-expr/src/utils.rs @@ -837,7 +837,7 @@ mod tests { let col_c_expr = col("c", &schema)?; let mut schema_properties = SchemaProperties::new(Arc::new(schema.clone())); - schema_properties.add_equal_conditions((&col_a_expr, &col_c_expr)); + schema_properties.add_equal_conditions(&col_a_expr, &col_c_expr); let others = vec![ vec![PhysicalSortExpr { expr: col_b_expr.clone(), @@ -862,8 +862,8 @@ mod tests { }], ]); - let oeq_class = schema_properties.oeq_group().clone(); - let expected = expected_oeq.oeq_group(); + let oeq_class = schema_properties.oeq_class().clone(); + let expected = expected_oeq.oeq_class(); assert!(oeq_class.eq(expected)); Ok(()) @@ -899,7 +899,7 @@ mod tests { let projected_oeq = schema_properties.project(&source_to_target_mapping, projection_schema); let orderings = projected_oeq - .oeq_group() + .oeq_class() .output_ordering() .unwrap_or_default(); @@ -934,7 +934,7 @@ mod tests { let projected_oeq = schema_properties.project(&source_to_target_mapping, projection_schema); // After projection there is no ordering. - assert!(projected_oeq.oeq_group().output_ordering().is_none()); + assert!(projected_oeq.oeq_class().output_ordering().is_none()); Ok(()) } diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 23267c27dc07..79a3c4169983 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -545,7 +545,7 @@ impl AggregateExec { let aggregate_oeq = input .schema_properties() .project(&source_to_target_mapping, schema.clone()); - let output_ordering = aggregate_oeq.oeq_group().output_ordering(); + let output_ordering = aggregate_oeq.oeq_class().output_ordering(); Ok(AggregateExec { mode, @@ -2045,7 +2045,7 @@ mod tests { let col_c = &col("c", &test_schema)?; let mut schema_properties = SchemaProperties::new(test_schema); // Columns a and b are equal. - schema_properties.add_equal_conditions((col_a, col_b)); + schema_properties.add_equal_conditions(col_a, col_b); // Aggregate requirements are // [None], [a ASC], [a ASC, b ASC, c ASC], [a ASC, b ASC] respectively let mut order_by_exprs = vec![ diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index 37b0f73a581c..9c044c3a846d 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -333,8 +333,7 @@ impl GroupedHashAggregateStream { let reservation = MemoryConsumer::new(name).register(context.memory_pool()); let ordered_section = agg .schema_properties() - .get_lex_ordering_section(&agg_group_by.output_exprs()) - .unwrap_or_default(); + .get_lex_ordering_section(&agg_group_by.output_exprs()); let group_ordering = GroupOrdering::try_new( &group_schema, &agg.partition_search_mode, diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index 525fd6e18ed5..3ae49e3f0d4e 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -152,7 +152,7 @@ impl ExecutionPlan for FilterExec { for (lhs, rhs) in equal_pairs { let lhs_expr = Arc::new(lhs.clone()) as _; let rhs_expr = Arc::new(rhs.clone()) as _; - filter_oeq.add_equal_conditions((&lhs_expr, &rhs_expr)) + filter_oeq.add_equal_conditions(&lhs_expr, &rhs_expr) } // Add the columns that have only one value (singleton) after filtering to constants. let constants = collect_columns(self.predicate()) diff --git a/datafusion/physical-plan/src/memory.rs b/datafusion/physical-plan/src/memory.rs index 75c76ddc93ec..3f074cf17774 100644 --- a/datafusion/physical-plan/src/memory.rs +++ b/datafusion/physical-plan/src/memory.rs @@ -301,7 +301,7 @@ mod tests { assert_eq!(mem_exec.output_ordering().unwrap(), expected_output_order); let order_eq = mem_exec.schema_properties(); - assert!(order_eq.oeq_group().contains(&expected_order_eq)); + assert!(order_eq.oeq_class().contains(&expected_order_eq)); Ok(()) } } diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index 0722a8787032..5402c85ea0cf 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -99,7 +99,7 @@ impl ProjectionExec { let input_oeq = input.schema_properties(); let project_oeq = input_oeq.project(&source_to_target_mapping, schema.clone()); - let project_orderings = project_oeq.oeq_group(); + let project_orderings = project_oeq.oeq_class(); let output_ordering = project_orderings.output_ordering(); Ok(Self { diff --git a/datafusion/physical-plan/src/union.rs b/datafusion/physical-plan/src/union.rs index 628f366118f6..a237e7de7ef2 100644 --- a/datafusion/physical-plan/src/union.rs +++ b/datafusion/physical-plan/src/union.rs @@ -234,7 +234,7 @@ impl ExecutionPlan for UnionExec { let mut union_oeq = SchemaProperties::new(self.schema()); // Get first ordering equivalent group as seed group. let mut existing_meets = child_oeqs[0] - .oeq_group() + .oeq_class() .iter() .map(|elem| elem.to_vec()) .collect::>(); @@ -243,7 +243,7 @@ impl ExecutionPlan for UnionExec { // Find the valid meet orderings of existing meet and new group. let mut next_meets = vec![]; for existing_meet in &existing_meets { - next_meets.extend(next_child_oeq.oeq_group().iter().filter_map( + next_meets.extend(next_child_oeq.oeq_class().iter().filter_map( |ordering| next_child_oeq.get_meet_ordering(ordering, existing_meet), )); } @@ -859,7 +859,7 @@ mod tests { let union = UnionExec::new(vec![child1, child2]); let union_schema_properties = union.schema_properties(); - let union_actual_orderings = union_schema_properties.oeq_group(); + let union_actual_orderings = union_schema_properties.oeq_class(); println!("union_orderings:{:?}", union_actual_orderings); let err_msg = format!( "Error in test id: {:?}, test case: {:?}", From a78a9188c773e7823087e6e874ede8257941281b Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 24 Oct 2023 17:48:10 +0300 Subject: [PATCH 090/122] Resolve linter errors --- datafusion/physical-expr/src/equivalence.rs | 28 +++++++++++-------- .../physical-plan/src/joins/cross_join.rs | 4 +-- .../physical-plan/src/joins/hash_join.rs | 4 +-- .../src/joins/sort_merge_join.rs | 4 +-- .../src/joins/symmetric_hash_join.rs | 4 +-- 5 files changed, 25 insertions(+), 19 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 239372b99c99..2b378dd57ff3 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -1088,8 +1088,8 @@ impl SchemaProperties { /// Calculate ordering equivalence properties for the given join operation. pub fn join_schema_properties( - left: &SchemaProperties, - right: &SchemaProperties, + left: SchemaProperties, + right: SchemaProperties, join_type: &JoinType, join_schema: SchemaRef, maintains_input_order: &[bool], @@ -1108,8 +1108,8 @@ pub fn join_schema_properties( assert_eq!(maintains_input_order.len(), 2); let left_maintains = maintains_input_order[0]; let right_maintains = maintains_input_order[1]; - let left_oeq_class = left.oeq_class(); - let right_oeq_class = right.oeq_class(); + let left_oeq_class = left.oeq_class; + let right_oeq_class = right.oeq_class; match (left_maintains, right_maintains) { (true, true) => { unreachable!("Cannot maintain ordering of both sides"); @@ -1134,13 +1134,13 @@ pub fn join_schema_properties( let out_oeq_class = left_oeq_class.join_postfix(&updated_right_oeq); new_properties.add_ordering_equivalent_group(out_oeq_class); } else { - new_properties.add_ordering_equivalent_group(left_oeq_class.clone()); + new_properties.add_ordering_equivalent_group(left_oeq_class); } } (false, true) => { let updated_right_oeq = get_updated_right_ordering_equivalent_group( join_type, - right.oeq_class(), + right_oeq_class, left_columns_len, ); // In this special case, left side ordering can be prefixed with right side ordering. @@ -1153,7 +1153,7 @@ pub fn join_schema_properties( // ordering of the left table is `a ASC`, then the ordering equivalence `b ASC` // for the right table should be converted to `a ASC, b ASC` before it is added // to the ordering equivalences of the join. - let out_oeq_class = updated_right_oeq.join_postfix(left_oeq_class); + let out_oeq_class = updated_right_oeq.join_postfix(&left_oeq_class); new_properties.add_ordering_equivalent_group(out_oeq_class); } else { new_properties.add_ordering_equivalent_group(updated_right_oeq); @@ -1314,7 +1314,7 @@ fn update_ordering( /// it can thereafter safely be used for ordering equivalence normalization. fn get_updated_right_ordering_equivalent_group( join_type: &JoinType, - right_oeq_group: &OrderingEquivalenceClass, + right_oeq_group: OrderingEquivalenceClass, left_columns_len: usize, ) -> OrderingEquivalenceClass { if matches!( @@ -1323,7 +1323,7 @@ fn get_updated_right_ordering_equivalent_group( ) { return right_oeq_group.add_offset(left_columns_len); } - right_oeq_group.clone() + right_oeq_group } #[cfg(test)] @@ -2086,7 +2086,7 @@ mod tests { let result = get_updated_right_ordering_equivalent_group( &join_type, - &right_oeq_class, + right_oeq_class, left_columns_len, ); join_schema_properties.add_ordering_equivalent_group(result); @@ -2265,7 +2265,13 @@ mod tests { let res: Vec<_> = schema_vec .into_iter() .zip(schema.fields.iter()) - .map(|(elem, field)| (field.name(), elem.unwrap())) + .map(|(elem, field)| { + ( + field.name(), + // Generate random values for columns that do not occur in any of the groups (equivalence, ordering equivalence, constants) + elem.unwrap_or_else(|| generate_random_array(n_elem, n_distinct)), + ) + }) .collect(); Ok(RecordBatch::try_from_iter(res)?) diff --git a/datafusion/physical-plan/src/joins/cross_join.rs b/datafusion/physical-plan/src/joins/cross_join.rs index d4fd4895678f..97268734738a 100644 --- a/datafusion/physical-plan/src/joins/cross_join.rs +++ b/datafusion/physical-plan/src/joins/cross_join.rs @@ -217,8 +217,8 @@ impl ExecutionPlan for CrossJoinExec { let left = self.left.schema_properties(); let right = self.right.schema_properties(); join_schema_properties( - &left, - &right, + left, + right, &JoinType::Full, self.schema(), &[false, false], diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index 4774a2bf2521..47ebaa328ea6 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -369,8 +369,8 @@ impl ExecutionPlan for HashJoinExec { let left = self.left.schema_properties(); let right = self.right.schema_properties(); join_schema_properties( - &left, - &right, + left, + right, &self.join_type, self.schema(), &self.maintains_input_order(), diff --git a/datafusion/physical-plan/src/joins/sort_merge_join.rs b/datafusion/physical-plan/src/joins/sort_merge_join.rs index 3c0adedbb46b..ced007dc84bc 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join.rs @@ -285,8 +285,8 @@ impl ExecutionPlan for SortMergeJoinExec { let left = self.left.schema_properties(); let right = self.right.schema_properties(); join_schema_properties( - &left, - &right, + left, + right, &self.join_type, self.schema(), &self.maintains_input_order(), diff --git a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs index da50ff8ff6ff..bddb5cb5a4e1 100644 --- a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs +++ b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs @@ -433,8 +433,8 @@ impl ExecutionPlan for SymmetricHashJoinExec { let left = self.left.schema_properties(); let right = self.right.schema_properties(); join_schema_properties( - &left, - &right, + left, + right, &self.join_type, self.schema(), &self.maintains_input_order(), From 18e4e1120df19b30f398268c4a602e4de17ab145 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 25 Oct 2023 13:29:24 +0300 Subject: [PATCH 091/122] Simplify test codes --- datafusion/physical-expr/src/equivalence.rs | 401 +++++++------------- 1 file changed, 146 insertions(+), 255 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 2b378dd57ff3..15b4e81b2a25 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -882,7 +882,7 @@ impl SchemaProperties { /// Please note that pruned version may not functionally equivalent to the argument. /// Empty result means that requirement is already satisfied. /// Non-empty result means that requirement is not satisfied. - /// This util shouldn't e used outside this context. + /// This util shouldn't be used outside this context. fn prune_lex_req(&self, sort_req: &[PhysicalSortRequirement]) -> LexOrderingReq { // Make sure to use a standardized version of the requirement let mut normalized_sort_req = self.normalize_sort_requirements(sort_req); @@ -927,15 +927,6 @@ impl SchemaProperties { normalized_sort_req } - /// Checks whether `leading_requirement` is contained in any of the ordering - /// equivalence classes. - pub fn satisfies_leading_requirement( - &self, - leading_requirement: &PhysicalSortRequirement, - ) -> bool { - self.ordering_satisfy_requirement(&[leading_requirement.clone()]) - } - /// Projects argument `expr` according to mapping inside `source_to_target_mapping`. /// While doing so consider equalities also. /// As an example assume `source_to_target_mapping` contains following mapping @@ -1035,6 +1026,7 @@ impl SchemaProperties { exprs: &[Arc], ) -> Option> { let ordered_section = self.get_lex_ordering_section(exprs); + // Make sure ordered section covers all expressions. if ordered_section.len() == exprs.len() { return Some(ordered_section); } @@ -1051,6 +1043,8 @@ impl SchemaProperties { let normalized_exprs = self.eq_group.normalize_exprs(exprs.to_vec()); // After normalization size of the vector shouldn't change. assert_eq!(normalized_exprs.len(), exprs.len()); + + // Stores the index and SortOption of the ordered expression. let mut ordered_exprs: Vec<(usize, SortOptions)> = vec![]; for ordering in self.normalized_oeq_class().iter() { for sort_expr in ordering { @@ -1076,6 +1070,7 @@ impl SchemaProperties { } } } + // Construct ordered section by getting entries at indices of ordered_exprs (first entry in the tuple) ordered_exprs .into_iter() .map(|(idx, options)| PhysicalSortExpr { @@ -1367,54 +1362,41 @@ mod tests { /// Column [a=c] (e.g they are aliases). fn create_test_params() -> Result<(SchemaRef, SchemaProperties)> { let test_schema = create_test_schema()?; - let col_a_expr = &col("a", &test_schema)?; - let col_b_expr = &col("b", &test_schema)?; - let col_c_expr = &col("c", &test_schema)?; - let col_d_expr = &col("d", &test_schema)?; - let col_e_expr = &col("e", &test_schema)?; - let col_f_expr = &col("f", &test_schema)?; - let col_g_expr = &col("g", &test_schema)?; + let col_a = &col("a", &test_schema)?; + let col_b = &col("b", &test_schema)?; + let col_c = &col("c", &test_schema)?; + let col_d = &col("d", &test_schema)?; + let col_e = &col("e", &test_schema)?; + let col_f = &col("f", &test_schema)?; + let col_g = &col("g", &test_schema)?; let mut schema_properties = SchemaProperties::new(test_schema.clone()); - schema_properties.add_equal_conditions(col_a_expr, col_c_expr); + schema_properties.add_equal_conditions(col_a, col_c); - let option1 = SortOptions { + let option_asc = SortOptions { descending: false, nulls_first: false, }; - let option2 = SortOptions { + let option_desc = SortOptions { descending: true, nulls_first: true, }; - schema_properties.add_new_orderings(&[ - vec![PhysicalSortExpr { - expr: col_a_expr.clone(), - options: option1, - }], + let orderings = vec![ + // [a ASC] + vec![(col_a, option_asc)], + // [d ASC, b ASC] + vec![(col_d, option_asc), (col_b, option_asc)], + // [e DESC, f ASC, g ASC] vec![ - PhysicalSortExpr { - expr: col_d_expr.clone(), - options: option1, - }, - PhysicalSortExpr { - expr: col_b_expr.clone(), - options: option1, - }, + (col_e, option_desc), + (col_f, option_asc), + (col_g, option_asc), ], - vec![ - PhysicalSortExpr { - expr: col_e_expr.clone(), - options: option2, - }, - PhysicalSortExpr { - expr: col_f_expr.clone(), - options: option1, - }, - PhysicalSortExpr { - expr: col_g_expr.clone(), - options: option1, - }, - ], - ]); + ]; + let orderings = orderings + .into_iter() + .map(|sort_exprs| convert_to_sort_exprs(&sort_exprs)) + .collect::>(); + schema_properties.add_new_orderings(&orderings); Ok((test_schema, schema_properties)) } @@ -1481,6 +1463,7 @@ mod tests { } /// Checks whether the given physical expression slices are equal. + /// If any permutation is equal returns true. pub fn physical_exprs_set_equal( lhs: &[Arc], rhs: &[Arc], @@ -1580,13 +1563,13 @@ mod tests { Field::new("c", DataType::Int64, true), ])); - let mut input_properties = SchemaProperties::new(input_schema); - let col_a_expr = Arc::new(Column::new("a", 0)) as Arc; - let col_b_expr = Arc::new(Column::new("b", 1)) as Arc; - let col_c_expr = Arc::new(Column::new("c", 2)) as Arc; + let mut input_properties = SchemaProperties::new(input_schema.clone()); + let col_a = col("a", &input_schema)?; + let col_b = col("b", &input_schema)?; + let col_c = col("c", &input_schema)?; - input_properties.add_equal_conditions(&col_a_expr, &col_b_expr); - input_properties.add_equal_conditions(&col_b_expr, &col_c_expr); + input_properties.add_equal_conditions(&col_a, &col_b); + input_properties.add_equal_conditions(&col_b, &col_c); let out_schema = Arc::new(Schema::new(vec![ Field::new("a1", DataType::Int64, true), @@ -1595,15 +1578,15 @@ mod tests { Field::new("a4", DataType::Int64, true), ])); - let col_a1_expr = Arc::new(Column::new("a1", 0)) as Arc; - let col_a2_expr = Arc::new(Column::new("a2", 1)) as Arc; - let col_a3_expr = Arc::new(Column::new("a3", 2)) as Arc; - let col_a4_expr = Arc::new(Column::new("a4", 2)) as Arc; + let col_a1 = &col("a1", &out_schema)?; + let col_a2 = &col("a2", &out_schema)?; + let col_a3 = &col("a3", &out_schema)?; + let col_a4 = &col("a4", &out_schema)?; let source_to_target_mapping = vec![ - (col_a_expr.clone(), col_a1_expr.clone()), - (col_a_expr.clone(), col_a2_expr.clone()), - (col_a_expr.clone(), col_a3_expr.clone()), - (col_a_expr.clone(), col_a4_expr.clone()), + (col_a.clone(), col_a1.clone()), + (col_a.clone(), col_a2.clone()), + (col_a.clone(), col_a3.clone()), + (col_a.clone(), col_a4.clone()), ]; let out_properties = input_properties.project(&source_to_target_mapping, out_schema); @@ -1611,10 +1594,10 @@ mod tests { assert_eq!(out_properties.eq_group().len(), 1); let eq_class = &out_properties.eq_group().classes[0]; assert_eq!(eq_class.len(), 4); - assert!(physical_exprs_contains(eq_class, &col_a1_expr)); - assert!(physical_exprs_contains(eq_class, &col_a2_expr)); - assert!(physical_exprs_contains(eq_class, &col_a3_expr)); - assert!(physical_exprs_contains(eq_class, &col_a4_expr)); + assert!(physical_exprs_contains(eq_class, col_a1)); + assert!(physical_exprs_contains(eq_class, col_a2)); + assert!(physical_exprs_contains(eq_class, col_a3)); + assert!(physical_exprs_contains(eq_class, col_a4)); Ok(()) } @@ -1866,104 +1849,62 @@ mod tests { #[test] fn test_ordering_satisfy_different_lengths() -> Result<()> { let test_schema = create_test_schema()?; - let col_a = col("a", &test_schema)?; - let col_b = col("b", &test_schema)?; - let col_c = col("c", &test_schema)?; - let col_d = col("d", &test_schema)?; - let col_e = col("e", &test_schema)?; - let col_f = col("f", &test_schema)?; + let col_a = &col("a", &test_schema)?; + let col_b = &col("b", &test_schema)?; + let col_c = &col("c", &test_schema)?; + let col_d = &col("d", &test_schema)?; + let col_e = &col("e", &test_schema)?; + let col_f = &col("f", &test_schema)?; let options = SortOptions { descending: false, nulls_first: false, }; // Column a and c are aliases. let mut schema_properties = SchemaProperties::new(test_schema); - schema_properties.add_equal_conditions(&col_a, &col_c); + schema_properties.add_equal_conditions(col_a, col_c); - // Column a and e are ordering equivalent (e.g global ordering of the table can be described both as a ASC and e ASC.) - schema_properties.add_new_orderings(&[ - vec![PhysicalSortExpr { - expr: col_a.clone(), - options, - }], - vec![PhysicalSortExpr { - expr: col_e.clone(), - options, - }], - ]); + let orderings = vec![ + vec![(col_a, options)], + vec![(col_e, options)], + vec![(col_d, options), (col_f, options)], + ]; + let orderings = orderings + .into_iter() + .map(|sort_exprs| convert_to_sort_exprs(&sort_exprs)) + .collect::>(); - // Column a and d,f are ordering equivalent (e.g global ordering of the table can be described both as [a ASC] and [d ASC, f ASC].) - schema_properties.add_new_orderings(&[ - vec![PhysicalSortExpr { - expr: col_a.clone(), - options, - }], - vec![ - PhysicalSortExpr { - expr: col_d.clone(), - options, - }, - PhysicalSortExpr { - expr: col_f.clone(), - options, - }, - ], - ]); - // All of the orderings [a ASC], [d ASC, f ASC], [e ASC]] + // Column [a ASC], [e ASC], [d ASC, f ASC] are all valid orderings for the schema. + schema_properties.add_new_orderings(&orderings); + + // All of the orderings [a ASC], [e ASC], [d ASC, f ASC]] // are valid for the table // Also Columns a and c are equal + let test_cases = vec![ + // [c ASC, a ASC, e ASC], expected represents this requirement is satisfied + ( + vec![(col_c, options), (col_a, options), (col_e, options)], + true, + ), + (vec![(col_c, options), (col_b, options)], false), + (vec![(col_c, options), (col_d, options)], true), + ( + vec![(col_d, options), (col_f, options), (col_b, options)], + false, + ), + (vec![(col_d, options), (col_f, options)], true), + ]; - let sort_req_a = PhysicalSortExpr { - expr: col_a.clone(), - options, - }; - let sort_req_b = PhysicalSortExpr { - expr: col_b.clone(), - options, - }; - let sort_req_c = PhysicalSortExpr { - expr: col_c.clone(), - options, - }; - let sort_req_d = PhysicalSortExpr { - expr: col_d.clone(), - options, - }; - let sort_req_e = PhysicalSortExpr { - expr: col_e.clone(), - options, - }; - let sort_req_f = PhysicalSortExpr { - expr: col_f.clone(), - options, - }; - - assert!(schema_properties.ordering_satisfy( - // After normalization would be a ASC - &[sort_req_c.clone(), sort_req_a.clone(), sort_req_e.clone(),], - )); - assert!(!schema_properties.ordering_satisfy( - // After normalization would be a ASC, b ASC - // which is not satisfied - &[sort_req_c.clone(), sort_req_b.clone(),], - )); - - assert!(schema_properties.ordering_satisfy( - // After normalization would be a ASC - &[sort_req_c.clone(), sort_req_d.clone(),], - )); - - assert!(!schema_properties.ordering_satisfy( - // After normalization would be a ASC, b ASC - // which is not satisfied - &[sort_req_d.clone(), sort_req_f.clone(), sort_req_b.clone(),], - )); - - assert!(schema_properties.ordering_satisfy( - // After normalization would be a ASC - // which is satisfied - &[sort_req_d.clone(), sort_req_f.clone()], - )); + for (reqs, expected) in test_cases { + let err_msg = + format!("error in test reqs: {:?}, expected: {:?}", reqs, expected,); + let reqs = convert_to_sort_exprs(&reqs); + assert_eq!( + schema_properties.ordering_satisfy(&reqs), + expected, + "{}", + err_msg + ); + } Ok(()) } @@ -2040,30 +1981,27 @@ mod tests { #[test] fn test_get_updated_right_ordering_equivalence_properties() -> Result<()> { let join_type = JoinType::Inner; - + // Join right child schema + let child_fields: Fields = ["x", "y", "z", "w"] + .into_iter() + .map(|name| Field::new(name, DataType::Int32, true)) + .collect(); + let child_schema = Schema::new(child_fields); + let col_x = &col("x", &child_schema)?; + let col_y = &col("y", &child_schema)?; + let col_z = &col("z", &child_schema)?; + let col_w = &col("w", &child_schema)?; let options = SortOptions::default(); - let right_oeq_class = OrderingEquivalenceClass::new(vec![ - vec![ - PhysicalSortExpr { - expr: Arc::new(Column::new("x", 0)), - options, - }, - PhysicalSortExpr { - expr: Arc::new(Column::new("y", 1)), - options, - }, - ], - vec![ - PhysicalSortExpr { - expr: Arc::new(Column::new("z", 2)), - options, - }, - PhysicalSortExpr { - expr: Arc::new(Column::new("w", 3)), - options, - }, - ], - ]); + let orderings = vec![ + vec![(col_x, options), (col_y, options)], + vec![(col_z, options), (col_w, options)], + ]; + let orderings = orderings + .into_iter() + .map(|sort_exprs| convert_to_sort_exprs(&sort_exprs)) + .collect::>(); + // Right child ordering equivalences + let right_oeq_class = OrderingEquivalenceClass::new(orderings); let left_columns_len = 4; @@ -2072,17 +2010,18 @@ mod tests { .map(|name| Field::new(name, DataType::Int32, true)) .collect(); + // Join Schema let schema = Schema::new(fields); - let col_a_expr = col("a", &schema)?; - let col_d_expr = col("d", &schema)?; - let col_x_expr = col("x", &schema)?; - let col_y_expr = col("y", &schema)?; - let col_z_expr = col("z", &schema)?; - let col_w_expr = col("w", &schema)?; + let col_a = &col("a", &schema)?; + let col_d = &col("d", &schema)?; + let col_x = &col("x", &schema)?; + let col_y = &col("y", &schema)?; + let col_z = &col("z", &schema)?; + let col_w = &col("w", &schema)?; let mut join_schema_properties = SchemaProperties::new(Arc::new(schema)); - join_schema_properties.add_equal_conditions(&col_a_expr, &col_x_expr); - join_schema_properties.add_equal_conditions(&col_d_expr, &col_w_expr); + join_schema_properties.add_equal_conditions(col_a, col_x); + join_schema_properties.add_equal_conditions(col_d, col_w); let result = get_updated_right_ordering_equivalent_group( &join_type, @@ -2092,28 +2031,15 @@ mod tests { join_schema_properties.add_ordering_equivalent_group(result); let result = join_schema_properties.oeq_class().clone(); - let expected = OrderingEquivalenceClass::new(vec![ - vec![ - PhysicalSortExpr { - expr: col_x_expr, - options, - }, - PhysicalSortExpr { - expr: col_y_expr, - options, - }, - ], - vec![ - PhysicalSortExpr { - expr: col_z_expr, - options, - }, - PhysicalSortExpr { - expr: col_w_expr, - options, - }, - ], - ]); + let orderings = vec![ + vec![(col_x, options), (col_y, options)], + vec![(col_z, options), (col_w, options)], + ]; + let orderings = orderings + .into_iter() + .map(|sort_exprs| convert_to_sort_exprs(&sort_exprs)) + .collect::>(); + let expected = OrderingEquivalenceClass::new(orderings); assert_eq!(result, expected); @@ -2316,62 +2242,27 @@ mod tests { }; // Assume that column a and c are aliases. let (test_schema, schema_properties) = create_test_params()?; - let col_a_expr = &col("a", &test_schema)?; - let _col_b_expr = &col("b", &test_schema)?; - let col_c_expr = &col("c", &test_schema)?; - let col_d_expr = &col("d", &test_schema)?; - let _col_e_expr = &col("e", &test_schema)?; + let col_a = &col("a", &test_schema)?; + let col_c = &col("c", &test_schema)?; + let col_d = &col("d", &test_schema)?; // Test cases for equivalence normalization // First entry in the tuple is PhysicalExpr, second entry is its ordering, third entry is result after normalization. - let expressions = vec![ - ( - vec![PhysicalSortRequirement { - expr: col_a_expr.clone(), - options: Some(option1), - }], - vec![PhysicalSortRequirement { - expr: col_a_expr.clone(), - options: Some(option1), - }], - ), + let test_cases = vec![ + (vec![(col_a, Some(option1))], vec![(col_a, Some(option1))]), // In the normalized version column c should be replace with column a - ( - vec![PhysicalSortRequirement { - expr: col_c_expr.clone(), - options: Some(option1), - }], - vec![PhysicalSortRequirement { - expr: col_a_expr.clone(), - options: Some(option1), - }], - ), - ( - vec![PhysicalSortRequirement { - expr: col_c_expr.clone(), - options: None, - }], - vec![PhysicalSortRequirement { - expr: col_a_expr.clone(), - options: None, - }], - ), - ( - vec![PhysicalSortRequirement { - expr: col_d_expr.clone(), - options: Some(option1), - }], - vec![PhysicalSortRequirement { - expr: col_d_expr.clone(), - options: Some(option1), - }], - ), + (vec![(col_c, Some(option1))], vec![(col_a, Some(option1))]), + (vec![(col_c, None)], vec![(col_a, None)]), + (vec![(col_d, Some(option1))], vec![(col_d, Some(option1))]), ]; - for (arg, expected) in expressions.into_iter() { - let normalized = schema_properties.normalize_sort_requirements(&arg); + for (reqs, expected) in test_cases.into_iter() { + let reqs = convert_to_sort_reqs(&reqs); + let expected = convert_to_sort_reqs(&expected); + + let normalized = schema_properties.normalize_sort_requirements(&reqs); assert!( expected.eq(&normalized), - "error in test: arg: {arg:?}, expected: {expected:?}, normalized: {normalized:?}" + "error in test: reqs: {reqs:?}, expected: {expected:?}, normalized: {normalized:?}" ); } From f0b105286b9035c906f2c423e9c75b6df509692f Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Wed, 25 Oct 2023 15:25:56 +0300 Subject: [PATCH 092/122] Review Part 9 --- .../physical-expr/src/expressions/in_list.rs | 5 +- datafusion/physical-expr/src/lib.rs | 5 +- datafusion/physical-expr/src/physical_expr.rs | 230 +++++++++++------- datafusion/physical-expr/src/utils.rs | 71 +----- 4 files changed, 152 insertions(+), 159 deletions(-) diff --git a/datafusion/physical-expr/src/expressions/in_list.rs b/datafusion/physical-expr/src/expressions/in_list.rs index 643bbfd820a6..8d55fb70bd9e 100644 --- a/datafusion/physical-expr/src/expressions/in_list.rs +++ b/datafusion/physical-expr/src/expressions/in_list.rs @@ -22,8 +22,7 @@ use std::fmt::Debug; use std::hash::{Hash, Hasher}; use std::sync::Arc; -use crate::physical_expr::down_cast_any_ref; -use crate::utils::expr_list_eq_any_order; +use crate::physical_expr::{down_cast_any_ref, physical_exprs_bag_equal}; use crate::PhysicalExpr; use arrow::array::*; @@ -410,7 +409,7 @@ impl PartialEq for InListExpr { .downcast_ref::() .map(|x| { self.expr.eq(&x.expr) - && expr_list_eq_any_order(&self.list, &x.list) + && physical_exprs_bag_equal(&self.list, &x.list) && self.negated == x.negated }) .unwrap_or(false) diff --git a/datafusion/physical-expr/src/lib.rs b/datafusion/physical-expr/src/lib.rs index 02d666f5fa36..09c64ab7a0c4 100644 --- a/datafusion/physical-expr/src/lib.rs +++ b/datafusion/physical-expr/src/lib.rs @@ -59,7 +59,8 @@ pub use equivalence::{ pub use partitioning::{Distribution, Partitioning}; pub use physical_expr::{ - physical_exprs_contains, physical_exprs_equal, PhysicalExpr, PhysicalExprRef, + physical_exprs_contains, physical_exprs_equal, physical_exprs_bag_equal, + PhysicalExpr, PhysicalExprRef, }; pub use planner::create_physical_expr; pub use scalar_function::ScalarFunctionExpr; @@ -67,4 +68,4 @@ pub use sort_expr::{ LexOrdering, LexOrderingRef, LexOrderingReq, PhysicalSortExpr, PhysicalSortRequirement, }; -pub use utils::{expr_list_eq_any_order, reverse_order_bys, split_conjunction}; +pub use utils::{reverse_order_bys, split_conjunction}; diff --git a/datafusion/physical-expr/src/physical_expr.rs b/datafusion/physical-expr/src/physical_expr.rs index df96ac6d594f..4dbb798309f4 100644 --- a/datafusion/physical-expr/src/physical_expr.rs +++ b/datafusion/physical-expr/src/physical_expr.rs @@ -228,26 +228,6 @@ pub fn physical_exprs_contains( .any(|physical_expr| physical_expr.eq(expr)) } -/// This utility function removes duplicates from the given `physical_exprs` -/// vector. Once we can use `HashSet`s with `Arc`, this -/// function should use a `HashSet` to reduce computational complexity. -/// -/// Note that this function does not necessarily preserve its input ordering. -pub fn deduplicate_physical_exprs(physical_exprs: &mut Vec>) { - let mut i = 0; - while i < physical_exprs.len() { - let mut j = i + 1; - while j < physical_exprs.len() { - if physical_exprs[i].eq(&physical_exprs[j]) { - physical_exprs.swap_remove(j); - } else { - j += 1; - } - } - i += 1; - } -} - /// Checks whether the given slices have any common entries. pub fn have_common_entries( lhs: &[Arc], @@ -264,20 +244,63 @@ pub fn physical_exprs_equal( lhs.len() == rhs.len() && izip!(lhs, rhs).all(|(lhs, rhs)| lhs.eq(rhs)) } +/// Checks whether the given physical expression slices are equal in the sense +/// of bags (multi-sets), disregarding their orderings. +pub fn physical_exprs_bag_equal( + lhs: &[Arc], + rhs: &[Arc], +) -> bool { + // TODO: Once we can use `HashMap`s with `Arc`, this + // function should use a `HashMap` to reduce computational complexity. + if lhs.len() == rhs.len() { + let mut rhs_vec = rhs.to_vec(); + for expr in lhs { + if let Some(idx) = rhs_vec.iter().position(|e| expr.eq(e)) { + rhs_vec.swap_remove(idx); + } else { + return false; + } + } + true + } else { + false + } +} + +/// This utility function removes duplicates from the given `physical_exprs` +/// vector. Note that this function does not necessarily preserve its input +/// ordering. +pub fn deduplicate_physical_exprs(physical_exprs: &mut Vec>) { + // TODO: Once we can use `HashSet`s with `Arc`, this + // function should use a `HashSet` to reduce computational complexity. + let mut idx = 0; + while idx < physical_exprs.len() { + let mut rest_idx = idx + 1; + while rest_idx < physical_exprs.len() { + if physical_exprs[idx].eq(&physical_exprs[rest_idx]) { + physical_exprs.swap_remove(rest_idx); + } else { + rest_idx += 1; + } + } + idx += 1; + } +} + #[cfg(test)] mod tests { use std::sync::Arc; use crate::expressions::{Column, Literal}; use crate::physical_expr::{ - deduplicate_physical_exprs, have_common_entries, physical_exprs_equal, + deduplicate_physical_exprs, have_common_entries, physical_exprs_contains, + physical_exprs_equal, physical_exprs_bag_equal, PhysicalExpr, }; - use crate::{physical_exprs_contains, PhysicalExpr}; - use datafusion_common::{Result, ScalarValue}; + use datafusion_common::ScalarValue; #[test] - fn test_physical_exprs_contains() -> Result<()> { + fn test_physical_exprs_contains() { let lit_true = Arc::new(Literal::new(ScalarValue::Boolean(Some(true)))) as Arc; let lit_false = Arc::new(Literal::new(ScalarValue::Boolean(Some(false)))) @@ -309,51 +332,10 @@ mod tests { // below expressions are not inside physical_exprs assert!(!physical_exprs_contains(&physical_exprs, &col_c_expr)); assert!(!physical_exprs_contains(&physical_exprs, &lit1)); - Ok(()) - } - - #[test] - fn test_deduplicate_physical_exprs() -> Result<()> { - let lit_true = &(Arc::new(Literal::new(ScalarValue::Boolean(Some(true)))) - as Arc); - let lit_false = &(Arc::new(Literal::new(ScalarValue::Boolean(Some(false)))) - as Arc); - let lit4 = &(Arc::new(Literal::new(ScalarValue::Int32(Some(4)))) - as Arc); - let lit2 = &(Arc::new(Literal::new(ScalarValue::Int32(Some(2)))) - as Arc); - let col_a_expr = &(Arc::new(Column::new("a", 0)) as Arc); - let col_b_expr = &(Arc::new(Column::new("b", 1)) as Arc); - - // First vector in the tuple is arguments, second one is the expected value. - let test_cases = vec![ - // ---------- TEST CASE 1----------// - ( - vec![ - lit_true, lit_false, lit4, lit2, col_a_expr, col_a_expr, col_b_expr, - lit_true, lit2, - ], - // lit(true), lit(false), lit(4), lit(2), Col(a), Col(b) - vec![lit_true, lit_false, lit4, lit2, col_a_expr, col_b_expr], - ), - // ---------- TEST CASE 2----------// - ( - vec![lit_true, lit_true, lit_false, lit4], - vec![lit_true, lit4, lit_false], - ), - ]; - for (physical_exprs, expected) in test_cases { - let mut physical_exprs = - physical_exprs.into_iter().cloned().collect::>(); - let expected = expected.into_iter().cloned().collect::>(); - deduplicate_physical_exprs(&mut physical_exprs); - assert!(physical_exprs_equal(&physical_exprs, &expected)); - } - Ok(()) } #[test] - fn test_have_common_entries() -> Result<()> { + fn test_have_common_entries() { let lit_true = Arc::new(Literal::new(ScalarValue::Boolean(Some(true)))) as Arc; let lit_false = Arc::new(Literal::new(ScalarValue::Boolean(Some(false)))) @@ -364,49 +346,129 @@ mod tests { Arc::new(Literal::new(ScalarValue::Int32(Some(1)))) as Arc; let col_b_expr = Arc::new(Column::new("b", 1)) as Arc; - let vec1: Vec> = vec![lit_true.clone(), lit_false.clone()]; - - let vec2: Vec> = vec![lit_true.clone(), col_b_expr.clone()]; - - let vec3: Vec> = vec![lit2.clone(), lit1.clone()]; + let vec1 = vec![lit_true.clone(), lit_false.clone()]; + let vec2 = vec![lit_true.clone(), col_b_expr.clone()]; + let vec3 = vec![lit2.clone(), lit1.clone()]; // lit_true is common assert!(have_common_entries(&vec1, &vec2)); - // there is no common entry assert!(!have_common_entries(&vec1, &vec3)); assert!(!have_common_entries(&vec2, &vec3)); - Ok(()) } #[test] - fn test_physical_exprs_equal() -> Result<()> { + fn test_physical_exprs_equal() { let lit_true = Arc::new(Literal::new(ScalarValue::Boolean(Some(true)))) as Arc; let lit_false = Arc::new(Literal::new(ScalarValue::Boolean(Some(false)))) as Arc; - let lit2 = - Arc::new(Literal::new(ScalarValue::Int32(Some(2)))) as Arc; let lit1 = Arc::new(Literal::new(ScalarValue::Int32(Some(1)))) as Arc; + let lit2 = + Arc::new(Literal::new(ScalarValue::Int32(Some(2)))) as Arc; let col_b_expr = Arc::new(Column::new("b", 1)) as Arc; - let vec1: Vec> = vec![lit_true.clone(), lit_false.clone()]; - - let vec2: Vec> = vec![lit_true.clone(), col_b_expr.clone()]; - - let vec3: Vec> = vec![lit2.clone(), lit1.clone()]; - - let vec4: Vec> = vec![lit_true.clone(), lit_false.clone()]; + let vec1 = vec![lit_true.clone(), lit_false.clone()]; + let vec2 = vec![lit_true.clone(), col_b_expr.clone()]; + let vec3 = vec![lit2.clone(), lit1.clone()]; + let vec4 = vec![lit_true.clone(), lit_false.clone()]; // these vectors are same assert!(physical_exprs_equal(&vec1, &vec1)); assert!(physical_exprs_equal(&vec1, &vec4)); + assert!(physical_exprs_bag_equal(&vec1, &vec1)); + assert!(physical_exprs_bag_equal(&vec1, &vec4)); // these vectors are different assert!(!physical_exprs_equal(&vec1, &vec2)); assert!(!physical_exprs_equal(&vec1, &vec3)); + assert!(!physical_exprs_bag_equal(&vec1, &vec2)); + assert!(!physical_exprs_bag_equal(&vec1, &vec3)); + } + + #[test] + fn test_physical_exprs_set_equal() { + let list1: Vec> = vec![ + Arc::new(Column::new("a", 0)), + Arc::new(Column::new("a", 0)), + Arc::new(Column::new("b", 1)), + ]; + let list2: Vec> = vec![ + Arc::new(Column::new("b", 1)), + Arc::new(Column::new("b", 1)), + Arc::new(Column::new("a", 0)), + ]; + assert!(!physical_exprs_bag_equal( + list1.as_slice(), + list2.as_slice() + )); + assert!(!physical_exprs_bag_equal( + list2.as_slice(), + list1.as_slice() + )); + assert!(!physical_exprs_equal(list1.as_slice(), list2.as_slice())); + assert!(!physical_exprs_equal(list2.as_slice(), list1.as_slice())); + + let list3: Vec> = vec![ + Arc::new(Column::new("a", 0)), + Arc::new(Column::new("b", 1)), + Arc::new(Column::new("c", 2)), + Arc::new(Column::new("a", 0)), + Arc::new(Column::new("b", 1)), + ]; + let list4: Vec> = vec![ + Arc::new(Column::new("b", 1)), + Arc::new(Column::new("b", 1)), + Arc::new(Column::new("a", 0)), + Arc::new(Column::new("c", 2)), + Arc::new(Column::new("a", 0)), + ]; + assert!(physical_exprs_bag_equal(list3.as_slice(), list4.as_slice())); + assert!(physical_exprs_bag_equal(list4.as_slice(), list3.as_slice())); + assert!(physical_exprs_bag_equal(list3.as_slice(), list3.as_slice())); + assert!(physical_exprs_bag_equal(list4.as_slice(), list4.as_slice())); + assert!(!physical_exprs_equal(list3.as_slice(), list4.as_slice())); + assert!(!physical_exprs_equal(list4.as_slice(), list3.as_slice())); + assert!(physical_exprs_bag_equal(list3.as_slice(), list3.as_slice())); + assert!(physical_exprs_bag_equal(list4.as_slice(), list4.as_slice())); + } + + #[test] + fn test_deduplicate_physical_exprs() { + let lit_true = &(Arc::new(Literal::new(ScalarValue::Boolean(Some(true)))) + as Arc); + let lit_false = &(Arc::new(Literal::new(ScalarValue::Boolean(Some(false)))) + as Arc); + let lit4 = &(Arc::new(Literal::new(ScalarValue::Int32(Some(4)))) + as Arc); + let lit2 = &(Arc::new(Literal::new(ScalarValue::Int32(Some(2)))) + as Arc); + let col_a_expr = &(Arc::new(Column::new("a", 0)) as Arc); + let col_b_expr = &(Arc::new(Column::new("b", 1)) as Arc); - Ok(()) + // First vector in the tuple is arguments, second one is the expected value. + let test_cases = vec![ + // ---------- TEST CASE 1----------// + ( + vec![ + lit_true, lit_false, lit4, lit2, col_a_expr, col_a_expr, col_b_expr, + lit_true, lit2, + ], + vec![lit_true, lit_false, lit4, lit2, col_a_expr, col_b_expr], + ), + // ---------- TEST CASE 2----------// + ( + vec![lit_true, lit_true, lit_false, lit4], + vec![lit_true, lit4, lit_false], + ), + ]; + for (physical_exprs, expected) in test_cases { + let mut physical_exprs = + physical_exprs.into_iter().cloned().collect::>(); + let expected = expected.into_iter().cloned().collect::>(); + deduplicate_physical_exprs(&mut physical_exprs); + assert!(physical_exprs_equal(&physical_exprs, &expected)); + } } } diff --git a/datafusion/physical-expr/src/utils.rs b/datafusion/physical-expr/src/utils.rs index 85c04df6b42c..8f80a34eec07 100644 --- a/datafusion/physical-expr/src/utils.rs +++ b/datafusion/physical-expr/src/utils.rs @@ -35,30 +35,6 @@ use itertools::Itertools; use petgraph::graph::NodeIndex; use petgraph::stable_graph::StableGraph; -/// Compare the two expr lists are equal no matter the order. -/// For example two InListExpr can be considered to be equals no matter the order: -/// -/// In('a','b','c') == In('c','b','a') -pub fn expr_list_eq_any_order( - list1: &[Arc], - list2: &[Arc], -) -> bool { - if list1.len() == list2.len() { - let mut expr_vec1 = list1.to_vec(); - let mut expr_vec2 = list2.to_vec(); - while let Some(expr1) = expr_vec1.pop() { - if let Some(idx) = expr_vec2.iter().position(|expr2| expr1.eq(expr2)) { - expr_vec2.swap_remove(idx); - } else { - break; - } - } - expr_vec1.is_empty() && expr_vec2.is_empty() - } else { - false - } -} - /// Assume the predicate is in the form of CNF, split the predicate to a Vec of PhysicalExprs. /// /// For example, split "a1 = a2 AND b1 <= b2 AND c1 != c2" into ["a1 = a2", "b1 <= b2", "c1 != c2"] @@ -392,7 +368,7 @@ mod tests { use super::*; use crate::equivalence::SchemaProperties; use crate::expressions::{binary, cast, col, in_list, lit, Column, Literal}; - use crate::{physical_exprs_equal, PhysicalSortExpr}; + use crate::PhysicalSortExpr; use arrow::compute::SortOptions; use arrow_array::Int32Array; @@ -535,51 +511,6 @@ mod tests { assert_eq!(get_indices_of_exprs_strict(&list2, &list1), vec![1, 2, 0]); } - #[test] - fn expr_list_eq_test() -> Result<()> { - let list1: Vec> = vec![ - Arc::new(Column::new("a", 0)), - Arc::new(Column::new("a", 0)), - Arc::new(Column::new("b", 1)), - ]; - let list2: Vec> = vec![ - Arc::new(Column::new("b", 1)), - Arc::new(Column::new("b", 1)), - Arc::new(Column::new("a", 0)), - ]; - assert!(!expr_list_eq_any_order(list1.as_slice(), list2.as_slice())); - assert!(!expr_list_eq_any_order(list2.as_slice(), list1.as_slice())); - - assert!(!physical_exprs_equal(list1.as_slice(), list2.as_slice())); - assert!(!physical_exprs_equal(list2.as_slice(), list1.as_slice())); - - let list3: Vec> = vec![ - Arc::new(Column::new("a", 0)), - Arc::new(Column::new("b", 1)), - Arc::new(Column::new("c", 2)), - Arc::new(Column::new("a", 0)), - Arc::new(Column::new("b", 1)), - ]; - let list4: Vec> = vec![ - Arc::new(Column::new("b", 1)), - Arc::new(Column::new("b", 1)), - Arc::new(Column::new("a", 0)), - Arc::new(Column::new("c", 2)), - Arc::new(Column::new("a", 0)), - ]; - assert!(expr_list_eq_any_order(list3.as_slice(), list4.as_slice())); - assert!(expr_list_eq_any_order(list4.as_slice(), list3.as_slice())); - assert!(expr_list_eq_any_order(list3.as_slice(), list3.as_slice())); - assert!(expr_list_eq_any_order(list4.as_slice(), list4.as_slice())); - - assert!(!physical_exprs_equal(list3.as_slice(), list4.as_slice())); - assert!(!physical_exprs_equal(list4.as_slice(), list3.as_slice())); - assert!(expr_list_eq_any_order(list3.as_slice(), list3.as_slice())); - assert!(expr_list_eq_any_order(list4.as_slice(), list4.as_slice())); - - Ok(()) - } - #[test] fn test_reassign_predicate_columns_in_list() { let int_field = Field::new("should_not_matter", DataType::Int64, true); From 4e845e4aa0f481d36c2d82e00e28e84e35eb7ffe Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 25 Oct 2023 16:44:11 +0300 Subject: [PATCH 093/122] Add unit tests for remove_redundant entries --- datafusion/physical-expr/src/equivalence.rs | 96 ++++++++++++++++++- datafusion/physical-expr/src/lib.rs | 2 +- datafusion/physical-expr/src/physical_expr.rs | 4 +- datafusion/physical-plan/src/union.rs | 1 - 4 files changed, 98 insertions(+), 5 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 15b4e81b2a25..7da631f19595 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -1955,7 +1955,7 @@ mod tests { } #[test] - fn test_remove_redundant_entries() -> Result<()> { + fn test_remove_redundant_entries_eq_group() -> Result<()> { let entries = vec![ vec![lit(1), lit(1), lit(2)], // This group is meaningless should be removed @@ -1978,6 +1978,100 @@ mod tests { Ok(()) } + #[test] + fn test_remove_redundant_entries_oeq_class() -> Result<()> { + let schema = create_test_schema()?; + let col_a = &col("a", &schema)?; + let col_b = &col("b", &schema)?; + let col_c = &col("c", &schema)?; + + let option_asc = SortOptions { + descending: false, + nulls_first: false, + }; + let option_desc = SortOptions { + descending: true, + nulls_first: true, + }; + + let test_cases = vec![ + // ------- TEST CASE 1 --------- + ( + // ORDERINGS GIVEN + vec![ + // [a ASC, b ASC] + vec![(col_a, option_asc), (col_b, option_asc)], + ], + // EXPECTED orderings that is succint. + vec![ + // [a ASC, b ASC] + vec![(col_a, option_asc), (col_b, option_asc)], + ], + ), + // ------- TEST CASE 2 --------- + ( + // ORDERINGS GIVEN + vec![ + // [a ASC, b ASC] + vec![(col_a, option_asc), (col_b, option_asc)], + // [a ASC, b ASC, c ASC] + vec![ + (col_a, option_asc), + (col_b, option_asc), + (col_c, option_asc), + ], + ], + // EXPECTED orderings that is succint. + vec![ + // [a ASC, b ASC, c ASC] + vec![ + (col_a, option_asc), + (col_b, option_asc), + (col_c, option_asc), + ], + ], + ), + // ------- TEST CASE 3 --------- + ( + // ORDERINGS GIVEN + vec![ + // [a ASC, b DESC] + vec![(col_a, option_asc), (col_b, option_desc)], + // [a ASC] + vec![(col_a, option_asc)], + // [a ASC, c ASC] + vec![(col_a, option_asc), (col_c, option_asc)], + ], + // EXPECTED orderings that is succint. + vec![ + // [a ASC, b DESC] + vec![(col_a, option_asc), (col_b, option_desc)], + // [a ASC, c ASC] + vec![(col_a, option_asc), (col_c, option_asc)], + ], + ), + ]; + for (orderings, expected) in test_cases { + let orderings = orderings + .into_iter() + .map(|sort_exprs| convert_to_sort_exprs(&sort_exprs)) + .collect::>(); + let expected: Vec = expected + .into_iter() + .map(|sort_exprs| convert_to_sort_exprs(&sort_exprs)) + .collect::>(); + let actual = OrderingEquivalenceClass::new(orderings.clone()); + let actual = actual.orderings; + let err_msg = format!("orderings: {:?}, expected: {:?}", orderings, expected); + assert_eq!(actual.len(), expected.len(), "{}", err_msg); + for elem in actual { + assert!(expected.contains(&elem), "{}", err_msg); + } + } + + Ok(()) + } + #[test] fn test_get_updated_right_ordering_equivalence_properties() -> Result<()> { let join_type = JoinType::Inner; diff --git a/datafusion/physical-expr/src/lib.rs b/datafusion/physical-expr/src/lib.rs index 09c64ab7a0c4..2b6123910912 100644 --- a/datafusion/physical-expr/src/lib.rs +++ b/datafusion/physical-expr/src/lib.rs @@ -59,7 +59,7 @@ pub use equivalence::{ pub use partitioning::{Distribution, Partitioning}; pub use physical_expr::{ - physical_exprs_contains, physical_exprs_equal, physical_exprs_bag_equal, + physical_exprs_bag_equal, physical_exprs_contains, physical_exprs_equal, PhysicalExpr, PhysicalExprRef, }; pub use planner::create_physical_expr; diff --git a/datafusion/physical-expr/src/physical_expr.rs b/datafusion/physical-expr/src/physical_expr.rs index 4dbb798309f4..a3f8f33b24a8 100644 --- a/datafusion/physical-expr/src/physical_expr.rs +++ b/datafusion/physical-expr/src/physical_expr.rs @@ -293,8 +293,8 @@ mod tests { use crate::expressions::{Column, Literal}; use crate::physical_expr::{ - deduplicate_physical_exprs, have_common_entries, physical_exprs_contains, - physical_exprs_equal, physical_exprs_bag_equal, PhysicalExpr, + deduplicate_physical_exprs, have_common_entries, physical_exprs_bag_equal, + physical_exprs_contains, physical_exprs_equal, PhysicalExpr, }; use datafusion_common::ScalarValue; diff --git a/datafusion/physical-plan/src/union.rs b/datafusion/physical-plan/src/union.rs index a237e7de7ef2..12c59a135735 100644 --- a/datafusion/physical-plan/src/union.rs +++ b/datafusion/physical-plan/src/union.rs @@ -860,7 +860,6 @@ mod tests { let union = UnionExec::new(vec![child1, child2]); let union_schema_properties = union.schema_properties(); let union_actual_orderings = union_schema_properties.oeq_class(); - println!("union_orderings:{:?}", union_actual_orderings); let err_msg = format!( "Error in test id: {:?}, test case: {:?}", test_idx, test_cases[test_idx] From 39793718cf307cfc49bbab8981ab641b2c7d7d69 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 25 Oct 2023 16:50:48 +0300 Subject: [PATCH 094/122] Simplifications --- datafusion/physical-expr/src/equivalence.rs | 40 +++++++++------------ 1 file changed, 16 insertions(+), 24 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 7da631f19595..f8040164fb00 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -1392,10 +1392,7 @@ mod tests { (col_g, option_asc), ], ]; - let orderings = orderings - .into_iter() - .map(|sort_exprs| convert_to_sort_exprs(&sort_exprs)) - .collect::>(); + let orderings = convert_to_orderings(&orderings); schema_properties.add_new_orderings(&orderings); Ok((test_schema, schema_properties)) } @@ -1501,6 +1498,16 @@ mod tests { .collect::>() } + // Convert each inner tuple to PhysicalSortExpr + fn convert_to_orderings( + orderings: &[Vec<(&Arc, SortOptions)>], + ) -> Vec> { + orderings + .iter() + .map(|sort_exprs| convert_to_sort_exprs(sort_exprs)) + .collect() + } + #[test] fn add_equal_conditions_test() -> Result<()> { let schema = Arc::new(Schema::new(vec![ @@ -1868,10 +1875,7 @@ mod tests { vec![(col_e, options)], vec![(col_d, options), (col_f, options)], ]; - let orderings = orderings - .into_iter() - .map(|sort_exprs| convert_to_sort_exprs(&sort_exprs)) - .collect::>(); + let orderings = convert_to_orderings(&orderings); // Column [a ASC], [e ASC], [d ASC, f ASC] are all valid orderings for the schema. schema_properties.add_new_orderings(&orderings); @@ -2052,14 +2056,8 @@ mod tests { ), ]; for (orderings, expected) in test_cases { - let orderings = orderings - .into_iter() - .map(|sort_exprs| convert_to_sort_exprs(&sort_exprs)) - .collect::>(); - let expected: Vec = expected - .into_iter() - .map(|sort_exprs| convert_to_sort_exprs(&sort_exprs)) - .collect::>(); + let orderings = convert_to_orderings(&orderings); + let expected = convert_to_orderings(&expected); let actual = OrderingEquivalenceClass::new(orderings.clone()); let actual = actual.orderings; let err_msg = format!("orderings: {:?}, expected: {:?}", orderings, expected); @@ -2090,10 +2088,7 @@ mod tests { vec![(col_x, options), (col_y, options)], vec![(col_z, options), (col_w, options)], ]; - let orderings = orderings - .into_iter() - .map(|sort_exprs| convert_to_sort_exprs(&sort_exprs)) - .collect::>(); + let orderings = convert_to_orderings(&orderings); // Right child ordering equivalences let right_oeq_class = OrderingEquivalenceClass::new(orderings); @@ -2129,10 +2124,7 @@ mod tests { vec![(col_x, options), (col_y, options)], vec![(col_z, options), (col_w, options)], ]; - let orderings = orderings - .into_iter() - .map(|sort_exprs| convert_to_sort_exprs(&sort_exprs)) - .collect::>(); + let orderings = convert_to_orderings(&orderings); let expected = OrderingEquivalenceClass::new(orderings); assert_eq!(result, expected); From fef62d6c6ed4d733f5549761c8dc53f8b9f86227 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Wed, 25 Oct 2023 17:22:46 +0300 Subject: [PATCH 095/122] Review Part 10 --- datafusion/physical-expr/src/equivalence.rs | 141 +++++++----------- datafusion/physical-expr/src/physical_expr.rs | 24 ++- 2 files changed, 68 insertions(+), 97 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index f8040164fb00..b017a7b976aa 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -230,11 +230,12 @@ impl EquivalenceGroup { &self, sort_reqs: &[PhysicalSortRequirement], ) -> Vec { - let normalized_sort_reqs = sort_reqs - .iter() - .map(|sort_req| self.normalize_sort_requirement(sort_req.clone())) - .collect::>(); - collapse_lex_req(normalized_sort_reqs) + collapse_lex_req( + sort_reqs + .iter() + .map(|sort_req| self.normalize_sort_requirement(sort_req.clone())) + .collect(), + ) } /// This function applies the `normalize_sort_expr` function for all sort @@ -322,7 +323,7 @@ impl EquivalenceGroup { .collect::>(); (new_class.len() > 1).then_some(new_class) }); - // TODO: Convert below algorithm to the version that use HashMap. + // TODO: Convert the algorithm below to a version that uses `HashMap`. // once `Arc` can be stored in `HashMap`. let mut new_classes = vec![]; for (source, target) in mapping { @@ -388,7 +389,7 @@ impl EquivalenceGroup { } // In we have an inner join, expressions in the "on" condition are equal // at the resulting table. - if *join_type == JoinType::Inner { + if join_type == &JoinType::Inner { for (lhs, rhs) in on.iter() { let index = rhs.index() + left_column_count; let new_lhs = Arc::new(lhs.clone()) as _; @@ -438,11 +439,9 @@ impl OrderingEquivalenceClass { /// Adds `ordering` to this equivalence class. fn push(&mut self, ordering: LexOrdering) { - if !self.contains(&ordering) { - self.orderings.push(ordering); - // Make sure that there are no redundant orderings: - self.remove_redundant_entries(); - } + self.orderings.push(ordering); + // Make sure that there are no redundant orderings: + self.remove_redundant_entries(); } /// Checks whether this ordering equivalence class is empty. @@ -467,44 +466,40 @@ impl OrderingEquivalenceClass { /// Extend this ordering equivalence class with the `other` class. pub fn extend(&mut self, other: OrderingEquivalenceClass) { - for ordering in other.iter() { - if !self.contains(ordering) { - self.orderings.push(ordering.clone()) - } - } + self.orderings.extend(other.orderings); + // Make sure that there are no redundant orderings: + self.remove_redundant_entries(); } /// Adds new orderings into this ordering equivalence class. pub fn add_new_orderings(&mut self, orderings: &[LexOrdering]) { - for ordering in orderings.iter() { - self.push(ordering.clone()); - } + self.orderings.extend(orderings.iter().cloned()); + // Make sure that there are no redundant orderings: + self.remove_redundant_entries(); } /// Removes redundant orderings from the state. - /// For instance, If we already know that - /// ordering: [a ASC, b ASC, c DESC] is valid for the schema. - /// There is no need to keep ordering [a ASC, b ASC] in the state. + /// For instance, If we already have the ordering [a ASC, b ASC, c DESC], + /// then there is no need to keep ordering [a ASC, b ASC] in the state. fn remove_redundant_entries(&mut self) { - // Make sure there is no redundant entry - let mut res: Vec = vec![]; - for ordering in self.iter() { - let mut is_inside = false; - for item in &mut res { - if let Some(finer) = Self::get_finer_strict(item, ordering) { - *item = finer.to_vec(); - is_inside = true; + let mut idx = 0; + while idx < self.orderings.len() { + let mut removal = false; + for ordering in self.orderings[0..idx].iter() { + if is_finer(ordering, &self.orderings[idx]) { + self.orderings.swap_remove(idx); + removal = true; + break; } } - if !is_inside { - res.push(ordering.clone()); + if !removal { + idx += 1; } } - self.orderings = res; } - /// Get first ordering entry in the ordering equivalences - /// This is one of the many valid orderings (if available) + /// Gets the first ordering entry in this ordering equivalence class. + /// This is one of the many valid orderings (if there are multiple). pub fn output_ordering(&self) -> Option> { self.orderings.first().cloned() } @@ -538,21 +533,6 @@ impl OrderingEquivalenceClass { ) } - /// Return finer ordering between lhs and rhs. - fn get_finer_strict<'a>( - lhs: &'a [PhysicalSortExpr], - rhs: &'a [PhysicalSortExpr], - ) -> Option<&'a [PhysicalSortExpr]> { - if izip!(lhs.iter(), rhs.iter()).all(|(lhs, rhs)| lhs.eq(rhs)) { - if lhs.len() > rhs.len() { - return Some(lhs); - } else { - return Some(rhs); - } - } - None - } - /// Get leading ordering of the expression if it is ordered. /// `None` means expression is not ordered. fn get_ordering(&self, expr: &Arc) -> Option { @@ -566,6 +546,11 @@ impl OrderingEquivalenceClass { } } +/// Returns `true` if the ordering `lhs` is at least as fine as the ordering `rhs`. +fn is_finer(lhs: &[PhysicalSortExpr], rhs: &[PhysicalSortExpr]) -> bool { + lhs.len() >= rhs.len() && lhs.iter().zip(rhs.iter()).all(|(lhs, rhs)| lhs.eq(rhs)) +} + /// `SchemaProperties` keeps track of useful information related to schema. /// Currently, it keeps track of /// - Equivalent columns, e.g columns that have same value. @@ -666,7 +651,7 @@ impl SchemaProperties { /// Extends `SchemaProperties` by adding ordering inside the `other` /// to the `self.oeq_class`. - pub fn add_ordering_equivalent_group(&mut self, other: OrderingEquivalenceClass) { + pub fn add_ordering_equivalence_class(&mut self, other: OrderingEquivalenceClass) { for ordering in other.into_iter() { if !self.oeq_class.contains(&ordering) { self.oeq_class.push(ordering); @@ -674,18 +659,20 @@ impl SchemaProperties { } } - /// Adds new ordering into the ordering equivalent class. + /// Adds new orderings into the existing ordering equivalence class. pub fn add_new_orderings(&mut self, orderings: &[LexOrdering]) { self.oeq_class.add_new_orderings(orderings); } - /// Add new equivalence group to state. - pub fn add_equivalent_groups(&mut self, other_eq_group: EquivalenceGroup) { + /// Incorporates the given equivalence group to into the existing + /// equivalence group in this schema. + pub fn add_equivalence_group(&mut self, other_eq_group: EquivalenceGroup) { self.eq_group.extend(other_eq_group); } - /// Adds new equality group into the equivalent groups. - /// If equalities are new, otherwise extends corresponding group. + /// Adds a new equality condition into the existing equivalence group. + /// If the given equality defines a new equivalence class, adds this new + /// equivalence class to the equivalence group. pub fn add_equal_conditions( &mut self, left: &Arc, @@ -1097,7 +1084,7 @@ pub fn join_schema_properties( let join_eq_groups = left.eq_group() .join(join_type, right.eq_group(), left_columns_len, on); - new_properties.add_equivalent_groups(join_eq_groups); + new_properties.add_equivalence_group(join_eq_groups); // All joins have 2 children assert_eq!(maintains_input_order.len(), 2); @@ -1112,7 +1099,7 @@ pub fn join_schema_properties( (true, false) => { // In this special case, right side ordering can be prefixed with left side ordering. if let (Some(JoinSide::Left), JoinType::Inner) = (probe_side, join_type) { - let updated_right_oeq = get_updated_right_ordering_equivalent_group( + let updated_right_oeq = get_updated_right_ordering_equivalence_class( join_type, right_oeq_class, left_columns_len, @@ -1127,13 +1114,13 @@ pub fn join_schema_properties( // for the right table should be converted to `a ASC, b ASC` before it is added // to the ordering equivalences of the join. let out_oeq_class = left_oeq_class.join_postfix(&updated_right_oeq); - new_properties.add_ordering_equivalent_group(out_oeq_class); + new_properties.add_ordering_equivalence_class(out_oeq_class); } else { - new_properties.add_ordering_equivalent_group(left_oeq_class); + new_properties.add_ordering_equivalence_class(left_oeq_class); } } (false, true) => { - let updated_right_oeq = get_updated_right_ordering_equivalent_group( + let updated_right_oeq = get_updated_right_ordering_equivalence_class( join_type, right_oeq_class, left_columns_len, @@ -1149,9 +1136,9 @@ pub fn join_schema_properties( // for the right table should be converted to `a ASC, b ASC` before it is added // to the ordering equivalences of the join. let out_oeq_class = updated_right_oeq.join_postfix(&left_oeq_class); - new_properties.add_ordering_equivalent_group(out_oeq_class); + new_properties.add_ordering_equivalence_class(out_oeq_class); } else { - new_properties.add_ordering_equivalent_group(updated_right_oeq); + new_properties.add_ordering_equivalence_class(updated_right_oeq); } } (false, false) => {} @@ -1167,7 +1154,7 @@ pub fn schema_properties_helper( let mut oep = SchemaProperties::new(schema); if !orderings.is_empty() { let group = OrderingEquivalenceClass::new(orderings.to_vec()); - oep.add_ordering_equivalent_group(group); + oep.add_ordering_equivalence_class(group); } oep } @@ -1307,7 +1294,7 @@ fn update_ordering( /// /// This way; once we normalize an expression according to equivalence properties, /// it can thereafter safely be used for ordering equivalence normalization. -fn get_updated_right_ordering_equivalent_group( +fn get_updated_right_ordering_equivalence_class( join_type: &JoinType, right_oeq_group: OrderingEquivalenceClass, left_columns_len: usize, @@ -1327,7 +1314,7 @@ mod tests { use super::*; use crate::expressions::{col, lit, Column}; - use crate::physical_expr::physical_exprs_equal; + use crate::physical_expr::{physical_exprs_bag_equal, physical_exprs_equal}; use arrow::compute::{lexsort_to_indices, SortColumn}; use arrow::datatypes::{DataType, Field, Schema}; @@ -1459,20 +1446,6 @@ mod tests { Ok((test_schema, schema_properties)) } - /// Checks whether the given physical expression slices are equal. - /// If any permutation is equal returns true. - pub fn physical_exprs_set_equal( - lhs: &[Arc], - rhs: &[Arc], - ) -> bool { - if lhs.len() != rhs.len() { - false - } else { - lhs.iter() - .all(|lhs_entry| physical_exprs_contains(rhs, lhs_entry)) - } - } - // Convert each tuple to PhysicalSortRequirement fn convert_to_sort_reqs( in_data: &[(&Arc, Option)], @@ -1949,7 +1922,7 @@ mod tests { assert_eq!(eq_groups.len(), expected.len(), "{}", err_msg); for idx in 0..eq_groups.len() { assert!( - physical_exprs_set_equal(&eq_groups[idx], &expected[idx]), + physical_exprs_bag_equal(&eq_groups[idx], &expected[idx]), "{}", err_msg ); @@ -2112,12 +2085,12 @@ mod tests { join_schema_properties.add_equal_conditions(col_a, col_x); join_schema_properties.add_equal_conditions(col_d, col_w); - let result = get_updated_right_ordering_equivalent_group( + let result = get_updated_right_ordering_equivalence_class( &join_type, right_oeq_class, left_columns_len, ); - join_schema_properties.add_ordering_equivalent_group(result); + join_schema_properties.add_ordering_equivalence_class(result); let result = join_schema_properties.oeq_class().clone(); let orderings = vec![ diff --git a/datafusion/physical-expr/src/physical_expr.rs b/datafusion/physical-expr/src/physical_expr.rs index a3f8f33b24a8..7f70de80e4df 100644 --- a/datafusion/physical-expr/src/physical_expr.rs +++ b/datafusion/physical-expr/src/physical_expr.rs @@ -267,18 +267,17 @@ pub fn physical_exprs_bag_equal( } } -/// This utility function removes duplicates from the given `physical_exprs` -/// vector. Note that this function does not necessarily preserve its input -/// ordering. -pub fn deduplicate_physical_exprs(physical_exprs: &mut Vec>) { +/// This utility function removes duplicates from the given `exprs` vector. +/// Note that this function does not necessarily preserve its input ordering. +pub fn deduplicate_physical_exprs(exprs: &mut Vec>) { // TODO: Once we can use `HashSet`s with `Arc`, this // function should use a `HashSet` to reduce computational complexity. let mut idx = 0; - while idx < physical_exprs.len() { + while idx < exprs.len() { let mut rest_idx = idx + 1; - while rest_idx < physical_exprs.len() { - if physical_exprs[idx].eq(&physical_exprs[rest_idx]) { - physical_exprs.swap_remove(rest_idx); + while rest_idx < exprs.len() { + if exprs[idx].eq(&exprs[rest_idx]) { + exprs.swap_remove(rest_idx); } else { rest_idx += 1; } @@ -463,12 +462,11 @@ mod tests { vec![lit_true, lit4, lit_false], ), ]; - for (physical_exprs, expected) in test_cases { - let mut physical_exprs = - physical_exprs.into_iter().cloned().collect::>(); + for (exprs, expected) in test_cases { + let mut exprs = exprs.into_iter().cloned().collect::>(); let expected = expected.into_iter().cloned().collect::>(); - deduplicate_physical_exprs(&mut physical_exprs); - assert!(physical_exprs_equal(&physical_exprs, &expected)); + deduplicate_physical_exprs(&mut exprs); + assert!(physical_exprs_equal(&exprs, &expected)); } } } From bf011544ef8184dc40c79ff44b3f695d03c57222 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 25 Oct 2023 17:54:19 +0300 Subject: [PATCH 096/122] Fix test --- datafusion/physical-expr/src/equivalence.rs | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index b017a7b976aa..2c62593a6546 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -485,11 +485,15 @@ impl OrderingEquivalenceClass { let mut idx = 0; while idx < self.orderings.len() { let mut removal = false; - for ordering in self.orderings[0..idx].iter() { + for (ordering_idx, ordering) in self.orderings[0..idx].iter().enumerate() { if is_finer(ordering, &self.orderings[idx]) { self.orderings.swap_remove(idx); removal = true; break; + } else if is_finer(&self.orderings[idx], ordering) { + self.orderings.swap_remove(ordering_idx); + removal = true; + break; } } if !removal { @@ -2033,7 +2037,10 @@ mod tests { let expected = convert_to_orderings(&expected); let actual = OrderingEquivalenceClass::new(orderings.clone()); let actual = actual.orderings; - let err_msg = format!("orderings: {:?}, expected: {:?}", orderings, expected); + let err_msg = format!( + "orderings: {:?}, expected: {:?}, actual :{:?}", + orderings, expected, actual + ); assert_eq!(actual.len(), expected.len(), "{}", err_msg); for elem in actual { assert!(expected.contains(&elem), "{}", err_msg); From dfd2060f98058816536dbecee7c3f9da315cab03 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Wed, 25 Oct 2023 18:31:53 +0300 Subject: [PATCH 097/122] Add new test case, fix implementation --- datafusion/physical-expr/src/equivalence.rs | 48 +++++++++++++++++---- 1 file changed, 39 insertions(+), 9 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 2c62593a6546..d95d00794e9f 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -486,17 +486,18 @@ impl OrderingEquivalenceClass { while idx < self.orderings.len() { let mut removal = false; for (ordering_idx, ordering) in self.orderings[0..idx].iter().enumerate() { - if is_finer(ordering, &self.orderings[idx]) { - self.orderings.swap_remove(idx); - removal = true; - break; - } else if is_finer(&self.orderings[idx], ordering) { - self.orderings.swap_remove(ordering_idx); + if let Some(left_finer) = finer_side(ordering, &self.orderings[idx]) { + if !left_finer { + // Right side is finer + self.orderings.swap(ordering_idx, idx); + } removal = true; break; } } - if !removal { + if removal { + self.orderings.swap_remove(idx); + } else { idx += 1; } } @@ -551,8 +552,12 @@ impl OrderingEquivalenceClass { } /// Returns `true` if the ordering `lhs` is at least as fine as the ordering `rhs`. -fn is_finer(lhs: &[PhysicalSortExpr], rhs: &[PhysicalSortExpr]) -> bool { - lhs.len() >= rhs.len() && lhs.iter().zip(rhs.iter()).all(|(lhs, rhs)| lhs.eq(rhs)) +/// Returns `false` if the ordering `rhs` is finer than the ordering `lhs`. +/// Returns `None`, if sort_exprs are not compatible. +fn finer_side(lhs: &[PhysicalSortExpr], rhs: &[PhysicalSortExpr]) -> Option { + let left_larger = lhs.len() >= rhs.len(); + let all_equal = lhs.iter().zip(rhs.iter()).all(|(lhs, rhs)| lhs.eq(rhs)); + all_equal.then_some(left_larger) } /// `SchemaProperties` keeps track of useful information related to schema. @@ -2031,6 +2036,31 @@ mod tests { vec![(col_a, option_asc), (col_c, option_asc)], ], ), + // ------- TEST CASE 4 --------- + ( + // ORDERINGS GIVEN + vec![ + // [a ASC, b ASC] + vec![(col_a, option_asc), (col_b, option_asc)], + // [a ASC, b ASC, c ASC] + vec![ + (col_a, option_asc), + (col_b, option_asc), + (col_c, option_asc), + ], + // [a ASC] + vec![(col_a, option_asc)], + ], + // EXPECTED orderings that is succint. + vec![ + // [a ASC, b ASC, c ASC] + vec![ + (col_a, option_asc), + (col_b, option_asc), + (col_c, option_asc), + ], + ], + ), ]; for (orderings, expected) in test_cases { let orderings = convert_to_orderings(&orderings); From 29c8ae2979eb390998af48a5024f5424492d6838 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Wed, 25 Oct 2023 23:04:05 +0300 Subject: [PATCH 098/122] Review Part 11 --- datafusion/physical-expr/src/equivalence.rs | 96 +++++++++------------ 1 file changed, 41 insertions(+), 55 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index d95d00794e9f..7ff0ee3baa9a 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -340,15 +340,11 @@ impl EquivalenceGroup { } // Only add equivalence classes with at least two members as singleton // equivalence classes are meaningless. - EquivalenceGroup::new( - projected_classes - .chain( - new_classes - .into_iter() - .filter_map(|(_, values)| (values.len() > 1).then_some(values)), - ) - .collect(), - ) + let new_classes = new_classes + .into_iter() + .filter_map(|(_, values)| (values.len() > 1).then_some(values)); + let classes = projected_classes.chain(new_classes).collect(); + EquivalenceGroup::new(classes) } /// Returns the equivalence class that contains `expr`. @@ -376,7 +372,7 @@ impl EquivalenceGroup { result.extend(self.clone()); let updated_eq_classes = right_equivalences .iter() - .map(|cls| add_offset_to_exprs(cls.to_vec(), left_column_count)) + .map(|cls| add_offset_to_exprs(cls, left_column_count)) .collect(); result.extend(EquivalenceGroup::new(updated_eq_classes)); } @@ -486,9 +482,8 @@ impl OrderingEquivalenceClass { while idx < self.orderings.len() { let mut removal = false; for (ordering_idx, ordering) in self.orderings[0..idx].iter().enumerate() { - if let Some(left_finer) = finer_side(ordering, &self.orderings[idx]) { - if !left_finer { - // Right side is finer + if let Some(right_finer) = finer_side(ordering, &self.orderings[idx]) { + if right_finer { self.orderings.swap(ordering_idx, idx); } removal = true; @@ -528,22 +523,22 @@ impl OrderingEquivalenceClass { OrderingEquivalenceClass::new(res) } - /// Adds `offset` value to the index of each expression inside `OrderingEquivalentGroup`. - pub fn add_offset(&self, offset: usize) -> OrderingEquivalenceClass { - OrderingEquivalenceClass::new( - self.orderings - .iter() - .map(|ordering| add_offset_to_lex_ordering(ordering, offset)) - .collect(), - ) + /// Adds `offset` value to the index of each expression inside this + /// ordering equivalence class. + pub fn add_offset(&mut self, offset: usize) { + for ordering in self.orderings.iter_mut() { + for sort_expr in ordering { + sort_expr.expr = add_offset_to_expr(sort_expr.expr.clone(), offset); + } + } } - /// Get leading ordering of the expression if it is ordered. - /// `None` means expression is not ordered. - fn get_ordering(&self, expr: &Arc) -> Option { + /// Gets sort options associated with this expression if it is a leading + /// ordering expression. Otherwise, returns `None`. + fn get_options(&self, expr: &Arc) -> Option { for ordering in self.iter() { let leading_ordering = &ordering[0]; - if expr.eq(&leading_ordering.expr) { + if leading_ordering.expr.eq(expr) { return Some(leading_ordering.options); } } @@ -551,13 +546,12 @@ impl OrderingEquivalenceClass { } } -/// Returns `true` if the ordering `lhs` is at least as fine as the ordering `rhs`. -/// Returns `false` if the ordering `rhs` is finer than the ordering `lhs`. -/// Returns `None`, if sort_exprs are not compatible. +/// Returns `true` if the ordering `rhs` is strictly finer than the ordering `rhs`, +/// `false` if the ordering `lhs` is at least as fine as the ordering `lhs`, and +/// `None` otherwise (i.e. when given orderings are incomparable). fn finer_side(lhs: &[PhysicalSortExpr], rhs: &[PhysicalSortExpr]) -> Option { - let left_larger = lhs.len() >= rhs.len(); let all_equal = lhs.iter().zip(rhs.iter()).all(|(lhs, rhs)| lhs.eq(rhs)); - all_equal.then_some(left_larger) + all_equal.then_some(lhs.len() < rhs.len()) } /// `SchemaProperties` keeps track of useful information related to schema. @@ -1196,18 +1190,6 @@ fn prune_sort_reqs_with_constants( .collect() } -/// Adds the `offset` value to `Column` indices inside `expr`. This function is -/// generally used during the update of the right table schema in join operations. -fn add_offset_to_exprs( - exprs: Vec>, - offset: usize, -) -> Vec> { - exprs - .into_iter() - .map(|item| add_offset_to_expr(item, offset)) - .collect() -} - /// Adds the `offset` value to `Column` indices inside `expr`. This function is /// generally used during the update of the right table schema in join operations. pub fn add_offset_to_expr( @@ -1226,18 +1208,19 @@ pub fn add_offset_to_expr( // an `Ok` value. } -/// Adds the `offset` value to `Column` indices inside `sort_expr.expr`. -fn add_offset_to_sort_expr( - sort_expr: &PhysicalSortExpr, +/// Adds the `offset` value to `Column` indices inside `exprs`. +fn add_offset_to_exprs( + exprs: &[Arc], offset: usize, -) -> PhysicalSortExpr { - PhysicalSortExpr { - expr: add_offset_to_expr(sort_expr.expr.clone(), offset), - options: sort_expr.options, - } +) -> Vec> { + exprs + .iter() + .cloned() + .map(|item| add_offset_to_expr(item, offset)) + .collect() } -/// Adds the `offset` value to `Column` indices for each `sort_expr.expr` +/// Adds the `offset` value to `Column` indices for each sort expression /// inside `sort_exprs`. pub fn add_offset_to_lex_ordering( sort_exprs: LexOrderingRef, @@ -1245,7 +1228,10 @@ pub fn add_offset_to_lex_ordering( ) -> LexOrdering { sort_exprs .iter() - .map(|sort_expr| add_offset_to_sort_expr(sort_expr, offset)) + .map(|sort_expr| PhysicalSortExpr { + expr: add_offset_to_expr(sort_expr.expr.clone(), offset), + options: sort_expr.options, + }) .collect() } @@ -1275,7 +1261,7 @@ fn update_ordering( } else if node.expr.as_any().is::() { // We have a Column, which is one of the two possible leaf node types: let normalized_expr = eq_groups.normalize_expr(node.expr.clone()); - if let Some(options) = oeq_group.get_ordering(&normalized_expr) { + if let Some(options) = oeq_group.get_options(&normalized_expr) { node.state = Some(SortProperties::Ordered(options)); Ok(Transformed::Yes(node)) } else { @@ -1305,14 +1291,14 @@ fn update_ordering( /// it can thereafter safely be used for ordering equivalence normalization. fn get_updated_right_ordering_equivalence_class( join_type: &JoinType, - right_oeq_group: OrderingEquivalenceClass, + mut right_oeq_group: OrderingEquivalenceClass, left_columns_len: usize, ) -> OrderingEquivalenceClass { if matches!( join_type, JoinType::Inner | JoinType::Left | JoinType::Full | JoinType::Right ) { - return right_oeq_group.add_offset(left_columns_len); + right_oeq_group.add_offset(left_columns_len); } right_oeq_group } From da18b45ac87e6158156c8e51576e65ebc5ee0842 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Thu, 26 Oct 2023 01:05:19 +0300 Subject: [PATCH 099/122] Review Part 12 --- .../physical_optimizer/output_requirements.rs | 6 +- .../src/physical_optimizer/sort_pushdown.rs | 14 +- .../core/src/physical_optimizer/utils.rs | 4 +- datafusion/physical-expr/src/equivalence.rs | 199 ++++++++---------- datafusion/physical-expr/src/lib.rs | 2 +- datafusion/physical-expr/src/sort_expr.rs | 15 +- .../physical-plan/src/aggregates/mod.rs | 6 +- 7 files changed, 116 insertions(+), 130 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/output_requirements.rs b/datafusion/core/src/physical_optimizer/output_requirements.rs index f5eacd5ee60c..d9cdc292dd56 100644 --- a/datafusion/core/src/physical_optimizer/output_requirements.rs +++ b/datafusion/core/src/physical_optimizer/output_requirements.rs @@ -33,7 +33,7 @@ use datafusion_common::config::ConfigOptions; use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::{Result, Statistics}; use datafusion_physical_expr::{ - Distribution, LexOrderingReq, PhysicalSortExpr, PhysicalSortRequirement, + Distribution, LexRequirement, PhysicalSortExpr, PhysicalSortRequirement, }; use datafusion_physical_plan::sorts::sort_preserving_merge::SortPreservingMergeExec; @@ -90,14 +90,14 @@ enum RuleMode { #[derive(Debug)] struct OutputRequirementExec { input: Arc, - order_requirement: Option, + order_requirement: Option, dist_requirement: Distribution, } impl OutputRequirementExec { fn new( input: Arc, - requirements: Option, + requirements: Option, dist_requirement: Distribution, ) -> Self { Self { diff --git a/datafusion/core/src/physical_optimizer/sort_pushdown.rs b/datafusion/core/src/physical_optimizer/sort_pushdown.rs index 77ba54528e5d..7f359a09a22c 100644 --- a/datafusion/core/src/physical_optimizer/sort_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/sort_pushdown.rs @@ -32,7 +32,9 @@ use datafusion_common::tree_node::{Transformed, TreeNode, VisitRecursion}; use datafusion_common::{plan_err, DataFusionError, JoinSide, Result}; use datafusion_expr::JoinType; use datafusion_physical_expr::expressions::Column; -use datafusion_physical_expr::{PhysicalSortExpr, PhysicalSortRequirement}; +use datafusion_physical_expr::{ + LexRequirementRef, PhysicalSortExpr, PhysicalSortRequirement, +}; use itertools::izip; @@ -184,7 +186,7 @@ pub(crate) fn pushdown_sorts( fn pushdown_requirement_to_children( plan: &Arc, - parent_required: &[PhysicalSortRequirement], + parent_required: LexRequirementRef, ) -> Result>>>> { let maintains_input_order = plan.maintains_input_order(); if is_window(plan) { @@ -300,8 +302,8 @@ fn pushdown_requirement_to_children( /// 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: &[PhysicalSortRequirement], - request_child: &[PhysicalSortRequirement], + parent_required: LexRequirementRef, + request_child: LexRequirementRef, child_plan: Arc, ) -> RequirementsCompatibility { if child_plan @@ -327,7 +329,7 @@ fn determine_children_requirement( } fn try_pushdown_requirements_to_join( smj: &SortMergeJoinExec, - parent_required: &[PhysicalSortRequirement], + parent_required: LexRequirementRef, sort_expr: Vec, push_side: JoinSide, ) -> Result>>>> { @@ -416,7 +418,7 @@ fn expr_source_sides( } fn shift_right_required( - parent_required: &[PhysicalSortRequirement], + parent_required: LexRequirementRef, left_columns_len: usize, ) -> Result> { let new_right_required: Vec = parent_required diff --git a/datafusion/core/src/physical_optimizer/utils.rs b/datafusion/core/src/physical_optimizer/utils.rs index bff1ca8fc321..cd1fc6247538 100644 --- a/datafusion/core/src/physical_optimizer/utils.rs +++ b/datafusion/core/src/physical_optimizer/utils.rs @@ -30,7 +30,7 @@ use crate::physical_plan::union::UnionExec; use crate::physical_plan::windows::{BoundedWindowAggExec, WindowAggExec}; use crate::physical_plan::{displayable, ExecutionPlan}; -use datafusion_physical_expr::PhysicalSortRequirement; +use datafusion_physical_expr::{LexRequirementRef, PhysicalSortRequirement}; /// This object implements a tree that we use while keeping track of paths /// leading to [`SortExec`]s. @@ -99,7 +99,7 @@ pub(crate) fn get_children_exectrees( /// given ordering requirements while preserving the original partitioning. pub fn add_sort_above( node: &mut Arc, - sort_requirement: &[PhysicalSortRequirement], + sort_requirement: LexRequirementRef, fetch: Option, ) { // If the ordering requirement is already satisfied, do not add a sort. diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 7ff0ee3baa9a..b64abb38c441 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -22,8 +22,8 @@ use crate::expressions::Column; use crate::physical_expr::{deduplicate_physical_exprs, have_common_entries}; use crate::sort_properties::{ExprOrdering, SortProperties}; use crate::{ - physical_exprs_contains, LexOrdering, LexOrderingRef, LexOrderingReq, PhysicalExpr, - PhysicalSortExpr, PhysicalSortRequirement, + physical_exprs_contains, LexOrdering, LexOrderingRef, LexRequirement, + LexRequirementRef, PhysicalExpr, PhysicalSortExpr, PhysicalSortRequirement, }; use arrow::datatypes::SchemaRef; @@ -53,7 +53,7 @@ pub struct EquivalenceGroup { impl EquivalenceGroup { /// Creates an empty equivalence group. fn empty() -> Self { - EquivalenceGroup { classes: vec![] } + Self { classes: vec![] } } /// Creates an equivalence group from the given equivalence classes. @@ -74,7 +74,7 @@ impl EquivalenceGroup { } /// Returns an iterator over the equivalence classes in this group. - fn iter(&self) -> impl Iterator>> { + fn iter(&self) -> impl Iterator { self.classes.iter() } @@ -164,7 +164,7 @@ impl EquivalenceGroup { } /// Extends this equivalence group with the `other` equivalence group. - fn extend(&mut self, other: EquivalenceGroup) { + fn extend(&mut self, other: Self) { self.classes.extend(other.classes); self.remove_redundant_entries(); } @@ -185,16 +185,17 @@ impl EquivalenceGroup { .unwrap_or(expr) } - /// This function applies the `normalize_expr` function for all expressions - /// in `exprs` and returns the corresponding normalized physical expressions. - pub fn normalize_exprs( + /// Normalizes the given sort expression according to this group. + /// The underlying physical expression is replaced with the first expression + /// in the equivalence class it matches with (if any). If the underlying + /// expression does not belong to any equivalence class in this group, returns + /// the sort expression as is. + pub fn normalize_sort_expr( &self, - exprs: Vec>, - ) -> Vec> { - exprs - .into_iter() - .map(|expr| self.normalize_expr(expr)) - .collect() + mut sort_expr: PhysicalSortExpr, + ) -> PhysicalSortExpr { + sort_expr.expr = self.normalize_expr(sort_expr.expr); + sort_expr } /// Normalizes the given sort requirement according to this group. @@ -210,17 +211,28 @@ impl EquivalenceGroup { sort_requirement } - /// Normalizes the given sort expression according to this group. - /// The underlying physical expression is replaced with the first expression - /// in the equivalence class it matches with (if any). If the underlying - /// expression does not belong to any equivalence class in this group, returns - /// the sort expression as is. - pub fn normalize_sort_expr( + /// This function applies the `normalize_expr` function for all expressions + /// in `exprs` and returns the corresponding normalized physical expressions. + pub fn normalize_exprs( &self, - mut sort_expr: PhysicalSortExpr, - ) -> PhysicalSortExpr { - sort_expr.expr = self.normalize_expr(sort_expr.expr); - sort_expr + exprs: Vec>, + ) -> Vec> { + exprs + .into_iter() + .map(|expr| self.normalize_expr(expr)) + .collect() + } + + /// This function applies the `normalize_sort_expr` function for all sort + /// expressions in `sort_exprs` and returns the corresponding normalized + /// sort expressions. + pub fn normalize_sort_exprs(&self, sort_exprs: LexOrderingRef) -> LexOrdering { + // Convert sort expressions to sort requirements: + let sort_reqs = PhysicalSortRequirement::from_sort_exprs(sort_exprs.iter()); + // Normalize the requirements: + let normalized_sort_reqs = self.normalize_sort_requirements(&sort_reqs); + // Convert sort requirements back to sort expressions: + PhysicalSortRequirement::to_sort_exprs(normalized_sort_reqs) } /// This function applies the `normalize_sort_requirement` function for all @@ -228,8 +240,8 @@ impl EquivalenceGroup { /// sort requirements. pub fn normalize_sort_requirements( &self, - sort_reqs: &[PhysicalSortRequirement], - ) -> Vec { + sort_reqs: LexRequirementRef, + ) -> LexRequirement { collapse_lex_req( sort_reqs .iter() @@ -238,22 +250,6 @@ impl EquivalenceGroup { ) } - /// This function applies the `normalize_sort_expr` function for all sort - /// expressions in `sort_exprs` and returns the corresponding normalized - /// sort expressions. - pub fn normalize_sort_exprs( - &self, - sort_exprs: &[PhysicalSortExpr], - ) -> Vec { - // Convert `PhysicalSortExpr`s to `PhysicalSortRequirement`s: - let sort_requirements = - PhysicalSortRequirement::from_sort_exprs(sort_exprs.iter()); - let normalized_sort_requirement = - self.normalize_sort_requirements(&sort_requirements); - // Convert back `PhysicalSortRequirement`s to `PhysicalSortExpr`s: - PhysicalSortRequirement::to_sort_exprs(normalized_sort_requirement) - } - /// Projects `expr` according to the given projection mapping. /// If the resulting expression is invalid after projection, returns `None`. fn project_expr( @@ -294,8 +290,8 @@ impl EquivalenceGroup { fn project_ordering( &self, mapping: &ProjectionMapping, - ordering: &[PhysicalSortExpr], - ) -> Option> { + ordering: LexOrderingRef, + ) -> Option { // If any sort expression is invalid after projection, rest of the // ordering shouldn't be projected either. For example, if input ordering // is [a ASC, b ASC, c ASC], and column b is not valid after projection, @@ -315,7 +311,7 @@ impl EquivalenceGroup { } /// Projects this equivalence group according to the given projection mapping. - pub fn project(&self, mapping: &ProjectionMapping) -> EquivalenceGroup { + pub fn project(&self, mapping: &ProjectionMapping) -> Self { let projected_classes = self.iter().filter_map(|cls| { let new_class = cls .iter() @@ -344,7 +340,7 @@ impl EquivalenceGroup { .into_iter() .filter_map(|(_, values)| (values.len() > 1).then_some(values)); let classes = projected_classes.chain(new_classes).collect(); - EquivalenceGroup::new(classes) + Self::new(classes) } /// Returns the equivalence class that contains `expr`. @@ -362,11 +358,11 @@ impl EquivalenceGroup { pub fn join( &self, join_type: &JoinType, - right_equivalences: &EquivalenceGroup, + right_equivalences: &Self, left_column_count: usize, on: &[(Column, Column)], - ) -> EquivalenceGroup { - let mut result = EquivalenceGroup::empty(); + ) -> Self { + let mut result = Self::empty(); match join_type { JoinType::Inner | JoinType::Left | JoinType::Full | JoinType::Right => { result.extend(self.clone()); @@ -374,7 +370,7 @@ impl EquivalenceGroup { .iter() .map(|cls| add_offset_to_exprs(cls, left_column_count)) .collect(); - result.extend(EquivalenceGroup::new(updated_eq_classes)); + result.extend(Self::new(updated_eq_classes)); } JoinType::LeftSemi | JoinType::LeftAnti => { result.extend(self.clone()); @@ -418,12 +414,12 @@ pub struct OrderingEquivalenceClass { impl OrderingEquivalenceClass { /// Creates new empty ordering equivalent class. fn empty() -> Self { - OrderingEquivalenceClass { orderings: vec![] } + Self { orderings: vec![] } } /// Creates new ordering equivalence class from the given orderings. pub fn new(orderings: Vec) -> Self { - let mut result = OrderingEquivalenceClass { orderings }; + let mut result = Self { orderings }; result.remove_redundant_entries(); result } @@ -461,7 +457,7 @@ impl OrderingEquivalenceClass { } /// Extend this ordering equivalence class with the `other` class. - pub fn extend(&mut self, other: OrderingEquivalenceClass) { + pub fn extend(&mut self, other: Self) { self.orderings.extend(other.orderings); // Make sure that there are no redundant orderings: self.remove_redundant_entries(); @@ -474,7 +470,7 @@ impl OrderingEquivalenceClass { self.remove_redundant_entries(); } - /// Removes redundant orderings from the state. + /// Removes redundant orderings from this equivalence class. /// For instance, If we already have the ordering [a ASC, b ASC, c DESC], /// then there is no need to keep ordering [a ASC, b ASC] in the state. fn remove_redundant_entries(&mut self) { @@ -500,27 +496,19 @@ impl OrderingEquivalenceClass { /// Gets the first ordering entry in this ordering equivalence class. /// This is one of the many valid orderings (if there are multiple). - pub fn output_ordering(&self) -> Option> { + pub fn output_ordering(&self) -> Option { self.orderings.first().cloned() } - // Append other as postfix to existing ordering equivalences - pub fn join_postfix( - &self, - other: &OrderingEquivalenceClass, - ) -> OrderingEquivalenceClass { - if other.is_empty() { - return OrderingEquivalenceClass::new(self.orderings.clone()); - } - let mut res = vec![]; - for ordering in self.iter() { - for postfix in other.iter() { - let mut new_ordering = ordering.clone(); - new_ordering.extend(postfix.clone()); - res.push(new_ordering) + // Append orderings in `other` to all existing orderings in this equivalence + // class. + pub fn join_postfix(mut self, other: &Self) -> Self { + for postfix in other.iter() { + for idx in 0..self.orderings.len() { + self.orderings[idx].extend(postfix.iter().cloned()); } } - OrderingEquivalenceClass::new(res) + self } /// Adds `offset` value to the index of each expression inside this @@ -549,7 +537,7 @@ impl OrderingEquivalenceClass { /// Returns `true` if the ordering `rhs` is strictly finer than the ordering `rhs`, /// `false` if the ordering `lhs` is at least as fine as the ordering `lhs`, and /// `None` otherwise (i.e. when given orderings are incomparable). -fn finer_side(lhs: &[PhysicalSortExpr], rhs: &[PhysicalSortExpr]) -> Option { +fn finer_side(lhs: LexOrderingRef, rhs: LexOrderingRef) -> Option { let all_equal = lhs.iter().zip(rhs.iter()).all(|(lhs, rhs)| lhs.eq(rhs)); all_equal.then_some(lhs.len() < rhs.len()) } @@ -640,7 +628,7 @@ impl SchemaProperties { } /// Add SchemaProperties of the other to the state. - pub fn extend(mut self, other: SchemaProperties) -> Self { + pub fn extend(mut self, other: Self) -> Self { self.eq_group.extend(other.eq_group); self.oeq_class.extend(other.oeq_class); self.with_constants(other.constants) @@ -697,7 +685,7 @@ impl SchemaProperties { /// Re-creates `SchemaProperties` given that /// schema is re-ordered by `sort_expr` in the argument. - pub fn with_reorder(mut self, sort_expr: Vec) -> SchemaProperties { + pub fn with_reorder(mut self, sort_expr: Vec) -> Self { // TODO: In some cases, existing ordering equivalences may still be valid add this analysis. // Reset ordering equivalent group with the new ordering. @@ -715,10 +703,7 @@ impl SchemaProperties { /// This function converts `sort_exprs` `vec![b ASC, c ASC]` to first `vec![a ASC, c ASC]` after considering `eq_groups` /// Then converts `vec![a ASC, c ASC]` to `vec![d ASC]` after considering `oeq_group`. /// Standardized version `vec![d ASC]` is used in subsequent operations. - fn normalize_sort_exprs( - &self, - sort_exprs: &[PhysicalSortExpr], - ) -> Vec { + fn normalize_sort_exprs(&self, sort_exprs: LexOrderingRef) -> LexOrdering { // Convert `PhysicalSortExpr`s to `PhysicalSortRequirement`s let sort_requirements = PhysicalSortRequirement::from_sort_exprs(sort_exprs.iter()); @@ -743,8 +728,8 @@ impl SchemaProperties { /// Standardized version `vec![d Some(ASC)]` is used in subsequent operations. fn normalize_sort_requirements( &self, - sort_reqs: &[PhysicalSortRequirement], - ) -> Vec { + sort_reqs: LexRequirementRef, + ) -> LexRequirement { let normalized_sort_reqs = self.eq_group.normalize_sort_requirements(sort_reqs); let constants_normalized = self.eq_group.normalize_exprs(self.constants.clone()); let normalized_sort_reqs = @@ -753,9 +738,9 @@ impl SchemaProperties { collapse_lex_req(normalized_sort_reqs) } - /// Checks whether the required [`PhysicalSortExpr`]s are satisfied by the - /// any of the existing orderings. - pub fn ordering_satisfy(&self, required: &[PhysicalSortExpr]) -> bool { + /// Checks whether the given ordering is satisfied by any of the existing + /// orderings. + pub fn ordering_satisfy(&self, required: LexOrderingRef) -> bool { // Convert `PhysicalSortExpr`s to `PhysicalSortRequirement`s let sort_requirements = PhysicalSortRequirement::from_sort_exprs(required.iter()); self.ordering_satisfy_requirement(&sort_requirements) @@ -763,10 +748,7 @@ impl SchemaProperties { /// Checks whether the given [`PhysicalSortRequirement`]s are satisfied by the /// provided [`PhysicalSortExpr`]s. - pub fn ordering_satisfy_requirement( - &self, - required: &[PhysicalSortRequirement], - ) -> bool { + pub fn ordering_satisfy_requirement(&self, required: LexRequirementRef) -> bool { self.prune_lex_req(required).is_empty() } @@ -774,8 +756,8 @@ impl SchemaProperties { /// specific than the provided [`PhysicalSortRequirement`]s. pub fn requirements_compatible( &self, - provided: &[PhysicalSortRequirement], - required: &[PhysicalSortRequirement], + provided: LexRequirementRef, + required: LexRequirementRef, ) -> bool { let provided_normalized = self.normalize_sort_requirements(provided); let required_normalized = self.normalize_sort_requirements(required); @@ -796,9 +778,9 @@ impl SchemaProperties { /// As an example finer ordering of [a ASC] and [a ASC, b ASC] is [a ASC, b ASC] pub fn get_finer_ordering( &self, - req1: &[PhysicalSortExpr], - req2: &[PhysicalSortExpr], - ) -> Option> { + req1: LexOrderingRef, + req2: LexOrderingRef, + ) -> Option { // Convert `PhysicalSortExpr` s to `PhysicalSortRequirement`s. let req1 = PhysicalSortRequirement::from_sort_exprs(req1); let req2 = PhysicalSortRequirement::from_sort_exprs(req2); @@ -812,9 +794,9 @@ impl SchemaProperties { /// e.g there is no requirement that satisfies both pub fn get_finer_requirement( &self, - req1: &[PhysicalSortRequirement], - req2: &[PhysicalSortRequirement], - ) -> Option> { + req1: LexRequirementRef, + req2: LexRequirementRef, + ) -> Option { let mut lhs = self.normalize_sort_requirements(req1); let mut rhs = self.normalize_sort_requirements(req2); if izip!(lhs.iter_mut(), rhs.iter_mut()).all(|(lhs, rhs)| { @@ -851,9 +833,9 @@ impl SchemaProperties { /// As an example meet ordering of [a ASC] and [a ASC, b ASC] is [a ASC] pub fn get_meet_ordering( &self, - req1: &[PhysicalSortExpr], - req2: &[PhysicalSortExpr], - ) -> Option> { + req1: LexOrderingRef, + req2: LexOrderingRef, + ) -> Option { let lhs = self.normalize_sort_exprs(req1); let rhs = self.normalize_sort_exprs(req2); let mut meet = vec![]; @@ -873,7 +855,7 @@ impl SchemaProperties { /// Empty result means that requirement is already satisfied. /// Non-empty result means that requirement is not satisfied. /// This util shouldn't be used outside this context. - fn prune_lex_req(&self, sort_req: &[PhysicalSortRequirement]) -> LexOrderingReq { + fn prune_lex_req(&self, sort_req: LexRequirementRef) -> LexRequirement { // Make sure to use a standardized version of the requirement let mut normalized_sort_req = self.normalize_sort_requirements(sort_req); @@ -900,12 +882,10 @@ impl SchemaProperties { if let (Some(current), Some(next)) = (window[0], window[1]) { if next > current { to_remove.push(next); - } else { - break; + continue; } - } else { - break; } + break; } } // can remove entries at the match_prefix indices @@ -1014,13 +994,10 @@ impl SchemaProperties { pub fn get_lex_ordering( &self, exprs: &[Arc], - ) -> Option> { + ) -> Option { let ordered_section = self.get_lex_ordering_section(exprs); // Make sure ordered section covers all expressions. - if ordered_section.len() == exprs.len() { - return Some(ordered_section); - } - None + (ordered_section.len() == exprs.len()).then_some(ordered_section) } /// Get ordering of the expressions in the argument @@ -1029,7 +1006,7 @@ impl SchemaProperties { pub fn get_lex_ordering_section( &self, exprs: &[Arc], - ) -> Vec { + ) -> LexOrdering { let normalized_exprs = self.eq_group.normalize_exprs(exprs.to_vec()); // After normalization size of the vector shouldn't change. assert_eq!(normalized_exprs.len(), exprs.len()); @@ -1165,7 +1142,7 @@ pub fn schema_properties_helper( /// This function constructs a duplicate-free `LexOrderingReq` by filtering out /// duplicate entries that have same physical expression inside. For example, /// `vec![a Some(Asc), a Some(Desc)]` collapses to `vec![a Some(Asc)]`. -pub fn collapse_lex_req(input: LexOrderingReq) -> LexOrderingReq { +pub fn collapse_lex_req(input: LexRequirement) -> LexRequirement { let mut output = vec![]; for item in input { if !output @@ -1180,9 +1157,9 @@ pub fn collapse_lex_req(input: LexOrderingReq) -> LexOrderingReq { /// Remove ordering requirements that have constant value fn prune_sort_reqs_with_constants( - ordering: &[PhysicalSortRequirement], + ordering: LexRequirementRef, constants: &[Arc], -) -> Vec { +) -> LexRequirement { ordering .iter() .filter(|&order| !physical_exprs_contains(constants, &order.expr)) diff --git a/datafusion/physical-expr/src/lib.rs b/datafusion/physical-expr/src/lib.rs index 2b6123910912..f24188504717 100644 --- a/datafusion/physical-expr/src/lib.rs +++ b/datafusion/physical-expr/src/lib.rs @@ -65,7 +65,7 @@ pub use physical_expr::{ pub use planner::create_physical_expr; pub use scalar_function::ScalarFunctionExpr; pub use sort_expr::{ - LexOrdering, LexOrderingRef, LexOrderingReq, PhysicalSortExpr, + LexOrdering, LexOrderingRef, LexRequirement, LexRequirementRef, PhysicalSortExpr, PhysicalSortRequirement, }; pub use utils::{reverse_order_bys, split_conjunction}; diff --git a/datafusion/physical-expr/src/sort_expr.rs b/datafusion/physical-expr/src/sort_expr.rs index 5bf7d70c186a..2b659b23a0a9 100644 --- a/datafusion/physical-expr/src/sort_expr.rs +++ b/datafusion/physical-expr/src/sort_expr.rs @@ -272,11 +272,18 @@ fn to_str(options: &SortOptions) -> &str { } } -///`LexOrdering` is a type alias for lexicographical ordering definition`Vec` +///`LexOrdering` is an alias for the type `Vec`, which represents +/// a lexicographical ordering. pub type LexOrdering = Vec; -///`LexOrderingRef` is a type alias for lexicographical ordering reference &`[PhysicalSortExpr]` +///`LexOrderingRef` is an alias for the type &`[PhysicalSortExpr]`, which represents +/// a reference to a lexicographical ordering. pub type LexOrderingRef<'a> = &'a [PhysicalSortExpr]; -///`LexOrderingReq` is a type alias for lexicographical ordering requirement definition`Vec` -pub type LexOrderingReq = Vec; +///`LexRequirement` is an alias for the type `Vec`, which +/// represents a lexicographical ordering requirement. +pub type LexRequirement = Vec; + +///`LexRequirementRef` is an alias for the type &`[PhysicalSortRequirement]`, which +/// represents a reference to a lexicographical ordering requirement. +pub type LexRequirementRef<'a> = &'a [PhysicalSortRequirement]; diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 79a3c4169983..fe2b2cc93b52 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -47,7 +47,7 @@ use datafusion_physical_expr::{ equivalence::collapse_lex_req, expressions::{Column, Max, Min, UnKnownColumn}, physical_exprs_contains, reverse_order_bys, AggregateExpr, LexOrdering, - LexOrderingReq, PhysicalExpr, PhysicalSortExpr, PhysicalSortRequirement, + LexRequirement, PhysicalExpr, PhysicalSortExpr, PhysicalSortRequirement, SchemaProperties, }; @@ -300,7 +300,7 @@ pub struct AggregateExec { source_to_target_mapping: Vec<(Arc, Arc)>, /// Execution metrics metrics: ExecutionPlanMetricsSet, - required_input_ordering: Option, + required_input_ordering: Option, partition_search_mode: PartitionSearchMode, output_ordering: Option, } @@ -803,7 +803,7 @@ impl ExecutionPlan for AggregateExec { } } - fn required_input_ordering(&self) -> Vec> { + fn required_input_ordering(&self) -> Vec> { vec![self.required_input_ordering.clone()] } From 155736a98b16ca13babbaec777d3431869cd3aa1 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 26 Oct 2023 16:08:00 +0300 Subject: [PATCH 100/122] Update comments --- datafusion/physical-expr/src/equivalence.rs | 263 +++++++++++--------- 1 file changed, 148 insertions(+), 115 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index b64abb38c441..6f51a9882e63 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -1252,20 +1252,12 @@ fn update_ordering( } } -/// Update right table ordering equivalences so that: -/// - They point to valid indices at the output of the join schema, and -/// - They are normalized with respect to equivalence columns. +/// Update right table ordering `OrderingEquivalenceClass`es so that: +/// - They point to valid indices at the output of the join schema /// /// To do so, we increment column indices by the size of the left table when /// join schema consists of a combination of left and right schema (Inner, -/// Left, Full, Right joins). Then, we normalize the sort expressions of -/// ordering equivalences one by one. We make sure that each expression in the -/// ordering equivalence is either: -/// - The head of the one of the equivalent classes, or -/// - Doesn't have an equivalent column. -/// -/// This way; once we normalize an expression according to equivalence properties, -/// it can thereafter safely be used for ordering equivalence normalization. +/// Left, Full, Right joins). For other cases indices don't change. fn get_updated_right_ordering_equivalence_class( join_type: &JoinType, mut right_oeq_group: OrderingEquivalenceClass, @@ -1376,26 +1368,25 @@ mod tests { /// Column e is constant. fn create_random_schema(seed: u64) -> Result<(SchemaRef, SchemaProperties)> { let test_schema = create_test_schema_2()?; - - let col_exprs = vec![ - col("a", &test_schema)?, - col("b", &test_schema)?, - col("c", &test_schema)?, - col("d", &test_schema)?, - col("e", &test_schema)?, - col("f", &test_schema)?, - ]; + let col_a = &col("a", &test_schema)?; + let col_b = &col("b", &test_schema)?; + let col_c = &col("c", &test_schema)?; + let col_d = &col("d", &test_schema)?; + let col_e = &col("e", &test_schema)?; + let col_f = &col("f", &test_schema)?; + let col_exprs = [col_a, col_b, col_c, col_d, col_e, col_f]; let mut schema_properties = SchemaProperties::new(test_schema.clone()); - // Define equivalent columns and constant columns - schema_properties.add_equal_conditions(&col_exprs[0], &col_exprs[5]); - schema_properties = schema_properties.with_constants(vec![col_exprs[4].clone()]); + // Define a and f are aliases + schema_properties.add_equal_conditions(col_a, col_f); + // Column e has constant value. + schema_properties = schema_properties.with_constants(vec![col_e.clone()]); // Randomly order columns for sorting let mut rng = StdRng::seed_from_u64(seed); let mut remaining_exprs = col_exprs[0..4].to_vec(); // only a, b, c, d are sorted - let sort_options = SortOptions { + let options_asc = SortOptions { descending: false, nulls_first: false, }; @@ -1407,8 +1398,8 @@ mod tests { let orderings: Vec<_> = remaining_exprs .drain(0..n_sort_expr) .map(|expr| PhysicalSortExpr { - expr, - options: sort_options, + expr: expr.clone(), + options: options_asc, }) .collect(); @@ -1470,9 +1461,11 @@ mod tests { let col_x_expr = Arc::new(Column::new("x", 3)) as Arc; let col_y_expr = Arc::new(Column::new("y", 4)) as Arc; + // a and b are aliases schema_properties.add_equal_conditions(&col_a_expr, &col_b_expr); assert_eq!(schema_properties.eq_group().len(), 1); + // This new entry is redundant, size shouldn't increase schema_properties.add_equal_conditions(&col_b_expr, &col_a_expr); assert_eq!(schema_properties.eq_group().len(), 1); let eq_groups = &schema_properties.eq_group().classes[0]; @@ -1480,6 +1473,8 @@ mod tests { assert!(physical_exprs_contains(eq_groups, &col_a_expr)); assert!(physical_exprs_contains(eq_groups, &col_b_expr)); + // b and c are aliases. Exising equivalence class should expand, + // however there shouldn't be any new equivalence class schema_properties.add_equal_conditions(&col_b_expr, &col_c_expr); assert_eq!(schema_properties.eq_group().len(), 1); let eq_groups = &schema_properties.eq_group().classes[0]; @@ -1515,13 +1510,8 @@ mod tests { Field::new("c", DataType::Int64, true), ])); - let mut input_properties = SchemaProperties::new(input_schema.clone()); + let input_properties = SchemaProperties::new(input_schema.clone()); let col_a = col("a", &input_schema)?; - let col_b = col("b", &input_schema)?; - let col_c = col("c", &input_schema)?; - - input_properties.add_equal_conditions(&col_a, &col_b); - input_properties.add_equal_conditions(&col_b, &col_c); let out_schema = Arc::new(Schema::new(vec![ Field::new("a1", DataType::Int64, true), @@ -1530,6 +1520,7 @@ mod tests { Field::new("a4", DataType::Int64, true), ])); + // a as a1, a as a2, a as a3, a as a3 let col_a1 = &col("a1", &out_schema)?; let col_a2 = &col("a2", &out_schema)?; let col_a3 = &col("a3", &out_schema)?; @@ -1543,6 +1534,7 @@ mod tests { let out_properties = input_properties.project(&source_to_target_mapping, out_schema); + // At the output a1=a2=a3=a4 assert_eq!(out_properties.eq_group().len(), 1); let eq_class = &out_properties.eq_group().classes[0]; assert_eq!(eq_class.len(), 4); @@ -1570,7 +1562,7 @@ mod tests { options: SortOptions::default(), }, ]; - // finer ordering satisfies, crude ordering shoul return true + // finer ordering satisfies, crude ordering should return true let empty_schema = &Arc::new(Schema::empty()); let mut schema_properties = SchemaProperties::new(empty_schema.clone()); schema_properties.oeq_class.push(finer.clone()); @@ -1597,11 +1589,11 @@ mod tests { let col_e = &col("e", &test_schema)?; let col_f = &col("f", &test_schema)?; let col_g = &col("g", &test_schema)?; - let option1 = SortOptions { + let option_asc = SortOptions { descending: false, nulls_first: false, }; - let option2 = SortOptions { + let option_desc = SortOptions { descending: true, nulls_first: true, }; @@ -1611,106 +1603,114 @@ mod tests { // First element in the tuple stores vector of requirement, second element is the expected return value for ordering_satisfy function let requirements = vec![ // `a ASC NULLS LAST`, expects `ordering_satisfy` to be `true`, since existing ordering `a ASC NULLS LAST, b ASC NULLS LAST` satisfies it - (vec![(col_a, option1)], true), - (vec![(col_a, option2)], false), + (vec![(col_a, option_asc)], true), + (vec![(col_a, option_desc)], false), // Test whether equivalence works as expected - (vec![(col_c, option1)], true), - (vec![(col_c, option2)], false), + (vec![(col_c, option_asc)], true), + (vec![(col_c, option_desc)], false), // Test whether ordering equivalence works as expected - (vec![(col_d, option1)], true), - (vec![(col_d, option1), (col_b, option1)], true), - (vec![(col_d, option2), (col_b, option1)], false), + (vec![(col_d, option_asc)], true), + (vec![(col_d, option_asc), (col_b, option_asc)], true), + (vec![(col_d, option_desc), (col_b, option_asc)], false), ( - vec![(col_e, option2), (col_f, option1), (col_g, option1)], + vec![ + (col_e, option_desc), + (col_f, option_asc), + (col_g, option_asc), + ], true, ), - (vec![(col_e, option2), (col_f, option1)], true), - (vec![(col_e, option1), (col_f, option1)], false), - (vec![(col_e, option2), (col_b, option1)], false), - (vec![(col_e, option1), (col_b, option1)], false), + (vec![(col_e, option_desc), (col_f, option_asc)], true), + (vec![(col_e, option_asc), (col_f, option_asc)], false), + (vec![(col_e, option_desc), (col_b, option_asc)], false), + (vec![(col_e, option_asc), (col_b, option_asc)], false), ( vec![ - (col_d, option1), - (col_b, option1), - (col_d, option1), - (col_b, option1), + (col_d, option_asc), + (col_b, option_asc), + (col_d, option_asc), + (col_b, option_asc), ], true, ), ( vec![ - (col_d, option1), - (col_b, option1), - (col_e, option2), - (col_f, option1), + (col_d, option_asc), + (col_b, option_asc), + (col_e, option_desc), + (col_f, option_asc), ], true, ), ( vec![ - (col_d, option1), - (col_b, option1), - (col_e, option2), - (col_b, option1), + (col_d, option_asc), + (col_b, option_asc), + (col_e, option_desc), + (col_b, option_asc), ], true, ), ( vec![ - (col_d, option1), - (col_b, option1), - (col_d, option2), - (col_b, option1), + (col_d, option_asc), + (col_b, option_asc), + (col_d, option_desc), + (col_b, option_asc), ], true, ), ( vec![ - (col_d, option1), - (col_b, option1), - (col_e, option1), - (col_f, option1), + (col_d, option_asc), + (col_b, option_asc), + (col_e, option_asc), + (col_f, option_asc), ], false, ), ( vec![ - (col_d, option1), - (col_b, option1), - (col_e, option1), - (col_b, option1), + (col_d, option_asc), + (col_b, option_asc), + (col_e, option_asc), + (col_b, option_asc), ], false, ), - (vec![(col_d, option1), (col_e, option2)], true), + (vec![(col_d, option_asc), (col_e, option_desc)], true), ( - vec![(col_d, option1), (col_c, option1), (col_b, option1)], + vec![ + (col_d, option_asc), + (col_c, option_asc), + (col_b, option_asc), + ], true, ), ( vec![ - (col_d, option1), - (col_e, option2), - (col_f, option1), - (col_b, option1), + (col_d, option_asc), + (col_e, option_desc), + (col_f, option_asc), + (col_b, option_asc), ], true, ), ( vec![ - (col_d, option1), - (col_e, option2), - (col_c, option1), - (col_b, option1), + (col_d, option_asc), + (col_e, option_desc), + (col_c, option_asc), + (col_b, option_asc), ], true, ), ( vec![ - (col_d, option1), - (col_e, option2), - (col_b, option1), - (col_f, option1), + (col_d, option_asc), + (col_e, option_desc), + (col_b, option_asc), + (col_f, option_asc), ], true, ), @@ -1726,6 +1726,7 @@ mod tests { }) .collect::>(); + // Check expected result with experimental result. assert_eq!( is_table_same_after_sort( required.clone(), @@ -1753,7 +1754,9 @@ mod tests { }; for seed in 0..N_RANDOM_SCHEMA { + // Create a random schema with random properties let (test_schema, schema_properties) = create_random_schema(seed as u64)?; + // Generate a data that satisfies properties given let table_data_with_properties = generate_table_for_schema_properties( &schema_properties, N_ELEMENTS, @@ -1785,6 +1788,8 @@ mod tests { "Error in test case requirement:{:?}, expected: {:?}", requirement, expected ); + // Check whether ordering_satisfy API result and + // experimental result matches. assert_eq!( schema_properties.ordering_satisfy(&requirement), expected, @@ -1811,7 +1816,7 @@ mod tests { descending: false, nulls_first: false, }; - // Column a and c are aliases. + // a=c (e.g they are aliases). let mut schema_properties = SchemaProperties::new(test_schema); schema_properties.add_equal_conditions(col_a, col_c); @@ -1825,9 +1830,9 @@ mod tests { // Column [a ASC], [e ASC], [d ASC, f ASC] are all valid orderings for the schema. schema_properties.add_new_orderings(&orderings); - // All of the orderings [a ASC], [e ASC], [d ASC, f ASC]] - // are valid for the table - // Also Columns a and c are equal + // First entry in the tuple is required ordering, second entry is the expected flag + // that indicates whether this required ordering is satisfied. + // ([a ASC], true) indicate a ASC requirement is already satisfied by existing orderings. let test_cases = vec![ // [c ASC, a ASC, e ASC], expected represents this requirement is satisfied ( @@ -1911,9 +1916,8 @@ mod tests { vec![lit(3), lit(3)], vec![lit(4), lit(5), lit(6)], ]; - // Expected is a bit weird. However, what we care is they expected contains distinct groups. - // where there is no common entry between any groups. - // Since we do check for vector equality, this version should be used during comparison in the test. + // Given equivalences classes are not in succinct form. + // Expected form is the most plain representation that is functionally same. let expected = vec![vec![lit(1), lit(2)], vec![lit(4), lit(5), lit(6)]]; let mut eq_groups = EquivalenceGroup::new(entries); eq_groups.remove_redundant_entries(); @@ -1943,6 +1947,8 @@ mod tests { nulls_first: true, }; + // First entry in the tuple is the given orderings for the table + // Second entry is the simplest version of the given orderings that is functionally equivalent. let test_cases = vec![ // ------- TEST CASE 1 --------- ( @@ -1951,7 +1957,7 @@ mod tests { // [a ASC, b ASC] vec![(col_a, option_asc), (col_b, option_asc)], ], - // EXPECTED orderings that is succint. + // EXPECTED orderings that is succinct. vec![ // [a ASC, b ASC] vec![(col_a, option_asc), (col_b, option_asc)], @@ -1970,7 +1976,7 @@ mod tests { (col_c, option_asc), ], ], - // EXPECTED orderings that is succint. + // EXPECTED orderings that is succinct. vec![ // [a ASC, b ASC, c ASC] vec![ @@ -1991,7 +1997,7 @@ mod tests { // [a ASC, c ASC] vec![(col_a, option_asc), (col_c, option_asc)], ], - // EXPECTED orderings that is succint. + // EXPECTED orderings that is succinct. vec![ // [a ASC, b DESC] vec![(col_a, option_asc), (col_b, option_desc)], @@ -2014,7 +2020,7 @@ mod tests { // [a ASC] vec![(col_a, option_asc)], ], - // EXPECTED orderings that is succint. + // EXPECTED orderings that is succinct. vec![ // [a ASC, b ASC, c ASC] vec![ @@ -2056,10 +2062,14 @@ mod tests { let col_y = &col("y", &child_schema)?; let col_z = &col("z", &child_schema)?; let col_w = &col("w", &child_schema)?; - let options = SortOptions::default(); + let option_asc = SortOptions { + descending: false, + nulls_first: false, + }; + // [x ASC, y ASC], [z ASC, w ASC] let orderings = vec![ - vec![(col_x, options), (col_y, options)], - vec![(col_z, options), (col_w, options)], + vec![(col_x, option_asc), (col_y, option_asc)], + vec![(col_z, option_asc), (col_w, option_asc)], ]; let orderings = convert_to_orderings(&orderings); // Right child ordering equivalences @@ -2082,6 +2092,7 @@ mod tests { let col_w = &col("w", &schema)?; let mut join_schema_properties = SchemaProperties::new(Arc::new(schema)); + // a=x and d=w join_schema_properties.add_equal_conditions(col_a, col_x); join_schema_properties.add_equal_conditions(col_d, col_w); @@ -2093,9 +2104,10 @@ mod tests { join_schema_properties.add_ordering_equivalence_class(result); let result = join_schema_properties.oeq_class().clone(); + // [x ASC, y ASC], [z ASC, w ASC] let orderings = vec![ - vec![(col_x, options), (col_y, options)], - vec![(col_z, options), (col_w, options)], + vec![(col_x, option_asc), (col_y, option_asc)], + vec![(col_z, option_asc), (col_w, option_asc)], ]; let orderings = convert_to_orderings(&orderings); let expected = OrderingEquivalenceClass::new(orderings); @@ -2167,6 +2179,9 @@ mod tests { Ok(sorted_indices == original_indices) } + // If we already generated a random result for one of the + // expressions in the equivalence classes. For other expressions in the same + // equivalence class use same result. This util gets already calculated result, when available. fn get_representative_arr( eq_group: &[Arc], existing_vec: &[Option], @@ -2182,7 +2197,8 @@ mod tests { None } - // Generate a table that satisfies schema properties, in terms of ordering equivalences. + // Generate a table that satisfies schema properties, + // in terms of ordering equivalences, equivalences, and constants. fn generate_table_for_schema_properties( schema_properties: &SchemaProperties, n_elem: usize, @@ -2276,7 +2292,9 @@ mod tests { // Test cases for equivalence normalization, // First entry in the tuple is argument, second entry is expected result after normalization. let expressions = vec![ - // Normalized version of the column a and c should go to a (since a is head) + // Normalized version of the column a and c should go to a + // (by convention all the expressions inside equivalence class are mapped to the first entry + // in this case a is the first entry in the equivalence class.) (&col_a_expr, &col_a_expr), (&col_c_expr, &col_a_expr), // Cannot normalize column b @@ -2306,7 +2324,8 @@ mod tests { let col_d = &col("d", &test_schema)?; // Test cases for equivalence normalization - // First entry in the tuple is PhysicalExpr, second entry is its ordering, third entry is result after normalization. + // First entry in the tuple is PhysicalSortRequirement, second entry in the tuple is + // expected PhysicalSortRequirement after normalization. let test_cases = vec![ (vec![(col_a, Some(option1))], vec![(col_a, Some(option1))]), // In the normalized version column c should be replace with column a @@ -2341,39 +2360,48 @@ mod tests { let col_d = &col("d", &test_schema)?; let col_e = &col("e", &test_schema)?; let col_f = &col("f", &test_schema)?; - let option1 = SortOptions { + let option_asc = SortOptions { descending: false, nulls_first: false, }; - let option2 = SortOptions { + let option_desc = SortOptions { descending: true, nulls_first: true, }; // First element in the tuple stores vector of requirement, second element is the expected return value for ordering_satisfy function let requirements = vec![ - (vec![(col_a, Some(option1))], vec![(col_a, Some(option1))]), - (vec![(col_a, Some(option2))], vec![(col_a, Some(option2))]), + ( + vec![(col_a, Some(option_asc))], + vec![(col_a, Some(option_asc))], + ), + ( + vec![(col_a, Some(option_desc))], + vec![(col_a, Some(option_desc))], + ), (vec![(col_a, None)], vec![(col_a, None)]), // Test whether equivalence works as expected - (vec![(col_c, Some(option1))], vec![(col_a, Some(option1))]), + ( + vec![(col_c, Some(option_asc))], + vec![(col_a, Some(option_asc))], + ), (vec![(col_c, None)], vec![(col_a, None)]), // Test whether ordering equivalence works as expected ( - vec![(col_d, Some(option1)), (col_b, Some(option1))], - vec![(col_d, Some(option1)), (col_b, Some(option1))], + vec![(col_d, Some(option_asc)), (col_b, Some(option_asc))], + vec![(col_d, Some(option_asc)), (col_b, Some(option_asc))], ), ( vec![(col_d, None), (col_b, None)], vec![(col_d, None), (col_b, None)], ), ( - vec![(col_e, Some(option2)), (col_f, Some(option1))], - vec![(col_e, Some(option2)), (col_f, Some(option1))], + vec![(col_e, Some(option_desc)), (col_f, Some(option_asc))], + vec![(col_e, Some(option_desc)), (col_f, Some(option_asc))], ), // We should be able to normalize in compatible requirements also (not exactly equal) ( - vec![(col_e, Some(option2)), (col_f, None)], - vec![(col_e, Some(option2)), (col_f, None)], + vec![(col_e, Some(option_desc)), (col_f, None)], + vec![(col_e, Some(option_desc)), (col_f, None)], ), ( vec![(col_e, None), (col_f, None)], @@ -2408,6 +2436,8 @@ mod tests { descending: true, nulls_first: true, }; + // First entry, and second entry are the physical sort requirement that are argument for get_finer_requirement. + // Third entry is the expected result. let tests_cases = vec![ // Get finer requirement between [a Some(ASC)] and [a None, b Some(ASC)] // result should be [a Some(ASC), b Some(ASC)] @@ -2514,6 +2544,8 @@ mod tests { descending: true, nulls_first: true, }; + // First entry is physical expression list + // Second entry is the ordered section that consists of given physical expressions. let test_cases = vec![ // TEST CASE 1 (vec![col_a], Some(vec![(col_a, option_asc)])), @@ -2566,6 +2598,7 @@ mod tests { descending: true, nulls_first: true, }; + // [d ASC, h ASC] also satisfies schema. schema_props.add_new_orderings(&[vec![ PhysicalSortExpr { expr: col_d.clone(), From 4bd7a8d40dfe00b2d4c91cf92f63209151653be4 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Fri, 27 Oct 2023 01:51:22 +0300 Subject: [PATCH 101/122] Review Part 13 --- datafusion/physical-expr/src/equivalence.rs | 194 +++++++++--------- .../physical-plan/src/repartition/mod.rs | 10 +- 2 files changed, 102 insertions(+), 102 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 6f51a9882e63..6de01aa9ccc0 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -412,11 +412,16 @@ pub struct OrderingEquivalenceClass { } impl OrderingEquivalenceClass { - /// Creates new empty ordering equivalent class. + /// Creates new empty ordering equivalence class. fn empty() -> Self { Self { orderings: vec![] } } + /// Clears (empties) this ordering equivalence class. + pub fn clear(&mut self) { + self.orderings.clear(); + } + /// Creates new ordering equivalence class from the given orderings. pub fn new(orderings: Vec) -> Self { let mut result = Self { orderings }; @@ -542,11 +547,13 @@ fn finer_side(lhs: LexOrderingRef, rhs: LexOrderingRef) -> Option { all_equal.then_some(lhs.len() < rhs.len()) } -/// `SchemaProperties` keeps track of useful information related to schema. -/// Currently, it keeps track of +/// A `SchemaProperties` object stores useful information related to a schema. +/// Currently, it keeps track of: /// - Equivalent columns, e.g columns that have same value. -/// - Valid ordering sort expressions for the schema. -/// Consider table below +/// - Valid sort expressions (orderings) for the schema. +/// +/// Consider table below: +/// /// ```text /// ┌-------┐ /// | a | b | @@ -557,10 +564,13 @@ fn finer_side(lhs: LexOrderingRef, rhs: LexOrderingRef) -> Option { /// | 5 | 5 | /// └---┴---┘ /// ``` +/// /// where both `a ASC` and `b DESC` can describe the table ordering. With -/// `SchemaProperties`, we can keep track of these different valid ordering expressions -/// and treat `a ASC` and `b DESC` as the same ordering requirement. -/// Similarly, as in the table below if we know that Column a and b have always same value. +/// `SchemaProperties`, we can keep track of these different valid sort +/// expressions and treat `a ASC` and `b DESC` on an equal footing. +/// +/// Similarly, consider the table below: +/// /// ```text /// ┌-------┐ /// | a | b | @@ -571,24 +581,28 @@ fn finer_side(lhs: LexOrderingRef, rhs: LexOrderingRef) -> Option { /// | 5 | 5 | /// └---┴---┘ /// ``` -/// We keep track of their equivalence inside schema properties. With this information -/// if partition requirement is Hash(a), and output partitioning is Hash(b). We can deduce that -/// existing partitioning satisfies the requirement. +/// where columns `a` and `b` always have the same value. We keep track of such +/// equivalences inside this object. With this information, we can optimize +/// things like partitioning. For example, if the partition requirement is +/// `Hash(a)` and output partitioning is `Hash(b)`, then we can deduce that +/// the existing partitioning satisfies the requirement. #[derive(Debug, Clone)] pub struct SchemaProperties { - /// Keeps track of expressions that have equivalent value. + /// Collection of equivalence classes that store expressions with the same + /// value. eq_group: EquivalenceGroup, - /// Keeps track of valid ordering that satisfied table. + /// Equivalent sort expressions for this table. oeq_class: OrderingEquivalenceClass, - /// Keeps track of expressions that have constant value. + /// Expressions whose values are constant throughout the table. /// TODO: We do not need to track constants separately, they can be tracked - /// inside `eq_groups` as `Literal` expressions. + /// inside `eq_groups` as `Literal` expressions. constants: Vec>, + /// Schema associated with this object. schema: SchemaRef, } impl SchemaProperties { - /// Create an empty `SchemaProperties` + /// Creates an empty `SchemaProperties` object. pub fn new(schema: SchemaRef) -> Self { Self { eq_group: EquivalenceGroup::empty(), @@ -598,24 +612,24 @@ impl SchemaProperties { } } - /// Get schema. + /// Returns the associated schema. pub fn schema(&self) -> &SchemaRef { &self.schema } - /// Return a reference to the ordering equivalence class + /// Returns a reference to the ordering equivalence class within. pub fn oeq_class(&self) -> &OrderingEquivalenceClass { &self.oeq_class } - /// Return a reference to the equivalent group + /// Returns a reference to the equivalence group within. pub fn eq_group(&self) -> &EquivalenceGroup { &self.eq_group } - /// Return the normalized version of the ordering equivalence class - /// Where constants, duplicates are removed and expressions are normalized - /// according to equivalent group. + /// Returns the normalized version of the ordering equivalence class within. + /// Normalization removes constants and duplicates as well as standardizing + /// expressions according to the equivalence group within. pub fn normalized_oeq_class(&self) -> OrderingEquivalenceClass { // Construct a new ordering group that is normalized // With equivalences, and constants are removed @@ -627,21 +641,21 @@ impl SchemaProperties { OrderingEquivalenceClass::new(normalized_orderings) } - /// Add SchemaProperties of the other to the state. + /// Extends this `SchemaProperties` with the `other` object. pub fn extend(mut self, other: Self) -> Self { self.eq_group.extend(other.eq_group); self.oeq_class.extend(other.oeq_class); self.with_constants(other.constants) } - /// Empties the `oeq_class` inside self, When existing orderings are invalidated. - pub fn with_empty_ordering_equivalence(mut self) -> Self { - self.oeq_class = OrderingEquivalenceClass::empty(); - self + /// Clears (empties) the ordering equivalence class within this object. + /// Call this method when existing orderings are invalidated. + pub fn clear(&mut self) { + self.oeq_class.clear(); } - /// Extends `SchemaProperties` by adding ordering inside the `other` - /// to the `self.oeq_class`. + /// Extends this `SchemaProperties` by adding the orderings inside the + /// ordering equivalence class `other`. pub fn add_ordering_equivalence_class(&mut self, other: OrderingEquivalenceClass) { for ordering in other.into_iter() { if !self.oeq_class.contains(&ordering) { @@ -656,7 +670,7 @@ impl SchemaProperties { } /// Incorporates the given equivalence group to into the existing - /// equivalence group in this schema. + /// equivalence group within. pub fn add_equivalence_group(&mut self, other_eq_group: EquivalenceGroup) { self.eq_group.extend(other_eq_group); } @@ -672,84 +686,84 @@ impl SchemaProperties { self.eq_group.add_equal_conditions(left, right); } - /// Add physical expression that have constant value to the `self.constants` + /// Track/register physical expressions with constant values. pub fn with_constants(mut self, constants: Vec>) -> Self { - let constants = self.eq_group.normalize_exprs(constants); - constants.into_iter().for_each(|constant| { - if !physical_exprs_contains(&self.constants, &constant) { - self.constants.push(constant); + for expr in self.eq_group.normalize_exprs(constants) { + if !physical_exprs_contains(&self.constants, &expr) { + self.constants.push(expr); } - }); + } self } - /// Re-creates `SchemaProperties` given that - /// schema is re-ordered by `sort_expr` in the argument. - pub fn with_reorder(mut self, sort_expr: Vec) -> Self { + /// Updates the ordering equivalence group within assuming that the table + /// is re-sorted according to the argument `sort_exprs`. Note that constants + /// and equivalence classes are unchanged as they are unaffected by a re-sort. + pub fn with_reorder(mut self, sort_exprs: Vec) -> Self { // TODO: In some cases, existing ordering equivalences may still be valid add this analysis. - - // Reset ordering equivalent group with the new ordering. - // Constants, and equivalent groups are still valid after re-sort. - // Hence only `oeq_group` is overwritten. - self.oeq_class = OrderingEquivalenceClass::new(vec![sort_expr]); + self.oeq_class = OrderingEquivalenceClass::new(vec![sort_exprs]); self } - /// Transform `sort_exprs` vector, to standardized version using `eq_groups` and `oeq_group` - /// Assume `eq_groups` states that `Column a` and `Column b` are aliases. - /// Also assume `oeq_group` states that ordering `vec![d ASC]` and `vec![a ASC, c ASC]` are - /// ordering equivalent (in the sense that both describe the ordering of the table). - /// If the `sort_exprs` input to this function were `vec![b ASC, c ASC]`, - /// This function converts `sort_exprs` `vec![b ASC, c ASC]` to first `vec![a ASC, c ASC]` after considering `eq_groups` - /// Then converts `vec![a ASC, c ASC]` to `vec![d ASC]` after considering `oeq_group`. - /// Standardized version `vec![d ASC]` is used in subsequent operations. + /// Normalizes the given sort expressions (i.e. `sort_exprs`) using the + /// equivalence group and the ordering equivalence class within. + /// + /// Assume that `self.eq_group` states column `a` and `b` are aliases. + /// Also assume that `self.oeq_class` states orderings `d ASC` and `a ASC, c ASC` + /// are equivalent (in the sense that both describe the ordering of the table). + /// If the `sort_exprs` argument were `vec![b ASC, c ASC]`, then this function + /// would return `vec![d ASC]`. fn normalize_sort_exprs(&self, sort_exprs: LexOrderingRef) -> LexOrdering { - // Convert `PhysicalSortExpr`s to `PhysicalSortRequirement`s - let sort_requirements = - PhysicalSortRequirement::from_sort_exprs(sort_exprs.iter()); - let normalized_exprs = self.normalize_sort_requirements(&sort_requirements); - // Convert back `PhysicalSortRequirement`s to `PhysicalSortExpr`s - PhysicalSortRequirement::to_sort_exprs(normalized_exprs) - } - - /// This function normalizes `sort_reqs` by - /// - removing expressions that have constant value from requirement - /// - replacing sections that are in the `self.oeq_group` with `oeq_group[0]` (e.g standard representative - /// version of the group) - /// - removing sections that satisfies global ordering that are in the post fix of requirement + // Convert sort expressions to sort requirements: + let sort_reqs = PhysicalSortRequirement::from_sort_exprs(sort_exprs.iter()); + // Normalize the requirements: + let normalized_sort_reqs = self.normalize_sort_requirements(&sort_reqs); + // Convert sort requirements back to sort expressions: + PhysicalSortRequirement::to_sort_exprs(normalized_sort_reqs) + } + + /// Normalizes the given sort requirements (i.e. `sort_reqs`) using the + /// equivalence group and the ordering equivalence class within. It works by: + /// - Removing expressions that have a constant value from the given requirement. + /// - Replacing sections that belong to some equivalence class in the equivalence + /// group with the first entry in the matching equivalence class. + /// - Removing sections that satisfy global ordering that are in the post fix of requirement. /// - /// Transform `sort_reqs` vector, to standardized version using `eq_groups` and `oeq_group` - /// Assume `eq_groups` states that `Column a` and `Column b` are aliases. - /// Also assume `oeq_group` states that ordering `vec![d ASC]` and `vec![a ASC, c ASC]` are - /// ordering equivalent (in the sense that both describe the ordering of the table). - /// If the `sort_reqs` input to this function were `vec![b Some(ASC), c None]`, - /// This function converts `sort_exprs` `vec![b Some(ASC), c None]` to first `vec![a Some(ASC), c None]` after considering `eq_groups` - /// Then converts `vec![a Some(ASC), c None]` to `vec![d Some(ASC)]` after considering `oeq_group`. - /// Standardized version `vec![d Some(ASC)]` is used in subsequent operations. + /// Assume that `self.eq_group` states column `a` and `b` are aliases. + /// Also assume that `self.oeq_class` states orderings `d ASC` and `a ASC, c ASC` + /// are equivalent (in the sense that both describe the ordering of the table). + /// If the `sort_reqs` argument were `vec![b ASC, c ASC]`, then this function + /// would return `vec![d ASC]`. fn normalize_sort_requirements( &self, sort_reqs: LexRequirementRef, ) -> LexRequirement { let normalized_sort_reqs = self.eq_group.normalize_sort_requirements(sort_reqs); let constants_normalized = self.eq_group.normalize_exprs(self.constants.clone()); - let normalized_sort_reqs = - prune_sort_reqs_with_constants(&normalized_sort_reqs, &constants_normalized); - // Prune redundant sections in the requirement. - collapse_lex_req(normalized_sort_reqs) + // Prune redundant sections in the requirement: + collapse_lex_req( + normalized_sort_reqs + .iter() + .filter(|&order| { + !physical_exprs_contains(&constants_normalized, &order.expr) + }) + .cloned() + .collect(), + ) } /// Checks whether the given ordering is satisfied by any of the existing /// orderings. - pub fn ordering_satisfy(&self, required: LexOrderingRef) -> bool { - // Convert `PhysicalSortExpr`s to `PhysicalSortRequirement`s - let sort_requirements = PhysicalSortRequirement::from_sort_exprs(required.iter()); + pub fn ordering_satisfy(&self, given: LexOrderingRef) -> bool { + // Convert the given sort expressions to sort requirements: + let sort_requirements = PhysicalSortRequirement::from_sort_exprs(given.iter()); self.ordering_satisfy_requirement(&sort_requirements) } - /// Checks whether the given [`PhysicalSortRequirement`]s are satisfied by the - /// provided [`PhysicalSortExpr`]s. - pub fn ordering_satisfy_requirement(&self, required: LexRequirementRef) -> bool { - self.prune_lex_req(required).is_empty() + /// Checks whether the given sort requirements are satisfied by any of the + /// existing orderings. + pub fn ordering_satisfy_requirement(&self, reqs: LexRequirementRef) -> bool { + self.prune_lex_req(reqs).is_empty() } /// Checks whether the given [`PhysicalSortRequirement`]s are equal or more @@ -1155,18 +1169,6 @@ pub fn collapse_lex_req(input: LexRequirement) -> LexRequirement { output } -/// Remove ordering requirements that have constant value -fn prune_sort_reqs_with_constants( - ordering: LexRequirementRef, - constants: &[Arc], -) -> LexRequirement { - ordering - .iter() - .filter(|&order| !physical_exprs_contains(constants, &order.expr)) - .cloned() - .collect() -} - /// Adds the `offset` value to `Column` indices inside `expr`. This function is /// generally used during the update of the right table schema in join operations. pub fn add_offset_to_expr( diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index f6823bdc9ecd..3cc59ecdca95 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -465,14 +465,12 @@ impl ExecutionPlan for RepartitionExec { } fn schema_properties(&self) -> SchemaProperties { - // If ordering is not preserved, reset ordering equivalent group. + let mut result = self.input.schema_properties(); + // If ordering is not preserved, reset ordering equivalence class. if !self.maintains_input_order()[0] { - self.input - .schema_properties() - .with_empty_ordering_equivalence() - } else { - self.input.schema_properties() + result.clear(); } + result } fn execute( From 299b758a431108da509aeba6dbbb1c1a3f7a006e Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Fri, 27 Oct 2023 18:27:05 +0300 Subject: [PATCH 102/122] Review Part 14 --- datafusion/physical-expr/src/equivalence.rs | 107 ++++++++------------ datafusion/physical-expr/src/sort_expr.rs | 42 +++----- datafusion/physical-plan/src/filter.rs | 2 +- datafusion/physical-plan/src/windows/mod.rs | 8 +- 4 files changed, 60 insertions(+), 99 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 6de01aa9ccc0..fe5fd8b9157d 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -15,6 +15,7 @@ // specific language governing permissions and limitations // under the License. +use std::collections::HashSet; use std::hash::Hash; use std::sync::Arc; @@ -507,7 +508,7 @@ impl OrderingEquivalenceClass { // Append orderings in `other` to all existing orderings in this equivalence // class. - pub fn join_postfix(mut self, other: &Self) -> Self { + pub fn join_suffix(mut self, other: &Self) -> Self { for postfix in other.iter() { for idx in 0..self.orderings.len() { self.orderings[idx].extend(postfix.iter().cloned()); @@ -645,7 +646,7 @@ impl SchemaProperties { pub fn extend(mut self, other: Self) -> Self { self.eq_group.extend(other.eq_group); self.oeq_class.extend(other.oeq_class); - self.with_constants(other.constants) + self.add_constants(other.constants) } /// Clears (empties) the ordering equivalence class within this object. @@ -687,7 +688,7 @@ impl SchemaProperties { } /// Track/register physical expressions with constant values. - pub fn with_constants(mut self, constants: Vec>) -> Self { + pub fn add_constants(mut self, constants: Vec>) -> Self { for expr in self.eq_group.normalize_exprs(constants) { if !physical_exprs_contains(&self.constants, &expr) { self.constants.push(expr); @@ -727,7 +728,6 @@ impl SchemaProperties { /// - Removing expressions that have a constant value from the given requirement. /// - Replacing sections that belong to some equivalence class in the equivalence /// group with the first entry in the matching equivalence class. - /// - Removing sections that satisfy global ordering that are in the post fix of requirement. /// /// Assume that `self.eq_group` states column `a` and `b` are aliases. /// Also assume that `self.oeq_class` states orderings `d ASC` and `a ASC, c ASC` @@ -763,7 +763,36 @@ impl SchemaProperties { /// Checks whether the given sort requirements are satisfied by any of the /// existing orderings. pub fn ordering_satisfy_requirement(&self, reqs: LexRequirementRef) -> bool { - self.prune_lex_req(reqs).is_empty() + // First, standardize the given requirement: + let normalized_reqs = self.normalize_sort_requirements(reqs); + if normalized_reqs.is_empty() { + // Requirements are tautologically satisfied if empty. + return true; + } + let mut indices = HashSet::new(); + for ordering in self.normalized_oeq_class().iter() { + let match_indices = ordering + .iter() + .map(|sort_expr| { + normalized_reqs + .iter() + .position(|sort_req| sort_expr.satisfy(sort_req, &self.schema)) + }) + .collect::>(); + // Find the largest contiguous increasing sequence starting from the first index: + if let Some(&Some(first)) = match_indices.first() { + indices.insert(first); + let mut iter = match_indices.windows(2); + while let Some([Some(current), Some(next)]) = iter.next() { + if next > current { + indices.insert(*next); + } else { + break; + } + } + } + } + indices.len() == normalized_reqs.len() } /// Checks whether the given [`PhysicalSortRequirement`]s are equal or more @@ -776,13 +805,11 @@ impl SchemaProperties { let provided_normalized = self.normalize_sort_requirements(provided); let required_normalized = self.normalize_sort_requirements(required); - if required_normalized.len() > provided_normalized.len() { - return false; - } - required_normalized - .into_iter() - .zip(provided_normalized) - .all(|(req, given)| given.compatible(&req)) + (required_normalized.len() <= provided_normalized.len()) + && required_normalized + .into_iter() + .zip(provided_normalized) + .all(|(req, given)| given.compatible(&req)) } /// Find the finer ordering among `req1` and `req2` @@ -863,54 +890,6 @@ impl SchemaProperties { (!meet.is_empty()).then_some(meet) } - /// This function prunes lexicographical ordering requirement - /// by removing sections inside `sort_req` that satisfies any of the existing ordering. - /// Please note that pruned version may not functionally equivalent to the argument. - /// Empty result means that requirement is already satisfied. - /// Non-empty result means that requirement is not satisfied. - /// This util shouldn't be used outside this context. - fn prune_lex_req(&self, sort_req: LexRequirementRef) -> LexRequirement { - // Make sure to use a standardized version of the requirement - let mut normalized_sort_req = self.normalize_sort_requirements(sort_req); - - // If empty immediately return - if normalized_sort_req.is_empty() { - return normalized_sort_req; - } - - for ordering in self.normalized_oeq_class().iter() { - let match_indices = ordering - .iter() - .map(|elem| { - normalized_sort_req.iter().position(|sort_req| { - elem.satisfy_with_schema(sort_req, &self.schema) - }) - }) - .collect::>(); - - // Find the largest contiguous increasing sequence starting from the first index - let mut to_remove = Vec::new(); - if let Some(&Some(first)) = match_indices.first() { - to_remove.push(first); - for window in match_indices.windows(2) { - if let (Some(current), Some(next)) = (window[0], window[1]) { - if next > current { - to_remove.push(next); - continue; - } - } - break; - } - } - // can remove entries at the match_prefix indices - // Remove with reverse iteration to not invalidate indices - for idx in to_remove.iter().rev() { - normalized_sort_req.remove(*idx); - } - } - normalized_sort_req - } - /// Projects argument `expr` according to mapping inside `source_to_target_mapping`. /// While doing so consider equalities also. /// As an example assume `source_to_target_mapping` contains following mapping @@ -1030,7 +1009,7 @@ impl SchemaProperties { for ordering in self.normalized_oeq_class().iter() { for sort_expr in ordering { if let Some(idx) = normalized_exprs.iter().position(|normalized_expr| { - sort_expr.satisfy_with_schema( + sort_expr.satisfy( &PhysicalSortRequirement { expr: normalized_expr.clone(), options: None, @@ -1107,7 +1086,7 @@ pub fn join_schema_properties( // ordering of the left table is `a ASC`, then the ordering equivalence `b ASC` // for the right table should be converted to `a ASC, b ASC` before it is added // to the ordering equivalences of the join. - let out_oeq_class = left_oeq_class.join_postfix(&updated_right_oeq); + let out_oeq_class = left_oeq_class.join_suffix(&updated_right_oeq); new_properties.add_ordering_equivalence_class(out_oeq_class); } else { new_properties.add_ordering_equivalence_class(left_oeq_class); @@ -1129,7 +1108,7 @@ pub fn join_schema_properties( // ordering of the left table is `a ASC`, then the ordering equivalence `b ASC` // for the right table should be converted to `a ASC, b ASC` before it is added // to the ordering equivalences of the join. - let out_oeq_class = updated_right_oeq.join_postfix(&left_oeq_class); + let out_oeq_class = updated_right_oeq.join_suffix(&left_oeq_class); new_properties.add_ordering_equivalence_class(out_oeq_class); } else { new_properties.add_ordering_equivalence_class(updated_right_oeq); @@ -1382,7 +1361,7 @@ mod tests { // Define a and f are aliases schema_properties.add_equal_conditions(col_a, col_f); // Column e has constant value. - schema_properties = schema_properties.with_constants(vec![col_e.clone()]); + schema_properties = schema_properties.add_constants(vec![col_e.clone()]); // Randomly order columns for sorting let mut rng = StdRng::seed_from_u64(seed); diff --git a/datafusion/physical-expr/src/sort_expr.rs b/datafusion/physical-expr/src/sort_expr.rs index 2b659b23a0a9..ffa89a9cf6ce 100644 --- a/datafusion/physical-expr/src/sort_expr.rs +++ b/datafusion/physical-expr/src/sort_expr.rs @@ -77,42 +77,26 @@ impl PhysicalSortExpr { }) } - /// Check whether sort expression satisfies [`PhysicalSortRequirement`]. - /// - /// If sort options is Some in `PhysicalSortRequirement`, `expr` - /// and `options` field are compared for equality. - /// - /// If sort options is None in `PhysicalSortRequirement`, only - /// `expr` is compared for equality. - pub fn satisfy(&self, requirement: &PhysicalSortRequirement) -> bool { - self.expr.eq(&requirement.expr) - && requirement - .options - .map_or(true, |opts| self.options == opts) - } - - /// Check whether sort expression satisfies [`PhysicalSortRequirement`]. - /// - /// If sort options is Some in `PhysicalSortRequirement`, `expr` - /// and `options` field are compared for equality. - /// - /// If sort options is None in `PhysicalSortRequirement`, only - /// `expr` is compared for equality. - pub fn satisfy_with_schema( + /// Checks whether this sort expression satisfies the given `requirement`. + /// If sort options are unspecified in `requirement`, only expressions are + /// compared for inequality. + pub fn satisfy( &self, requirement: &PhysicalSortRequirement, schema: &Schema, ) -> bool { + // If the column is not nullable, NULLS FIRST/LAST is not important. let nullable = self.expr.nullable(schema).unwrap_or(true); - if nullable { - self.satisfy(requirement) - } else { - self.expr.eq(&requirement.expr) - && requirement + self.expr.eq(&requirement.expr) + && if nullable { + requirement + .options + .map_or(true, |opts| self.options == opts) + } else { + requirement .options - // If the column is not nullable, NULLS FIRST/LAST is not important. .map_or(true, |opts| self.options.descending == opts.descending) - } + } } /// Returns a [`Display`]able list of `PhysicalSortExpr`. diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index 3ae49e3f0d4e..0440ffc3125e 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -160,7 +160,7 @@ impl ExecutionPlan for FilterExec { .filter(|column| stats.column_statistics[column.index()].is_singleton()) .map(|column| Arc::new(column) as _) .collect(); - filter_oeq.with_constants(constants) + filter_oeq.add_constants(constants) } fn with_new_children( diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index 62ee752d797a..06fe724238fc 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -487,7 +487,7 @@ pub fn get_window_mode( } // Treat partition by exprs as constant. During analysis of requirements are satisfied. - let partition_by_oeq = input_oeq.with_constants(partitionby_exprs.to_vec()); + let partition_by_oeq = input_oeq.add_constants(partitionby_exprs.to_vec()); let order_by_reqs = PhysicalSortRequirement::from_sort_exprs(orderby_keys); let reverse_order_by_reqs = PhysicalSortRequirement::from_sort_exprs(&reverse_order_bys(orderby_keys)); @@ -713,8 +713,7 @@ mod tests { nulls_first: req_nulls_first, }, }; - let res = - physical_ordering.satisfy_with_schema(&required_ordering.into(), &schema); + let res = physical_ordering.satisfy(&required_ordering.into(), &schema); assert_eq!(res, expected); } @@ -754,8 +753,7 @@ mod tests { nulls_first: req_nulls_first, }, }; - let res = - physical_ordering.satisfy_with_schema(&required_ordering.into(), &schema); + let res = physical_ordering.satisfy(&required_ordering.into(), &schema); assert_eq!(res, expected); } From e18ed11c60f14588acd349cb970d69ffeaa2588e Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Sat, 28 Oct 2023 01:23:38 +0300 Subject: [PATCH 103/122] Review Part 15 --- datafusion/physical-expr/src/equivalence.rs | 147 ++++++++++---------- 1 file changed, 74 insertions(+), 73 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index fe5fd8b9157d..f99eb575aa62 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -32,8 +32,6 @@ use arrow_schema::SortOptions; use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::{JoinSide, JoinType, Result}; -use itertools::izip; - /// An `EquivalenceClass` is a set of [`Arc`]s that are known /// to have the same value for all tuples in a relation. These are generated by /// equality predicates, typically equi-join conditions and equality conditions @@ -509,9 +507,9 @@ impl OrderingEquivalenceClass { // Append orderings in `other` to all existing orderings in this equivalence // class. pub fn join_suffix(mut self, other: &Self) -> Self { - for postfix in other.iter() { + for ordering in other.iter() { for idx in 0..self.orderings.len() { - self.orderings[idx].extend(postfix.iter().cloned()); + self.orderings[idx].extend(ordering.iter().cloned()); } } self @@ -632,14 +630,12 @@ impl SchemaProperties { /// Normalization removes constants and duplicates as well as standardizing /// expressions according to the equivalence group within. pub fn normalized_oeq_class(&self) -> OrderingEquivalenceClass { - // Construct a new ordering group that is normalized - // With equivalences, and constants are removed - let normalized_orderings = self - .oeq_class - .iter() - .map(|ordering| self.normalize_sort_exprs(ordering)) - .collect::>(); - OrderingEquivalenceClass::new(normalized_orderings) + OrderingEquivalenceClass::new( + self.oeq_class + .iter() + .map(|ordering| self.normalize_sort_exprs(ordering)) + .collect(), + ) } /// Extends this `SchemaProperties` with the `other` object. @@ -795,44 +791,52 @@ impl SchemaProperties { indices.len() == normalized_reqs.len() } - /// Checks whether the given [`PhysicalSortRequirement`]s are equal or more - /// specific than the provided [`PhysicalSortRequirement`]s. + /// Checks whether the `given`` sort requirements are equal or more specific + /// than the `reference` sort requirements. pub fn requirements_compatible( &self, - provided: LexRequirementRef, - required: LexRequirementRef, + given: LexRequirementRef, + reference: LexRequirementRef, ) -> bool { - let provided_normalized = self.normalize_sort_requirements(provided); - let required_normalized = self.normalize_sort_requirements(required); + let normalized_given = self.normalize_sort_requirements(given); + let normalized_reference = self.normalize_sort_requirements(reference); - (required_normalized.len() <= provided_normalized.len()) - && required_normalized + (normalized_reference.len() <= normalized_given.len()) + && normalized_reference .into_iter() - .zip(provided_normalized) - .all(|(req, given)| given.compatible(&req)) + .zip(normalized_given) + .all(|(reference, given)| given.compatible(&reference)) } - /// Find the finer ordering among `req1` and `req2` - /// Finer requirement is the ordering that satisfies both of the orderings in the arguments. - /// If `None`, this means that `req1` and `req2` are not compatible - /// e.g there is no requirement that satisfies both - /// As an example finer ordering of [a ASC] and [a ASC, b ASC] is [a ASC, b ASC] + /// Returns the finer ordering among the orderings `lhs` and `rhs`, breaking + /// any ties by choosing `lhs`. + /// + /// The finer ordering is the ordering that satisfies both of the orderings. + /// If the orderings are incomparable, returns `None`. + /// + /// For example, the finer ordering among `[a ASC]` and `[a ASC, b ASC]` is + /// the latter. pub fn get_finer_ordering( &self, - req1: LexOrderingRef, - req2: LexOrderingRef, + lhs: LexOrderingRef, + rhs: LexOrderingRef, ) -> Option { - // Convert `PhysicalSortExpr` s to `PhysicalSortRequirement`s. - let req1 = PhysicalSortRequirement::from_sort_exprs(req1); - let req2 = PhysicalSortRequirement::from_sort_exprs(req2); - let finer = self.get_finer_requirement(&req1, &req2); - // Convert back `PhysicalSortRequirement`s to `PhysicalSortExpr`s. + // Convert the given sort expressions to sort requirements: + let lhs = PhysicalSortRequirement::from_sort_exprs(lhs); + let rhs = PhysicalSortRequirement::from_sort_exprs(rhs); + let finer = self.get_finer_requirement(&lhs, &rhs); + // Convert the chosen sort requirements back to sort expressions: finer.map(PhysicalSortRequirement::to_sort_exprs) } - /// Find the finer requirement among `req1` and `req2` - /// If `None`, this means that `req1` and `req2` are not compatible - /// e.g there is no requirement that satisfies both + /// Returns the finer ordering among the requirements `lhs` and `rhs`, + /// breaking any ties by choosing `lhs`. + /// + /// The finer requirements are the ones that satisfy both of the given + /// requirements. If the requirements are incomparable, returns `None`. + /// + /// For example, the finer requirements among `[a ASC]` and `[a ASC, b ASC]` + /// is the latter. pub fn get_finer_requirement( &self, req1: LexRequirementRef, @@ -840,47 +844,44 @@ impl SchemaProperties { ) -> Option { let mut lhs = self.normalize_sort_requirements(req1); let mut rhs = self.normalize_sort_requirements(req2); - if izip!(lhs.iter_mut(), rhs.iter_mut()).all(|(lhs, rhs)| { - match (lhs.options, rhs.options) { - (Some(lhs_opt), Some(rhs_opt)) => { - lhs.expr.eq(&rhs.expr) && lhs_opt == rhs_opt - } - (Some(options), None) => { - rhs.options = Some(options); - lhs.expr.eq(&rhs.expr) - } - (None, Some(options)) => { - lhs.options = Some(options); - lhs.expr.eq(&rhs.expr) - } - (None, None) => lhs.expr.eq(&rhs.expr), - } - }) { - if lhs.len() >= rhs.len() { - return Some(lhs); - } else if rhs.len() >= lhs.len() { - return Some(rhs); - } - } - // Neither `req1` nor `req2` satisfies one another, they are incompatible. - None + lhs.iter_mut() + .zip(rhs.iter_mut()) + .all(|(lhs, rhs)| { + lhs.expr.eq(&rhs.expr) + && match (lhs.options, rhs.options) { + (Some(lhs_opt), Some(rhs_opt)) => lhs_opt == rhs_opt, + (Some(options), None) => { + rhs.options = Some(options); + true + } + (None, Some(options)) => { + lhs.options = Some(options); + true + } + (None, None) => true, + } + }) + .then_some(if lhs.len() >= rhs.len() { lhs } else { rhs }) } - /// Calculates the "meet" of given orderings. - /// The meet is the finest ordering that satisfied by all the given - /// orderings, see . - /// If `None`, this means that `req1` and `req2` are not compatible - /// e.g there is no ordering that is satisfied by both - /// As an example meet ordering of [a ASC] and [a ASC, b ASC] is [a ASC] + /// Calculates the "meet" of the given orderings (`lhs` and `rhs`). + /// The meet of a set of orderings is the finest ordering that is satisfied + /// by all the orderings in that set. For details, see: + /// + /// + /// + /// If there is no ordering that satisfies both `lhs` and `rhs`, returns + /// `None`. As an example, the meet of orderings `[a ASC]` and `[a ASC, b ASC]` + /// is `[a ASC]`. pub fn get_meet_ordering( &self, - req1: LexOrderingRef, - req2: LexOrderingRef, + lhs: LexOrderingRef, + rhs: LexOrderingRef, ) -> Option { - let lhs = self.normalize_sort_exprs(req1); - let rhs = self.normalize_sort_exprs(req2); + let lhs = self.normalize_sort_exprs(lhs); + let rhs = self.normalize_sort_exprs(rhs); let mut meet = vec![]; - for (lhs, rhs) in izip!(lhs.into_iter(), rhs.into_iter()) { + for (lhs, rhs) in lhs.into_iter().zip(rhs.into_iter()) { if lhs.eq(&rhs) { meet.push(lhs); } else { @@ -1267,7 +1268,7 @@ mod tests { use arrow_schema::{Fields, SortOptions}; use datafusion_common::Result; - use itertools::Itertools; + use itertools::{izip, Itertools}; use rand::rngs::StdRng; use rand::seq::SliceRandom; use rand::{Rng, SeedableRng}; From 9c47b4f39de6e0417b643f266a9165c021ad93f5 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Sat, 28 Oct 2023 17:13:14 +0300 Subject: [PATCH 104/122] Review Part 16 --- datafusion/physical-expr/src/equivalence.rs | 70 +++++++-------------- datafusion/physical-expr/src/lib.rs | 5 +- datafusion/physical-plan/src/joins/utils.rs | 11 +++- 3 files changed, 33 insertions(+), 53 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index f99eb575aa62..9d637e0de255 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -367,7 +367,12 @@ impl EquivalenceGroup { result.extend(self.clone()); let updated_eq_classes = right_equivalences .iter() - .map(|cls| add_offset_to_exprs(cls, left_column_count)) + .map(|cls| { + cls.iter() + .cloned() + .map(|item| add_offset_to_expr(item, left_column_count)) + .collect() + }) .collect(); result.extend(Self::new(updated_eq_classes)); } @@ -538,6 +543,24 @@ impl OrderingEquivalenceClass { } } +/// Adds the `offset` value to `Column` indices inside `expr`. This function is +/// generally used during the update of the right table schema in join operations. +pub fn add_offset_to_expr( + expr: Arc, + offset: usize, +) -> Arc { + expr.transform_down(&|e| match e.as_any().downcast_ref::() { + Some(col) => Ok(Transformed::Yes(Arc::new(Column::new( + col.name(), + offset + col.index(), + )))), + None => Ok(Transformed::No(e)), + }) + .unwrap() + // Note that we can safely unwrap here since our transform always returns + // an `Ok` value. +} + /// Returns `true` if the ordering `rhs` is strictly finer than the ordering `rhs`, /// `false` if the ordering `lhs` is at least as fine as the ordering `lhs`, and /// `None` otherwise (i.e. when given orderings are incomparable). @@ -1149,51 +1172,6 @@ pub fn collapse_lex_req(input: LexRequirement) -> LexRequirement { output } -/// Adds the `offset` value to `Column` indices inside `expr`. This function is -/// generally used during the update of the right table schema in join operations. -pub fn add_offset_to_expr( - expr: Arc, - offset: usize, -) -> Arc { - expr.transform_down(&|e| match e.as_any().downcast_ref::() { - Some(col) => Ok(Transformed::Yes(Arc::new(Column::new( - col.name(), - offset + col.index(), - )))), - None => Ok(Transformed::No(e)), - }) - .unwrap() - // Note that we can safely unwrap here since our transform always returns - // an `Ok` value. -} - -/// Adds the `offset` value to `Column` indices inside `exprs`. -fn add_offset_to_exprs( - exprs: &[Arc], - offset: usize, -) -> Vec> { - exprs - .iter() - .cloned() - .map(|item| add_offset_to_expr(item, offset)) - .collect() -} - -/// Adds the `offset` value to `Column` indices for each sort expression -/// inside `sort_exprs`. -pub fn add_offset_to_lex_ordering( - sort_exprs: LexOrderingRef, - offset: usize, -) -> LexOrdering { - sort_exprs - .iter() - .map(|sort_expr| PhysicalSortExpr { - expr: add_offset_to_expr(sort_expr.expr.clone(), offset), - options: sort_expr.options, - }) - .collect() -} - /// Calculates the [`SortProperties`] of a given [`ExprOrdering`] node. /// The node is either a leaf node, or an intermediate node: /// - If it is a leaf node, the children states are `None`. We directly find diff --git a/datafusion/physical-expr/src/lib.rs b/datafusion/physical-expr/src/lib.rs index f24188504717..95040e243505 100644 --- a/datafusion/physical-expr/src/lib.rs +++ b/datafusion/physical-expr/src/lib.rs @@ -53,10 +53,7 @@ pub use aggregate::groups_accumulator::{ }; pub use aggregate::AggregateExpr; pub use analysis::{analyze, AnalysisContext, ExprBoundaries}; -pub use equivalence::{ - add_offset_to_lex_ordering, schema_properties_helper, SchemaProperties, -}; - +pub use equivalence::{schema_properties_helper, SchemaProperties}; pub use partitioning::{Distribution, Partitioning}; pub use physical_expr::{ physical_exprs_bag_equal, physical_exprs_contains, physical_exprs_equal, diff --git a/datafusion/physical-plan/src/joins/utils.rs b/datafusion/physical-plan/src/joins/utils.rs index 3d6e590ea882..0cbf8b6b20ce 100644 --- a/datafusion/physical-plan/src/joins/utils.rs +++ b/datafusion/physical-plan/src/joins/utils.rs @@ -45,8 +45,7 @@ use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::intervals::{ExprIntervalGraph, Interval, IntervalBound}; use datafusion_physical_expr::utils::merge_vectors; use datafusion_physical_expr::{ - add_offset_to_lex_ordering, LexOrdering, LexOrderingRef, PhysicalExpr, - PhysicalSortExpr, + LexOrdering, LexOrderingRef, PhysicalExpr, PhysicalSortExpr, }; use futures::future::{BoxFuture, Shared}; @@ -178,7 +177,13 @@ pub fn calculate_join_output_ordering( // In the case below, right ordering should be offseted with the left // side length, since we append the right table to the left table. JoinType::Inner | JoinType::Left | JoinType::Right | JoinType::Full => { - add_offset_to_lex_ordering(right_ordering, left_columns_len) + right_ordering + .iter() + .map(|sort_expr| PhysicalSortExpr { + expr: add_offset_to_expr(sort_expr.expr.clone(), left_columns_len), + options: sort_expr.options, + }) + .collect() } _ => right_ordering.to_vec(), }; From d111db385e501eb7474936fe6f80247424b14bb3 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Sat, 28 Oct 2023 21:18:23 +0300 Subject: [PATCH 105/122] Review Part 17 --- .../datasource/physical_plan/arrow_file.rs | 9 ++--- .../core/src/datasource/physical_plan/avro.rs | 7 ++-- .../core/src/datasource/physical_plan/csv.rs | 7 ++-- .../core/src/datasource/physical_plan/json.rs | 7 ++-- .../src/datasource/physical_plan/parquet.rs | 8 +++-- datafusion/physical-expr/src/equivalence.rs | 34 +++++++++---------- datafusion/physical-expr/src/lib.rs | 2 +- datafusion/physical-plan/src/lib.rs | 3 +- datafusion/physical-plan/src/memory.rs | 4 +-- 9 files changed, 46 insertions(+), 35 deletions(-) diff --git a/datafusion/core/src/datasource/physical_plan/arrow_file.rs b/datafusion/core/src/datasource/physical_plan/arrow_file.rs index ee3465aa3a13..acec6f60bda2 100644 --- a/datafusion/core/src/datasource/physical_plan/arrow_file.rs +++ b/datafusion/core/src/datasource/physical_plan/arrow_file.rs @@ -32,9 +32,7 @@ use crate::physical_plan::{ use arrow_schema::SchemaRef; use datafusion_common::Statistics; use datafusion_execution::TaskContext; -use datafusion_physical_expr::{ - schema_properties_helper, LexOrdering, PhysicalSortExpr, SchemaProperties, -}; +use datafusion_physical_expr::{LexOrdering, PhysicalSortExpr, SchemaProperties}; use futures::StreamExt; use object_store::{GetResultPayload, ObjectStore}; @@ -106,7 +104,10 @@ impl ExecutionPlan for ArrowExec { } fn schema_properties(&self) -> SchemaProperties { - schema_properties_helper(self.schema(), &self.projected_output_ordering) + SchemaProperties::new_with_orderings( + self.schema(), + &self.projected_output_ordering, + ) } fn children(&self) -> Vec> { diff --git a/datafusion/core/src/datasource/physical_plan/avro.rs b/datafusion/core/src/datasource/physical_plan/avro.rs index 337cc6194847..f3778d11970c 100644 --- a/datafusion/core/src/datasource/physical_plan/avro.rs +++ b/datafusion/core/src/datasource/physical_plan/avro.rs @@ -31,7 +31,7 @@ use crate::physical_plan::{ use arrow::datatypes::SchemaRef; use datafusion_execution::TaskContext; -use datafusion_physical_expr::{schema_properties_helper, LexOrdering, SchemaProperties}; +use datafusion_physical_expr::{LexOrdering, SchemaProperties}; /// Execution plan for scanning Avro data source #[derive(Debug, Clone)] @@ -100,7 +100,10 @@ impl ExecutionPlan for AvroExec { } fn schema_properties(&self) -> SchemaProperties { - schema_properties_helper(self.schema(), &self.projected_output_ordering) + SchemaProperties::new_with_orderings( + self.schema(), + &self.projected_output_ordering, + ) } fn children(&self) -> Vec> { diff --git a/datafusion/core/src/datasource/physical_plan/csv.rs b/datafusion/core/src/datasource/physical_plan/csv.rs index 6b495bdc57fc..31f4aa2edfa8 100644 --- a/datafusion/core/src/datasource/physical_plan/csv.rs +++ b/datafusion/core/src/datasource/physical_plan/csv.rs @@ -41,7 +41,7 @@ use crate::physical_plan::{ use arrow::csv; use arrow::datatypes::SchemaRef; use datafusion_execution::TaskContext; -use datafusion_physical_expr::{schema_properties_helper, LexOrdering, SchemaProperties}; +use datafusion_physical_expr::{LexOrdering, SchemaProperties}; use bytes::{Buf, Bytes}; use futures::{ready, StreamExt, TryStreamExt}; @@ -185,7 +185,10 @@ impl ExecutionPlan for CsvExec { } fn schema_properties(&self) -> SchemaProperties { - schema_properties_helper(self.schema(), &self.projected_output_ordering) + SchemaProperties::new_with_orderings( + self.schema(), + &self.projected_output_ordering, + ) } fn children(&self) -> Vec> { diff --git a/datafusion/core/src/datasource/physical_plan/json.rs b/datafusion/core/src/datasource/physical_plan/json.rs index 554c26aeba7b..86283148c40a 100644 --- a/datafusion/core/src/datasource/physical_plan/json.rs +++ b/datafusion/core/src/datasource/physical_plan/json.rs @@ -40,7 +40,7 @@ use crate::physical_plan::{ use arrow::json::ReaderBuilder; use arrow::{datatypes::SchemaRef, json}; use datafusion_execution::TaskContext; -use datafusion_physical_expr::{schema_properties_helper, LexOrdering, SchemaProperties}; +use datafusion_physical_expr::{LexOrdering, SchemaProperties}; use bytes::{Buf, Bytes}; use futures::{ready, stream, StreamExt, TryStreamExt}; @@ -121,7 +121,10 @@ impl ExecutionPlan for NdJsonExec { } fn schema_properties(&self) -> SchemaProperties { - schema_properties_helper(self.schema(), &self.projected_output_ordering) + SchemaProperties::new_with_orderings( + self.schema(), + &self.projected_output_ordering, + ) } fn children(&self) -> Vec> { diff --git a/datafusion/core/src/datasource/physical_plan/parquet.rs b/datafusion/core/src/datasource/physical_plan/parquet.rs index 2bc457a63117..7f716ba75944 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet.rs @@ -45,8 +45,7 @@ use crate::{ use arrow::datatypes::{DataType, SchemaRef}; use arrow::error::ArrowError; use datafusion_physical_expr::{ - schema_properties_helper, LexOrdering, PhysicalExpr, PhysicalSortExpr, - SchemaProperties, + LexOrdering, PhysicalExpr, PhysicalSortExpr, SchemaProperties, }; use bytes::Bytes; @@ -336,7 +335,10 @@ impl ExecutionPlan for ParquetExec { } fn schema_properties(&self) -> SchemaProperties { - schema_properties_helper(self.schema(), &self.projected_output_ordering) + SchemaProperties::new_with_orderings( + self.schema(), + &self.projected_output_ordering, + ) } fn with_new_children( diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 9d637e0de255..41f2922932a6 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -634,6 +634,19 @@ impl SchemaProperties { } } + /// Creates a new `SchemaProperties` object with the given orderings. + pub fn new_with_orderings( + schema: SchemaRef, + orderings: &[LexOrdering], + ) -> SchemaProperties { + Self { + eq_group: EquivalenceGroup::empty(), + oeq_class: OrderingEquivalenceClass::new(orderings.to_vec()), + constants: vec![], + schema, + } + } + /// Returns the associated schema. pub fn schema(&self) -> &SchemaRef { &self.schema @@ -1143,19 +1156,6 @@ pub fn join_schema_properties( new_properties } -/// Constructs a `SchemaProperties` struct from the given `orderings`. -pub fn schema_properties_helper( - schema: SchemaRef, - orderings: &[LexOrdering], -) -> SchemaProperties { - let mut oep = SchemaProperties::new(schema); - if !orderings.is_empty() { - let group = OrderingEquivalenceClass::new(orderings.to_vec()); - oep.add_ordering_equivalence_class(group); - } - oep -} - /// This function constructs a duplicate-free `LexOrderingReq` by filtering out /// duplicate entries that have same physical expression inside. For example, /// `vec![a Some(Asc), a Some(Desc)]` collapses to `vec![a Some(Asc)]`. @@ -1189,16 +1189,16 @@ fn update_ordering( mut node: ExprOrdering, ordering_equal_properties: &SchemaProperties, ) -> Result> { - let eq_groups = &ordering_equal_properties.eq_group; - let oeq_group = &ordering_equal_properties.oeq_class; if let Some(children_sort_options) = &node.children_states { // We have an intermediate (non-leaf) node, account for its children: node.state = Some(node.expr.get_ordering(children_sort_options)); Ok(Transformed::Yes(node)) } else if node.expr.as_any().is::() { // We have a Column, which is one of the two possible leaf node types: - let normalized_expr = eq_groups.normalize_expr(node.expr.clone()); - if let Some(options) = oeq_group.get_options(&normalized_expr) { + let eq_group = &ordering_equal_properties.eq_group; + let normalized_expr = eq_group.normalize_expr(node.expr.clone()); + let oeq_class = &ordering_equal_properties.oeq_class; + if let Some(options) = oeq_class.get_options(&normalized_expr) { node.state = Some(SortProperties::Ordered(options)); Ok(Transformed::Yes(node)) } else { diff --git a/datafusion/physical-expr/src/lib.rs b/datafusion/physical-expr/src/lib.rs index 95040e243505..5d0dfec8cbab 100644 --- a/datafusion/physical-expr/src/lib.rs +++ b/datafusion/physical-expr/src/lib.rs @@ -53,7 +53,7 @@ pub use aggregate::groups_accumulator::{ }; pub use aggregate::AggregateExpr; pub use analysis::{analyze, AnalysisContext, ExprBoundaries}; -pub use equivalence::{schema_properties_helper, SchemaProperties}; +pub use equivalence::SchemaProperties; pub use partitioning::{Distribution, Partitioning}; pub use physical_expr::{ physical_exprs_bag_equal, physical_exprs_contains, physical_exprs_equal, diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs index a741afccf7b3..e463301eaabe 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -81,8 +81,7 @@ pub use datafusion_common::{internal_err, ColumnStatistics, Statistics}; pub use datafusion_expr::{Accumulator, ColumnarValue}; pub use datafusion_physical_expr::window::WindowExpr; pub use datafusion_physical_expr::{ - expressions, functions, schema_properties_helper, udf, AggregateExpr, Distribution, - Partitioning, PhysicalExpr, + expressions, functions, udf, AggregateExpr, Distribution, Partitioning, PhysicalExpr, }; // Backwards compatibility diff --git a/datafusion/physical-plan/src/memory.rs b/datafusion/physical-plan/src/memory.rs index 3f074cf17774..2725dad7aab7 100644 --- a/datafusion/physical-plan/src/memory.rs +++ b/datafusion/physical-plan/src/memory.rs @@ -32,7 +32,7 @@ use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; use datafusion_common::{internal_err, project_schema, DataFusionError, Result}; use datafusion_execution::TaskContext; -use datafusion_physical_expr::{schema_properties_helper, LexOrdering, SchemaProperties}; +use datafusion_physical_expr::{LexOrdering, SchemaProperties}; use futures::Stream; @@ -122,7 +122,7 @@ impl ExecutionPlan for MemoryExec { } fn schema_properties(&self) -> SchemaProperties { - schema_properties_helper(self.schema(), &self.sort_information) + SchemaProperties::new_with_orderings(self.schema(), &self.sort_information) } fn with_new_children( From 1206af1a0cd879ab286087e06113d87de5a543cb Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Sun, 29 Oct 2023 11:21:42 +0300 Subject: [PATCH 106/122] Review Part 18 --- .../src/physical_optimizer/sort_pushdown.rs | 12 ++-- datafusion/physical-expr/src/equivalence.rs | 72 +++++++++---------- datafusion/physical-expr/src/utils.rs | 22 +++--- .../physical-plan/src/aggregates/mod.rs | 12 ++-- datafusion/physical-plan/src/common.rs | 6 +- datafusion/physical-plan/src/filter.rs | 6 +- .../physical-plan/src/joins/cross_join.rs | 6 +- .../physical-plan/src/joins/hash_join.rs | 6 +- .../src/joins/sort_merge_join.rs | 6 +- .../src/joins/symmetric_hash_join.rs | 6 +- datafusion/physical-plan/src/memory.rs | 4 +- datafusion/physical-plan/src/projection.rs | 18 +++-- .../physical-plan/src/repartition/mod.rs | 4 +- datafusion/physical-plan/src/sorts/sort.rs | 7 +- datafusion/physical-plan/src/streaming.rs | 6 +- datafusion/physical-plan/src/union.rs | 38 +++++----- 16 files changed, 107 insertions(+), 124 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/sort_pushdown.rs b/datafusion/core/src/physical_optimizer/sort_pushdown.rs index 7f359a09a22c..ef59d81bb44c 100644 --- a/datafusion/core/src/physical_optimizer/sort_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/sort_pushdown.rs @@ -263,12 +263,12 @@ fn pushdown_requirement_to_children( } else if is_sort_preserving_merge(plan) { let new_ordering = PhysicalSortRequirement::to_sort_exprs(parent_required.to_vec()); - let mut spm_oeq = plan.schema_properties(); + let mut spm_eqs = plan.schema_properties(); // Sort preserving merge will have new ordering, one requirement above is pushed down to its below. - spm_oeq = spm_oeq.with_reorder(new_ordering); + spm_eqs = spm_eqs.with_reorder(new_ordering); // Do not push-down through SortPreservingMergeExec when // ordering requirement invalidates requirement of sort preserving merge exec. - if !spm_oeq.ordering_satisfy(plan.output_ordering().unwrap_or(&[])) { + if !spm_eqs.ordering_satisfy(plan.output_ordering().unwrap_or(&[])) { Ok(None) } else { // Can push-down through SortPreservingMergeExec, because parent requirement is finer @@ -350,10 +350,10 @@ fn try_pushdown_requirements_to_join( &smj.maintains_input_order(), Some(probe_side), ); - let mut smj_oeq = smj.schema_properties(); + let mut smj_eqs = smj.schema_properties(); // smj will have this ordering when its input changes. - smj_oeq = smj_oeq.with_reorder(new_output_ordering.unwrap_or_default()); - let should_pushdown = smj_oeq.ordering_satisfy_requirement(parent_required); + smj_eqs = smj_eqs.with_reorder(new_output_ordering.unwrap_or_default()); + let should_pushdown = smj_eqs.ordering_satisfy_requirement(parent_required); Ok(should_pushdown.then(|| { let mut required_input_ordering = smj.required_input_ordering(); let new_req = Some(PhysicalSortRequirement::from_sort_exprs(&sort_expr)); diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 41f2922932a6..7292fce83370 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -603,6 +603,7 @@ fn finer_side(lhs: LexOrderingRef, rhs: LexOrderingRef) -> Option { /// | 5 | 5 | /// └---┴---┘ /// ``` +/// /// where columns `a` and `b` always have the same value. We keep track of such /// equivalences inside this object. With this information, we can optimize /// things like partitioning. For example, if the partition requirement is @@ -683,7 +684,7 @@ impl SchemaProperties { /// Clears (empties) the ordering equivalence class within this object. /// Call this method when existing orderings are invalidated. - pub fn clear(&mut self) { + pub fn clear_orderings(&mut self) { self.oeq_class.clear(); } @@ -927,67 +928,59 @@ impl SchemaProperties { (!meet.is_empty()).then_some(meet) } - /// Projects argument `expr` according to mapping inside `source_to_target_mapping`. - /// While doing so consider equalities also. - /// As an example assume `source_to_target_mapping` contains following mapping + /// Projects argument `expr` according to `projection_mapping`, taking + /// equivalences into account. + /// + /// For example, assume that columns `a` and `c` are always equal, and that + /// `projection_mapping` encodes following mapping: + /// + /// ```text /// a -> a1 /// b -> b1 - /// Also assume that we know that a=c (they are equal) - /// This function projects - /// a+b to Some(a1+b1) - /// c+b to Some(a1+b1) - /// d to None. (meaning cannot be projected) + /// ``` + /// + /// Then, this function projects `a + b` to `Some(a1 + b1)`, `c + b` to + /// `Some(a1 + b1)` and `d` to `None`, meaning that it cannot be projected. pub fn project_expr( &self, - source_to_target_mapping: &ProjectionMapping, expr: &Arc, + projection_mapping: &ProjectionMapping, ) -> Option> { - self.eq_group.project_expr(source_to_target_mapping, expr) + self.eq_group.project_expr(projection_mapping, expr) } - /// Projects `SchemaProperties` according to mapping given in `source_to_target_mapping`. + /// Projects the equivalences within according to `projection_mapping` + /// and `output_schema`. pub fn project( &self, - source_to_target_mapping: &ProjectionMapping, + projection_mapping: &ProjectionMapping, output_schema: SchemaRef, ) -> SchemaProperties { - let mut projected_properties = SchemaProperties::new(output_schema); - - let projected_eq_groups = self.eq_group.project(source_to_target_mapping); - projected_properties.eq_group = projected_eq_groups; - - let projected_orderings = self + let mut projected_orderings = self .oeq_class .iter() - .filter_map(|order| { - self.eq_group - .project_ordering(source_to_target_mapping, order) - }) + .filter_map(|order| self.eq_group.project_ordering(projection_mapping, order)) .collect::>(); - // if empty, no need to track projected_orderings. - if !projected_orderings.is_empty() { - projected_properties.oeq_class = - OrderingEquivalenceClass::new(projected_orderings); - } - - for (source, target) in source_to_target_mapping { + for (source, target) in projection_mapping { let initial_expr = ExprOrdering::new(source.clone()); let transformed = initial_expr .transform_up(&|expr| update_ordering(expr, self)) .unwrap(); if let Some(SortProperties::Ordered(sort_options)) = transformed.state { - let sort_expr = PhysicalSortExpr { + // Push new ordering to the state. + projected_orderings.push(vec![PhysicalSortExpr { expr: target.clone(), options: sort_options, - }; - // Push new ordering to the state. - projected_properties.oeq_class.push(vec![sort_expr]); + }]); } } - // Remove redundant entries from ordering group if any. - // projected_properties.oeq_group.remove_redundant_entries(); - projected_properties + Self { + eq_group: self.eq_group.project(projection_mapping), + oeq_class: OrderingEquivalenceClass::new(projected_orderings), + constants: vec![], + schema: output_schema, + } } /// Check whether any permutation of the argument has a prefix with existing ordering. @@ -1485,14 +1478,13 @@ mod tests { let col_a2 = &col("a2", &out_schema)?; let col_a3 = &col("a3", &out_schema)?; let col_a4 = &col("a4", &out_schema)?; - let source_to_target_mapping = vec![ + let projection_mapping = vec![ (col_a.clone(), col_a1.clone()), (col_a.clone(), col_a2.clone()), (col_a.clone(), col_a3.clone()), (col_a.clone(), col_a4.clone()), ]; - let out_properties = - input_properties.project(&source_to_target_mapping, out_schema); + let out_properties = input_properties.project(&projection_mapping, out_schema); // At the output a1=a2=a3=a4 assert_eq!(out_properties.eq_group().len(), 1); diff --git a/datafusion/physical-expr/src/utils.rs b/datafusion/physical-expr/src/utils.rs index 8f80a34eec07..5db38749b72b 100644 --- a/datafusion/physical-expr/src/utils.rs +++ b/datafusion/physical-expr/src/utils.rs @@ -781,8 +781,8 @@ mod tests { ]; schema_properties.add_new_orderings(&others); - let mut expected_oeq = SchemaProperties::new(Arc::new(schema)); - expected_oeq.add_new_orderings(&[ + let mut expected_eqs = SchemaProperties::new(Arc::new(schema)); + expected_eqs.add_new_orderings(&[ vec![PhysicalSortExpr { expr: col_b_expr.clone(), options: sort_options, @@ -794,7 +794,7 @@ mod tests { ]); let oeq_class = schema_properties.oeq_class().clone(); - let expected = expected_oeq.oeq_class(); + let expected = expected_eqs.oeq_class(); assert!(oeq_class.eq(expected)); Ok(()) @@ -813,7 +813,7 @@ mod tests { options: SortOptions::default(), }]; schema_properties.add_new_orderings(&[ordering]); - let source_to_target_mapping = vec![ + let projection_mapping = vec![ ( Arc::new(Column::new("b", 1)) as _, Arc::new(Column::new("b_new", 0)) as _, @@ -827,9 +827,8 @@ mod tests { Field::new("b_new", DataType::Int32, true), Field::new("a_new", DataType::Int32, true), ])); - let projected_oeq = - schema_properties.project(&source_to_target_mapping, projection_schema); - let orderings = projected_oeq + let orderings = schema_properties + .project(&projection_mapping, projection_schema) .oeq_class() .output_ordering() .unwrap_or_default(); @@ -847,8 +846,8 @@ mod tests { Field::new("b", DataType::Int32, true), Field::new("c", DataType::Int32, true), ]); - let schema_properties = SchemaProperties::new(Arc::new(schema.clone())); - let source_to_target_mapping = vec![ + let schema_properties = SchemaProperties::new(Arc::new(schema)); + let projection_mapping = vec![ ( Arc::new(Column::new("c", 2)) as _, Arc::new(Column::new("c_new", 0)) as _, @@ -862,10 +861,9 @@ mod tests { Field::new("c_new", DataType::Int32, true), Field::new("b_new", DataType::Int32, true), ])); - let projected_oeq = - schema_properties.project(&source_to_target_mapping, projection_schema); + let projected = schema_properties.project(&projection_mapping, projection_schema); // After projection there is no ordering. - assert!(projected_oeq.oeq_class().output_ordering().is_none()); + assert!(projected.oeq_class().output_ordering().is_none()); Ok(()) } diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 82228295f1b3..22e2ae92893e 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -297,7 +297,7 @@ pub struct AggregateExec { /// The mapping used to normalize expressions like Partitioning and /// PhysicalSortExpr. The key is the expression from the input schema /// and the value is the expression from the output schema. - source_to_target_mapping: Vec<(Arc, Arc)>, + projection_mapping: Vec<(Arc, Arc)>, /// Execution metrics metrics: ExecutionPlanMetricsSet, required_input_ordering: Option, @@ -536,7 +536,7 @@ impl AggregateExec { new_requirement = collapse_lex_req(new_requirement); // construct a map from the input expression to the output expression of the Aggregation group by - let source_to_target_mapping = + let projection_mapping = calculate_projection_mapping(&group_by.expr, &input.schema())?; let required_input_ordering = @@ -544,7 +544,7 @@ impl AggregateExec { let aggregate_oeq = input .schema_properties() - .project(&source_to_target_mapping, schema.clone()); + .project(&projection_mapping, schema.clone()); let output_ordering = aggregate_oeq.oeq_class().output_ordering(); Ok(AggregateExec { @@ -556,7 +556,7 @@ impl AggregateExec { input, schema, input_schema, - source_to_target_mapping, + projection_mapping, metrics: ExecutionPlanMetricsSet::new(), required_input_ordering, limit: None, @@ -754,7 +754,7 @@ impl ExecutionPlan for AggregateExec { .into_iter() .map(|expr| { input_schema_properties - .project_expr(&self.source_to_target_mapping, &expr) + .project_expr(&expr, &self.projection_mapping) .unwrap_or_else(|| { Arc::new(UnKnownColumn::new(&expr.to_string())) }) @@ -810,7 +810,7 @@ impl ExecutionPlan for AggregateExec { fn schema_properties(&self) -> SchemaProperties { self.input .schema_properties() - .project(&self.source_to_target_mapping, self.schema()) + .project(&self.projection_mapping, self.schema()) } fn children(&self) -> Vec> { diff --git a/datafusion/physical-plan/src/common.rs b/datafusion/physical-plan/src/common.rs index 823fdb71c8d1..81a59ad7ab3c 100644 --- a/datafusion/physical-plan/src/common.rs +++ b/datafusion/physical-plan/src/common.rs @@ -381,7 +381,7 @@ pub fn calculate_projection_mapping( input_schema: &Arc, ) -> Result { // Construct a map from the input expressions to the output expression of the projection: - let mut source_to_target_mapping = vec![]; + let mut projection_mapping = vec![]; for (expr_idx, (expression, name)) in expr.iter().enumerate() { let target_expr = Arc::new(Column::new(name, expr_idx)) as _; @@ -402,9 +402,9 @@ pub fn calculate_projection_mapping( None => Ok(Transformed::No(e)), })?; - source_to_target_mapping.push((source_expr, target_expr)); + projection_mapping.push((source_expr, target_expr)); } - Ok(source_to_target_mapping) + Ok(projection_mapping) } #[cfg(test)] diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index 0440ffc3125e..c4786912cafa 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -147,12 +147,12 @@ impl ExecutionPlan for FilterExec { fn schema_properties(&self) -> SchemaProperties { let stats = self.statistics().unwrap(); // Combine the equal predicates with the input equivalence properties - let mut filter_oeq = self.input.schema_properties(); + let mut result = self.input.schema_properties(); let (equal_pairs, _) = collect_columns_from_predicate(&self.predicate); for (lhs, rhs) in equal_pairs { let lhs_expr = Arc::new(lhs.clone()) as _; let rhs_expr = Arc::new(rhs.clone()) as _; - filter_oeq.add_equal_conditions(&lhs_expr, &rhs_expr) + result.add_equal_conditions(&lhs_expr, &rhs_expr) } // Add the columns that have only one value (singleton) after filtering to constants. let constants = collect_columns(self.predicate()) @@ -160,7 +160,7 @@ impl ExecutionPlan for FilterExec { .filter(|column| stats.column_statistics[column.index()].is_singleton()) .map(|column| Arc::new(column) as _) .collect(); - filter_oeq.add_constants(constants) + result.add_constants(constants) } fn with_new_children( diff --git a/datafusion/physical-plan/src/joins/cross_join.rs b/datafusion/physical-plan/src/joins/cross_join.rs index e45d5414cef0..c2f9b3632364 100644 --- a/datafusion/physical-plan/src/joins/cross_join.rs +++ b/datafusion/physical-plan/src/joins/cross_join.rs @@ -215,11 +215,9 @@ impl ExecutionPlan for CrossJoinExec { } fn schema_properties(&self) -> SchemaProperties { - let left = self.left.schema_properties(); - let right = self.right.schema_properties(); join_schema_properties( - left, - right, + self.left.schema_properties(), + self.right.schema_properties(), &JoinType::Full, self.schema(), &[false, false], diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index 1e84dc378d46..3ff7784fb7a1 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -366,11 +366,9 @@ impl ExecutionPlan for HashJoinExec { } fn schema_properties(&self) -> SchemaProperties { - let left = self.left.schema_properties(); - let right = self.right.schema_properties(); join_schema_properties( - left, - right, + self.left.schema_properties(), + self.right.schema_properties(), &self.join_type, self.schema(), &self.maintains_input_order(), diff --git a/datafusion/physical-plan/src/joins/sort_merge_join.rs b/datafusion/physical-plan/src/joins/sort_merge_join.rs index 65ca5b2b66c3..d314a5b6eb03 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join.rs @@ -282,11 +282,9 @@ impl ExecutionPlan for SortMergeJoinExec { } fn schema_properties(&self) -> SchemaProperties { - let left = self.left.schema_properties(); - let right = self.right.schema_properties(); join_schema_properties( - left, - right, + self.left.schema_properties(), + self.right.schema_properties(), &self.join_type, self.schema(), &self.maintains_input_order(), diff --git a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs index bddb5cb5a4e1..c7ff6c21cb80 100644 --- a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs +++ b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs @@ -430,11 +430,9 @@ impl ExecutionPlan for SymmetricHashJoinExec { } fn schema_properties(&self) -> SchemaProperties { - let left = self.left.schema_properties(); - let right = self.right.schema_properties(); join_schema_properties( - left, - right, + self.left.schema_properties(), + self.right.schema_properties(), &self.join_type, self.schema(), &self.maintains_input_order(), diff --git a/datafusion/physical-plan/src/memory.rs b/datafusion/physical-plan/src/memory.rs index 2725dad7aab7..cda3e66ad81e 100644 --- a/datafusion/physical-plan/src/memory.rs +++ b/datafusion/physical-plan/src/memory.rs @@ -300,8 +300,8 @@ mod tests { .with_sort_information(sort_information); assert_eq!(mem_exec.output_ordering().unwrap(), expected_output_order); - let order_eq = mem_exec.schema_properties(); - assert!(order_eq.oeq_class().contains(&expected_order_eq)); + let eq_properties = mem_exec.schema_properties(); + assert!(eq_properties.oeq_class().contains(&expected_order_eq)); Ok(()) } } diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index 5402c85ea0cf..b17721a05b17 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -59,7 +59,7 @@ pub struct ProjectionExec { /// The mapping used to normalize expressions like Partitioning and /// PhysicalSortExpr. The key is the expression from the input schema /// and the value is the expression from the output schema. - source_to_target_mapping: Vec<(Arc, Arc)>, + projection_mapping: Vec<(Arc, Arc)>, /// Execution metrics metrics: ExecutionPlanMetricsSet, } @@ -94,20 +94,18 @@ impl ProjectionExec { )); // construct a map from the input expressions to the output expression of the Projection - let source_to_target_mapping = - calculate_projection_mapping(&expr, &input_schema)?; + let projection_mapping = calculate_projection_mapping(&expr, &input_schema)?; - let input_oeq = input.schema_properties(); - let project_oeq = input_oeq.project(&source_to_target_mapping, schema.clone()); - let project_orderings = project_oeq.oeq_class(); - let output_ordering = project_orderings.output_ordering(); + let input_eqs = input.schema_properties(); + let project_eqs = input_eqs.project(&projection_mapping, schema.clone()); + let output_ordering = project_eqs.oeq_class().output_ordering(); Ok(Self { expr, schema, input, output_ordering, - source_to_target_mapping, + projection_mapping, metrics: ExecutionPlanMetricsSet::new(), }) } @@ -182,7 +180,7 @@ impl ExecutionPlan for ProjectionExec { .into_iter() .map(|expr| { input_schema_properties - .project_expr(&self.source_to_target_mapping, &expr) + .project_expr(&expr, &self.projection_mapping) .unwrap_or_else(|| { Arc::new(UnKnownColumn::new(&expr.to_string())) }) @@ -206,7 +204,7 @@ impl ExecutionPlan for ProjectionExec { fn schema_properties(&self) -> SchemaProperties { self.input .schema_properties() - .project(&self.source_to_target_mapping, self.schema()) + .project(&self.projection_mapping, self.schema()) } fn with_new_children( diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 3cc59ecdca95..3e03f550d7ba 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -466,9 +466,9 @@ impl ExecutionPlan for RepartitionExec { fn schema_properties(&self) -> SchemaProperties { let mut result = self.input.schema_properties(); - // If ordering is not preserved, reset ordering equivalence class. + // If the ordering is lost, reset the ordering equivalence class. if !self.maintains_input_order()[0] { - result.clear(); + result.clear_orderings(); } result } diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index fcebff219c37..b311535a6f32 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -829,9 +829,10 @@ impl ExecutionPlan for SortExec { } fn schema_properties(&self) -> SchemaProperties { - let input_oeq = self.input.schema_properties(); - // reset ordering equivalent group with new ordering. - input_oeq.with_reorder(self.expr.to_vec()) + // Reset the ordering equivalence class with the new ordering: + self.input + .schema_properties() + .with_reorder(self.expr.to_vec()) } fn with_new_children( diff --git a/datafusion/physical-plan/src/streaming.rs b/datafusion/physical-plan/src/streaming.rs index 47a3362ad2e5..1b403978f3b4 100644 --- a/datafusion/physical-plan/src/streaming.rs +++ b/datafusion/physical-plan/src/streaming.rs @@ -157,11 +157,11 @@ impl ExecutionPlan for StreamingTableExec { } fn schema_properties(&self) -> SchemaProperties { - let mut oeq = SchemaProperties::new(self.schema()); + let mut result = SchemaProperties::new(self.schema()); if let Some(ordering) = &self.projected_output_ordering { - oeq.add_new_orderings(&[ordering.clone()]) + result.add_new_orderings(&[ordering.clone()]) } - oeq + result } fn children(&self) -> Vec> { diff --git a/datafusion/physical-plan/src/union.rs b/datafusion/physical-plan/src/union.rs index 12c59a135735..ebbcdeb540a6 100644 --- a/datafusion/physical-plan/src/union.rs +++ b/datafusion/physical-plan/src/union.rs @@ -224,34 +224,36 @@ impl ExecutionPlan for UnionExec { } fn schema_properties(&self) -> SchemaProperties { - // TODO: In some cases equivalent groups and constants - // can be preserved in union. Add support for these. - let child_oeqs = self + // TODO: In some cases, we should be able to preserve some equivalence + // classes and constants. Add support for such cases. + let children_eqs = self .inputs .iter() .map(|child| child.schema_properties()) .collect::>(); - let mut union_oeq = SchemaProperties::new(self.schema()); - // Get first ordering equivalent group as seed group. - let mut existing_meets = child_oeqs[0] + let mut result = SchemaProperties::new(self.schema()); + // Use the ordering equivalence class of the first child as the seed: + let mut meets = children_eqs[0] .oeq_class() .iter() - .map(|elem| elem.to_vec()) + .map(|item| item.to_vec()) .collect::>(); - // Iterate ordering equivalent group of other childs - for next_child_oeq in &child_oeqs[1..] { - // Find the valid meet orderings of existing meet and new group. + // Iterate over all the children: + for child_eqs in &children_eqs[1..] { + // Compute meet orderings of the current meets and the new ordering + // equivalence class. let mut next_meets = vec![]; - for existing_meet in &existing_meets { - next_meets.extend(next_child_oeq.oeq_class().iter().filter_map( - |ordering| next_child_oeq.get_meet_ordering(ordering, existing_meet), - )); + for current_meet in &meets { + next_meets.extend(child_eqs.oeq_class().iter().filter_map(|ordering| { + child_eqs.get_meet_ordering(ordering, current_meet) + })); } - existing_meets = next_meets; + meets = next_meets; } - // existing_meets contains the all of the valid orderings after union - union_oeq.add_new_orderings(&existing_meets); - union_oeq + // We know have all the valid orderings after union, remove redundant + // entries (implicitly) and return: + result.add_new_orderings(&meets); + result } fn with_new_children( From 36ba8a906307c2f7f06c03aee72cadf72596f811 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Sun, 29 Oct 2023 17:06:09 +0300 Subject: [PATCH 107/122] Review Part 19 --- datafusion/physical-expr/src/equivalence.rs | 93 ++++++++----------- datafusion/physical-expr/src/partitioning.rs | 4 +- datafusion/physical-expr/src/utils.rs | 8 +- .../physical-expr/src/window/built_in.rs | 12 +-- 4 files changed, 49 insertions(+), 68 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 7292fce83370..2f86e16662bd 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -32,6 +32,9 @@ use arrow_schema::SortOptions; use datafusion_common::tree_node::{Transformed, TreeNode}; use datafusion_common::{JoinSide, JoinType, Result}; +use indexmap::map::Entry; +use indexmap::IndexMap; + /// An `EquivalenceClass` is a set of [`Arc`]s that are known /// to have the same value for all tuples in a relation. These are generated by /// equality predicates, typically equi-join conditions and equality conditions @@ -636,10 +639,7 @@ impl SchemaProperties { } /// Creates a new `SchemaProperties` object with the given orderings. - pub fn new_with_orderings( - schema: SchemaRef, - orderings: &[LexOrdering], - ) -> SchemaProperties { + pub fn new_with_orderings(schema: SchemaRef, orderings: &[LexOrdering]) -> Self { Self { eq_group: EquivalenceGroup::empty(), oeq_class: OrderingEquivalenceClass::new(orderings.to_vec()), @@ -955,23 +955,21 @@ impl SchemaProperties { &self, projection_mapping: &ProjectionMapping, output_schema: SchemaRef, - ) -> SchemaProperties { + ) -> Self { let mut projected_orderings = self .oeq_class .iter() .filter_map(|order| self.eq_group.project_ordering(projection_mapping, order)) .collect::>(); - for (source, target) in projection_mapping { - let initial_expr = ExprOrdering::new(source.clone()); - let transformed = initial_expr + let expr_ordering = ExprOrdering::new(source.clone()) .transform_up(&|expr| update_ordering(expr, self)) .unwrap(); - if let Some(SortProperties::Ordered(sort_options)) = transformed.state { + if let Some(SortProperties::Ordered(options)) = expr_ordering.state { // Push new ordering to the state. projected_orderings.push(vec![PhysicalSortExpr { expr: target.clone(), - options: sort_options, + options, }]); } } @@ -983,37 +981,30 @@ impl SchemaProperties { } } - /// Check whether any permutation of the argument has a prefix with existing ordering. - /// Return indices that describes ordering and their ordering information. + /// Checks whether any permutation of `exprs` shares a prefix with an + /// existing ordering. If so, returns the indices describing the permutation. + /// Otherwise, returns `None`. pub fn set_satisfy(&self, exprs: &[Arc]) -> Option> { - let ordered_section = self.get_lex_ordering_section(exprs); - // Get indices of the ordered_section inside exprs - ordered_section + self.get_lex_ordering_section(exprs) .into_iter() .map(|sort_expr| exprs.iter().position(|expr| sort_expr.expr.eq(expr))) .collect::>>() } - /// Check whether one of the permutation of the exprs satisfies existing ordering. - /// If so, return indices and their orderings. - /// None, indicates that there is no permutation that satisfies ordering. + /// Checks whether any permutation of `exprs` satisfies the existing + /// ordering. If so, returns the indices describing the permutation. + /// Otherwise, returns `None`. pub fn set_exactly_satisfy( &self, exprs: &[Arc], ) -> Option> { - if let Some(indices) = self.set_satisfy(exprs) { - // A permutation of the exprs satisfies one of the existing orderings. - if indices.len() == exprs.len() { - return Some(indices); - } - } - None + self.set_satisfy(exprs) + .filter(|indices| indices.len() == exprs.len()) } - /// Get ordering of the expressions in the argument - /// Assumes arguments define lexicographical ordering. - /// None, represents none of the existing ordering satisfy - /// lexicographical ordering of the exprs. + /// Assuming that `exprs` contains order-defining expressions, pairs each + /// of these expressions with its sort options according to the orderings + /// within. If the premise is false, returns `None`. pub fn get_lex_ordering( &self, exprs: &[Arc], @@ -1023,39 +1014,29 @@ impl SchemaProperties { (ordered_section.len() == exprs.len()).then_some(ordered_section) } - /// Get ordering of the expressions in the argument - /// If a subset of the arguments defines ordering, that section is returned - /// Even if not all of the expression are ordered. + /// Assuming that `exprs` contains order-defining expressions, pairs each + /// of these expressions with its sort options according to the orderings + /// within. Expressions that do not define an ordering are filtered out + /// and they are not present in the return value. pub fn get_lex_ordering_section( &self, exprs: &[Arc], ) -> LexOrdering { let normalized_exprs = self.eq_group.normalize_exprs(exprs.to_vec()); - // After normalization size of the vector shouldn't change. - assert_eq!(normalized_exprs.len(), exprs.len()); - // Stores the index and SortOption of the ordered expression. - let mut ordered_exprs: Vec<(usize, SortOptions)> = vec![]; + let mut ordered_exprs = IndexMap::::new(); for ordering in self.normalized_oeq_class().iter() { for sort_expr in ordering { - if let Some(idx) = normalized_exprs.iter().position(|normalized_expr| { - sort_expr.satisfy( - &PhysicalSortRequirement { - expr: normalized_expr.clone(), - options: None, - }, - &self.schema, - ) - }) { - // We did not insert this entry before. - if ordered_exprs - .iter() - .all(|(existing_idx, _opt)| *existing_idx != idx) - { - ordered_exprs.push((idx, sort_expr.options)); + if let Some(idx) = normalized_exprs + .iter() + .position(|expr| sort_expr.expr.eq(expr)) + { + if let Entry::Vacant(e) = ordered_exprs.entry(idx) { + e.insert(sort_expr.options); } } else { - // Should find in consecutive chunks + // We only consider expressions that correspond to a prefix + // of an ordering. break; } } @@ -1067,7 +1048,7 @@ impl SchemaProperties { expr: exprs[idx].clone(), options, }) - .collect::>() + .collect() } } @@ -1213,16 +1194,16 @@ fn update_ordering( /// Left, Full, Right joins). For other cases indices don't change. fn get_updated_right_ordering_equivalence_class( join_type: &JoinType, - mut right_oeq_group: OrderingEquivalenceClass, + mut right_oeq_class: OrderingEquivalenceClass, left_columns_len: usize, ) -> OrderingEquivalenceClass { if matches!( join_type, JoinType::Inner | JoinType::Left | JoinType::Full | JoinType::Right ) { - right_oeq_group.add_offset(left_columns_len); + right_oeq_class.add_offset(left_columns_len); } - right_oeq_group + right_oeq_class } #[cfg(test)] diff --git a/datafusion/physical-expr/src/partitioning.rs b/datafusion/physical-expr/src/partitioning.rs index 6ca299a341d0..164c78133998 100644 --- a/datafusion/physical-expr/src/partitioning.rs +++ b/datafusion/physical-expr/src/partitioning.rs @@ -82,8 +82,8 @@ impl Partitioning { // If the required exprs do not match, need to leverage the schema_properties provided by the child // and normalize both exprs based on the equivalent groups. if !fast_match { - let schema_properties = schema_properties(); - let eq_groups = schema_properties.eq_group(); + let eq_properties = schema_properties(); + let eq_groups = eq_properties.eq_group(); if !eq_groups.is_empty() { let normalized_required_exprs = required_exprs .iter() diff --git a/datafusion/physical-expr/src/utils.rs b/datafusion/physical-expr/src/utils.rs index 5db38749b72b..80853303f928 100644 --- a/datafusion/physical-expr/src/utils.rs +++ b/datafusion/physical-expr/src/utils.rs @@ -646,8 +646,8 @@ mod tests { let col_a = &col("a", &schema)?; let col_b = &col("b", &schema)?; let required_columns = [col_b.clone(), col_a.clone()]; - let mut ordering_equal_properties = SchemaProperties::new(Arc::new(schema)); - ordering_equal_properties.add_new_orderings(&[vec![ + let mut eq_properties = SchemaProperties::new(Arc::new(schema)); + eq_properties.add_new_orderings(&[vec![ PhysicalSortExpr { expr: Arc::new(Column::new("b", 1)), options: sort_options_not, @@ -658,12 +658,12 @@ mod tests { }, ]]); assert_eq!( - ordering_equal_properties.set_exactly_satisfy(&required_columns), + eq_properties.set_exactly_satisfy(&required_columns), Some(vec![0, 1]) ); assert_eq!( - ordering_equal_properties.get_lex_ordering(&required_columns), + eq_properties.get_lex_ordering(&required_columns), Some(vec![ PhysicalSortExpr { expr: col_b.clone(), diff --git a/datafusion/physical-expr/src/window/built_in.rs b/datafusion/physical-expr/src/window/built_in.rs index ce439e2b8cd1..a63844250937 100644 --- a/datafusion/physical-expr/src/window/built_in.rs +++ b/datafusion/physical-expr/src/window/built_in.rs @@ -72,12 +72,12 @@ impl BuiltInWindowExpr { /// If `self.expr` doesn't have an ordering, ordering equivalence properties /// are not updated. Otherwise, ordering equivalence properties are updated /// by the ordering of `self.expr`. - pub fn add_equal_orderings(&self, schema_properties: &mut SchemaProperties) { - let schema = schema_properties.schema(); + pub fn add_equal_orderings(&self, eq_properties: &mut SchemaProperties) { + let schema = eq_properties.schema(); if let Some(fn_res_ordering) = self.expr.get_result_ordering(schema) { if self.partition_by.is_empty() { // In the absence of a PARTITION BY, ordering of `self.expr` is global: - schema_properties.add_new_orderings(&[vec![fn_res_ordering]]); + eq_properties.add_new_orderings(&[vec![fn_res_ordering]]); } else { // If we have a PARTITION BY, built-in functions can not introduce // a global ordering unless the existing ordering is compatible @@ -86,17 +86,17 @@ impl BuiltInWindowExpr { // set equality), we can prefix the ordering of `self.expr` with // the existing ordering. if let Some(indices) = - schema_properties.set_exactly_satisfy(&self.partition_by) + eq_properties.set_exactly_satisfy(&self.partition_by) { let lex_partition_by = indices .into_iter() .map(|idx| self.partition_by[idx].clone()) .collect::>(); if let Some(mut ordering) = - schema_properties.get_lex_ordering(&lex_partition_by) + eq_properties.get_lex_ordering(&lex_partition_by) { ordering.push(fn_res_ordering); - schema_properties.add_new_orderings(&[ordering]); + eq_properties.add_new_orderings(&[ordering]); } } } From 5b47a2b6ed626c1af7120cfe758c6e087f1eb7b3 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Sun, 29 Oct 2023 18:05:14 +0300 Subject: [PATCH 108/122] Review Part 20 --- datafusion/physical-expr/src/equivalence.rs | 70 ++++++++++--------- datafusion/physical-expr/src/utils.rs | 35 +++++----- .../physical-expr/src/window/built_in.rs | 4 +- datafusion/physical-plan/src/filter.rs | 3 +- datafusion/physical-plan/src/streaming.rs | 2 +- datafusion/physical-plan/src/union.rs | 2 +- datafusion/physical-plan/src/windows/mod.rs | 8 +-- 7 files changed, 63 insertions(+), 61 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 2f86e16662bd..723fa7df8a49 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -217,7 +217,7 @@ impl EquivalenceGroup { /// in `exprs` and returns the corresponding normalized physical expressions. pub fn normalize_exprs( &self, - exprs: Vec>, + exprs: impl IntoIterator>, ) -> Vec> { exprs .into_iter() @@ -476,8 +476,11 @@ impl OrderingEquivalenceClass { } /// Adds new orderings into this ordering equivalence class. - pub fn add_new_orderings(&mut self, orderings: &[LexOrdering]) { - self.orderings.extend(orderings.iter().cloned()); + pub fn add_new_orderings( + &mut self, + orderings: impl IntoIterator, + ) { + self.orderings.extend(orderings); // Make sure that there are no redundant orderings: self.remove_redundant_entries(); } @@ -699,7 +702,10 @@ impl SchemaProperties { } /// Adds new orderings into the existing ordering equivalence class. - pub fn add_new_orderings(&mut self, orderings: &[LexOrdering]) { + pub fn add_new_orderings( + &mut self, + orderings: impl IntoIterator, + ) { self.oeq_class.add_new_orderings(orderings); } @@ -721,7 +727,10 @@ impl SchemaProperties { } /// Track/register physical expressions with constant values. - pub fn add_constants(mut self, constants: Vec>) -> Self { + pub fn add_constants( + mut self, + constants: impl IntoIterator>, + ) -> Self { for expr in self.eq_group.normalize_exprs(constants) { if !physical_exprs_contains(&self.constants, &expr) { self.constants.push(expr); @@ -1016,14 +1025,14 @@ impl SchemaProperties { /// Assuming that `exprs` contains order-defining expressions, pairs each /// of these expressions with its sort options according to the orderings - /// within. Expressions that do not define an ordering are filtered out - /// and they are not present in the return value. + /// within. Expressions that do not define a prefix of some ordering are + /// filtered out and they are not present in the return value. pub fn get_lex_ordering_section( &self, exprs: &[Arc], ) -> LexOrdering { let normalized_exprs = self.eq_group.normalize_exprs(exprs.to_vec()); - // Stores the index and SortOption of the ordered expression. + // Use a map to associate expression indices with sort options: let mut ordered_exprs = IndexMap::::new(); for ordering in self.normalized_oeq_class().iter() { for sort_expr in ordering { @@ -1036,12 +1045,12 @@ impl SchemaProperties { } } else { // We only consider expressions that correspond to a prefix - // of an ordering. + // of one of the equivalent orderings we have. break; } } } - // Construct ordered section by getting entries at indices of ordered_exprs (first entry in the tuple) + // Construct the lexicographical ordering according to the fpermutation: ordered_exprs .into_iter() .map(|(idx, options)| PhysicalSortExpr { @@ -1134,12 +1143,9 @@ pub fn join_schema_properties( /// duplicate entries that have same physical expression inside. For example, /// `vec![a Some(Asc), a Some(Desc)]` collapses to `vec![a Some(Asc)]`. pub fn collapse_lex_req(input: LexRequirement) -> LexRequirement { - let mut output = vec![]; + let mut output = Vec::::new(); for item in input { - if !output - .iter() - .any(|req: &PhysicalSortRequirement| req.expr.eq(&item.expr)) - { + if !output.iter().any(|req| req.expr.eq(&item.expr)) { output.push(item); } } @@ -1254,8 +1260,8 @@ mod tests { let col_e = &col("e", &test_schema)?; let col_f = &col("f", &test_schema)?; let col_g = &col("g", &test_schema)?; - let mut schema_properties = SchemaProperties::new(test_schema.clone()); - schema_properties.add_equal_conditions(col_a, col_c); + let mut eq_properties = SchemaProperties::new(test_schema.clone()); + eq_properties.add_equal_conditions(col_a, col_c); let option_asc = SortOptions { descending: false, @@ -1278,8 +1284,8 @@ mod tests { ], ]; let orderings = convert_to_orderings(&orderings); - schema_properties.add_new_orderings(&orderings); - Ok((test_schema, schema_properties)) + eq_properties.add_new_orderings(orderings); + Ok((test_schema, eq_properties)) } // Generate a schema which consists of 6 columns (a, b, c, d, e, f) @@ -1310,11 +1316,11 @@ mod tests { let col_f = &col("f", &test_schema)?; let col_exprs = [col_a, col_b, col_c, col_d, col_e, col_f]; - let mut schema_properties = SchemaProperties::new(test_schema.clone()); + let mut eq_properties = SchemaProperties::new(test_schema.clone()); // Define a and f are aliases - schema_properties.add_equal_conditions(col_a, col_f); + eq_properties.add_equal_conditions(col_a, col_f); // Column e has constant value. - schema_properties = schema_properties.add_constants(vec![col_e.clone()]); + eq_properties = eq_properties.add_constants([col_e.clone()]); // Randomly order columns for sorting let mut rng = StdRng::seed_from_u64(seed); @@ -1329,7 +1335,7 @@ mod tests { let n_sort_expr = rng.gen_range(0..remaining_exprs.len() + 1); remaining_exprs.shuffle(&mut rng); - let orderings: Vec<_> = remaining_exprs + let ordering = remaining_exprs .drain(0..n_sort_expr) .map(|expr| PhysicalSortExpr { expr: expr.clone(), @@ -1337,10 +1343,10 @@ mod tests { }) .collect(); - schema_properties.add_new_orderings(&[orderings]); + eq_properties.add_new_orderings([ordering]); } - Ok((test_schema, schema_properties)) + Ok((test_schema, eq_properties)) } // Convert each tuple to PhysicalSortRequirement @@ -1750,8 +1756,8 @@ mod tests { nulls_first: false, }; // a=c (e.g they are aliases). - let mut schema_properties = SchemaProperties::new(test_schema); - schema_properties.add_equal_conditions(col_a, col_c); + let mut eq_properties = SchemaProperties::new(test_schema); + eq_properties.add_equal_conditions(col_a, col_c); let orderings = vec![ vec![(col_a, options)], @@ -1761,7 +1767,7 @@ mod tests { let orderings = convert_to_orderings(&orderings); // Column [a ASC], [e ASC], [d ASC, f ASC] are all valid orderings for the schema. - schema_properties.add_new_orderings(&orderings); + eq_properties.add_new_orderings(orderings); // First entry in the tuple is required ordering, second entry is the expected flag // that indicates whether this required ordering is satisfied. @@ -1786,7 +1792,7 @@ mod tests { format!("error in test reqs: {:?}, expected: {:?}", reqs, expected,); let reqs = convert_to_sort_exprs(&reqs); assert_eq!( - schema_properties.ordering_satisfy(&reqs), + eq_properties.ordering_satisfy(&reqs), expected, "{}", err_msg @@ -2515,7 +2521,7 @@ mod tests { // and // Column [a=c] (e.g they are aliases). // At below we add [d ASC, h DESC] also, for test purposes - let (test_schema, mut schema_props) = create_test_params()?; + let (test_schema, mut eq_properties) = create_test_params()?; let col_a = &col("a", &test_schema)?; let col_b = &col("b", &test_schema)?; let col_c = &col("c", &test_schema)?; @@ -2532,7 +2538,7 @@ mod tests { nulls_first: true, }; // [d ASC, h ASC] also satisfies schema. - schema_props.add_new_orderings(&[vec![ + eq_properties.add_new_orderings([vec![ PhysicalSortExpr { expr: col_d.clone(), options: option_asc, @@ -2564,7 +2570,7 @@ mod tests { for (exprs, expected) in test_cases { let exprs = exprs.into_iter().cloned().collect::>(); let expected = convert_to_sort_exprs(&expected); - let actual = schema_props.get_lex_ordering_section(&exprs); + let actual = eq_properties.get_lex_ordering_section(&exprs); assert_eq!(actual, expected); } diff --git a/datafusion/physical-expr/src/utils.rs b/datafusion/physical-expr/src/utils.rs index 80853303f928..28b2465325c4 100644 --- a/datafusion/physical-expr/src/utils.rs +++ b/datafusion/physical-expr/src/utils.rs @@ -647,7 +647,7 @@ mod tests { let col_b = &col("b", &schema)?; let required_columns = [col_b.clone(), col_a.clone()]; let mut eq_properties = SchemaProperties::new(Arc::new(schema)); - eq_properties.add_new_orderings(&[vec![ + eq_properties.add_new_orderings([vec![ PhysicalSortExpr { expr: Arc::new(Column::new("b", 1)), options: sort_options_not, @@ -684,8 +684,8 @@ mod tests { let col_a = &col("a", &schema)?; let col_b = &col("b", &schema)?; let required_columns = [col_b.clone(), col_a.clone()]; - let mut ordering_equal_properties = SchemaProperties::new(Arc::new(schema)); - ordering_equal_properties.add_new_orderings(&[ + let mut eq_properties = SchemaProperties::new(Arc::new(schema)); + eq_properties.add_new_orderings([ vec![PhysicalSortExpr { expr: Arc::new(Column::new("c", 2)), options: sort_options, @@ -702,12 +702,12 @@ mod tests { ], ]); assert_eq!( - ordering_equal_properties.set_exactly_satisfy(&required_columns), + eq_properties.set_exactly_satisfy(&required_columns), Some(vec![0, 1]) ); assert_eq!( - ordering_equal_properties.get_lex_ordering(&required_columns), + eq_properties.get_lex_ordering(&required_columns), Some(vec![ PhysicalSortExpr { expr: col_b.clone(), @@ -729,10 +729,10 @@ mod tests { Field::new("b", DataType::Int32, true), Field::new("c", DataType::Int32, true), ]); - let mut ordering_equal_properties = SchemaProperties::new(Arc::new(schema)); + let mut eq_properties = SchemaProperties::new(Arc::new(schema)); // not satisfied orders - ordering_equal_properties.add_new_orderings(&[vec![ + eq_properties.add_new_orderings([vec![ PhysicalSortExpr { expr: Arc::new(Column::new("b", 1)), options: sort_options_not, @@ -746,10 +746,7 @@ mod tests { options: sort_options, }, ]]); - assert_eq!( - ordering_equal_properties.set_exactly_satisfy(&required_columns), - None - ); + assert_eq!(eq_properties.set_exactly_satisfy(&required_columns), None); Ok(()) } @@ -766,9 +763,9 @@ mod tests { let col_a_expr = col("a", &schema)?; let col_b_expr = col("b", &schema)?; let col_c_expr = col("c", &schema)?; - let mut schema_properties = SchemaProperties::new(Arc::new(schema.clone())); + let mut eq_properties = SchemaProperties::new(Arc::new(schema.clone())); - schema_properties.add_equal_conditions(&col_a_expr, &col_c_expr); + eq_properties.add_equal_conditions(&col_a_expr, &col_c_expr); let others = vec![ vec![PhysicalSortExpr { expr: col_b_expr.clone(), @@ -779,10 +776,10 @@ mod tests { options: sort_options, }], ]; - schema_properties.add_new_orderings(&others); + eq_properties.add_new_orderings(others); let mut expected_eqs = SchemaProperties::new(Arc::new(schema)); - expected_eqs.add_new_orderings(&[ + expected_eqs.add_new_orderings([ vec![PhysicalSortExpr { expr: col_b_expr.clone(), options: sort_options, @@ -793,7 +790,7 @@ mod tests { }], ]); - let oeq_class = schema_properties.oeq_class().clone(); + let oeq_class = eq_properties.oeq_class().clone(); let expected = expected_eqs.oeq_class(); assert!(oeq_class.eq(expected)); @@ -807,12 +804,12 @@ mod tests { Field::new("b", DataType::Int32, true), Field::new("c", DataType::Int32, true), ]); - let mut schema_properties = SchemaProperties::new(Arc::new(schema.clone())); + let mut eq_properties = SchemaProperties::new(Arc::new(schema.clone())); let ordering = vec![PhysicalSortExpr { expr: Arc::new(Column::new("b", 1)), options: SortOptions::default(), }]; - schema_properties.add_new_orderings(&[ordering]); + eq_properties.add_new_orderings([ordering]); let projection_mapping = vec![ ( Arc::new(Column::new("b", 1)) as _, @@ -827,7 +824,7 @@ mod tests { Field::new("b_new", DataType::Int32, true), Field::new("a_new", DataType::Int32, true), ])); - let orderings = schema_properties + let orderings = eq_properties .project(&projection_mapping, projection_schema) .oeq_class() .output_ordering() diff --git a/datafusion/physical-expr/src/window/built_in.rs b/datafusion/physical-expr/src/window/built_in.rs index a63844250937..dd3bfc0558ba 100644 --- a/datafusion/physical-expr/src/window/built_in.rs +++ b/datafusion/physical-expr/src/window/built_in.rs @@ -77,7 +77,7 @@ impl BuiltInWindowExpr { if let Some(fn_res_ordering) = self.expr.get_result_ordering(schema) { if self.partition_by.is_empty() { // In the absence of a PARTITION BY, ordering of `self.expr` is global: - eq_properties.add_new_orderings(&[vec![fn_res_ordering]]); + eq_properties.add_new_orderings([vec![fn_res_ordering]]); } else { // If we have a PARTITION BY, built-in functions can not introduce // a global ordering unless the existing ordering is compatible @@ -96,7 +96,7 @@ impl BuiltInWindowExpr { eq_properties.get_lex_ordering(&lex_partition_by) { ordering.push(fn_res_ordering); - eq_properties.add_new_orderings(&[ordering]); + eq_properties.add_new_orderings([ordering]); } } } diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index c4786912cafa..656171d2aa84 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -158,8 +158,7 @@ impl ExecutionPlan for FilterExec { let constants = collect_columns(self.predicate()) .into_iter() .filter(|column| stats.column_statistics[column.index()].is_singleton()) - .map(|column| Arc::new(column) as _) - .collect(); + .map(|column| Arc::new(column) as _); result.add_constants(constants) } diff --git a/datafusion/physical-plan/src/streaming.rs b/datafusion/physical-plan/src/streaming.rs index 1b403978f3b4..85689ef3a0b9 100644 --- a/datafusion/physical-plan/src/streaming.rs +++ b/datafusion/physical-plan/src/streaming.rs @@ -159,7 +159,7 @@ impl ExecutionPlan for StreamingTableExec { fn schema_properties(&self) -> SchemaProperties { let mut result = SchemaProperties::new(self.schema()); if let Some(ordering) = &self.projected_output_ordering { - result.add_new_orderings(&[ordering.clone()]) + result.add_new_orderings([ordering.clone()]) } result } diff --git a/datafusion/physical-plan/src/union.rs b/datafusion/physical-plan/src/union.rs index ebbcdeb540a6..de8832f92569 100644 --- a/datafusion/physical-plan/src/union.rs +++ b/datafusion/physical-plan/src/union.rs @@ -252,7 +252,7 @@ impl ExecutionPlan for UnionExec { } // We know have all the valid orderings after union, remove redundant // entries (implicitly) and return: - result.add_new_orderings(&meets); + result.add_new_orderings(meets); result } diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index 06fe724238fc..3c24ca6497a2 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -465,12 +465,12 @@ pub fn get_window_mode( orderby_keys: &[PhysicalSortExpr], input: &Arc, ) -> Result> { - let input_oeq = input.schema_properties(); + let input_eqs = input.schema_properties(); let mut partition_search_mode = PartitionSearchMode::Linear; let mut partition_by_reqs: Vec = vec![]; if partitionby_exprs.is_empty() { partition_search_mode = PartitionSearchMode::Sorted; - } else if let Some(indices) = input_oeq.set_satisfy(partitionby_exprs) { + } else if let Some(indices) = input_eqs.set_satisfy(partitionby_exprs) { let item = indices .iter() .map(|&idx| PhysicalSortRequirement { @@ -487,7 +487,7 @@ pub fn get_window_mode( } // Treat partition by exprs as constant. During analysis of requirements are satisfied. - let partition_by_oeq = input_oeq.add_constants(partitionby_exprs.to_vec()); + let partition_by_eqs = input_eqs.add_constants(partitionby_exprs.iter().cloned()); let order_by_reqs = PhysicalSortRequirement::from_sort_exprs(orderby_keys); let reverse_order_by_reqs = PhysicalSortRequirement::from_sort_exprs(&reverse_order_bys(orderby_keys)); @@ -496,7 +496,7 @@ pub fn get_window_mode( { let req = [partition_by_reqs.clone(), order_by_reqs].concat(); let req = collapse_lex_req(req); - if partition_by_oeq.ordering_satisfy_requirement(&req) { + if partition_by_eqs.ordering_satisfy_requirement(&req) { // Window can be run with existing ordering return Ok(Some((should_swap, partition_search_mode))); } From ace5bebb893735a41e057b541eb36620cc79d05d Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Sun, 29 Oct 2023 18:55:50 +0300 Subject: [PATCH 109/122] Review Part 21 --- datafusion/physical-expr/src/equivalence.rs | 88 +++++++++------------ 1 file changed, 38 insertions(+), 50 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 723fa7df8a49..ad0a425005ea 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -364,39 +364,34 @@ impl EquivalenceGroup { left_column_count: usize, on: &[(Column, Column)], ) -> Self { - let mut result = Self::empty(); match join_type { JoinType::Inner | JoinType::Left | JoinType::Full | JoinType::Right => { - result.extend(self.clone()); - let updated_eq_classes = right_equivalences - .iter() - .map(|cls| { - cls.iter() - .cloned() - .map(|item| add_offset_to_expr(item, left_column_count)) - .collect() - }) - .collect(); - result.extend(Self::new(updated_eq_classes)); - } - JoinType::LeftSemi | JoinType::LeftAnti => { - result.extend(self.clone()); - } - JoinType::RightSemi | JoinType::RightAnti => { - result.extend(right_equivalences.clone()); - } - } - // In we have an inner join, expressions in the "on" condition are equal - // at the resulting table. - if join_type == &JoinType::Inner { - for (lhs, rhs) in on.iter() { - let index = rhs.index() + left_column_count; - let new_lhs = Arc::new(lhs.clone()) as _; - let new_rhs = Arc::new(Column::new(rhs.name(), index)) as _; - result.add_equal_conditions(&new_lhs, &new_rhs); + let mut result = Self::new( + self.iter() + .cloned() + .chain(right_equivalences.iter().map(|item| { + item.iter() + .cloned() + .map(|expr| add_offset_to_expr(expr, left_column_count)) + .collect() + })) + .collect(), + ); + // In we have an inner join, expressions in the "on" condition + // are equal in the resulting table. + if join_type == &JoinType::Inner { + for (lhs, rhs) in on.iter() { + let index = rhs.index() + left_column_count; + let new_lhs = Arc::new(lhs.clone()) as _; + let new_rhs = Arc::new(Column::new(rhs.name(), index)) as _; + result.add_equal_conditions(&new_lhs, &new_rhs); + } + } + result } + JoinType::LeftSemi | JoinType::LeftAnti => self.clone(), + JoinType::RightSemi | JoinType::RightAnti => right_equivalences.clone(), } - result } } @@ -442,6 +437,7 @@ impl OrderingEquivalenceClass { } /// Adds `ordering` to this equivalence class. + #[allow(dead_code)] fn push(&mut self, ordering: LexOrdering) { self.orderings.push(ordering); // Make sure that there are no redundant orderings: @@ -458,11 +454,6 @@ impl OrderingEquivalenceClass { self.orderings.iter() } - /// Returns an iterator over the equivalent orderings in this class. - fn into_iter(self) -> impl Iterator { - self.orderings.into_iter() - } - /// Returns how many equivalent orderings there are in this class. pub fn len(&self) -> usize { self.orderings.len() @@ -694,11 +685,7 @@ impl SchemaProperties { /// Extends this `SchemaProperties` by adding the orderings inside the /// ordering equivalence class `other`. pub fn add_ordering_equivalence_class(&mut self, other: OrderingEquivalenceClass) { - for ordering in other.into_iter() { - if !self.oeq_class.contains(&ordering) { - self.oeq_class.push(ordering); - } - } + self.oeq_class.extend(other); } /// Adds new orderings into the existing ordering equivalence class. @@ -1072,12 +1059,13 @@ pub fn join_schema_properties( on: &[(Column, Column)], ) -> SchemaProperties { let left_columns_len = left.schema.fields.len(); - let mut new_properties = SchemaProperties::new(join_schema); - - let join_eq_groups = - left.eq_group() - .join(join_type, right.eq_group(), left_columns_len, on); - new_properties.add_equivalence_group(join_eq_groups); + let mut result = SchemaProperties::new(join_schema); + result.add_equivalence_group(left.eq_group().join( + join_type, + right.eq_group(), + left_columns_len, + on, + )); // All joins have 2 children assert_eq!(maintains_input_order.len(), 2); @@ -1107,9 +1095,9 @@ pub fn join_schema_properties( // for the right table should be converted to `a ASC, b ASC` before it is added // to the ordering equivalences of the join. let out_oeq_class = left_oeq_class.join_suffix(&updated_right_oeq); - new_properties.add_ordering_equivalence_class(out_oeq_class); + result.add_ordering_equivalence_class(out_oeq_class); } else { - new_properties.add_ordering_equivalence_class(left_oeq_class); + result.add_ordering_equivalence_class(left_oeq_class); } } (false, true) => { @@ -1129,14 +1117,14 @@ pub fn join_schema_properties( // for the right table should be converted to `a ASC, b ASC` before it is added // to the ordering equivalences of the join. let out_oeq_class = updated_right_oeq.join_suffix(&left_oeq_class); - new_properties.add_ordering_equivalence_class(out_oeq_class); + result.add_ordering_equivalence_class(out_oeq_class); } else { - new_properties.add_ordering_equivalence_class(updated_right_oeq); + result.add_ordering_equivalence_class(updated_right_oeq); } } (false, false) => {} } - new_properties + result } /// This function constructs a duplicate-free `LexOrderingReq` by filtering out From fa9850c3d589b699f64682488a182b896cf9ee7d Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Sun, 29 Oct 2023 19:28:25 +0300 Subject: [PATCH 110/122] Review Part 22 --- datafusion/physical-expr/src/equivalence.rs | 17 ++++++----------- datafusion/physical-plan/src/joins/utils.rs | 17 ++++++----------- 2 files changed, 12 insertions(+), 22 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index ad0a425005ea..955e165a2bed 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -1067,17 +1067,10 @@ pub fn join_schema_properties( on, )); - // All joins have 2 children - assert_eq!(maintains_input_order.len(), 2); - let left_maintains = maintains_input_order[0]; - let right_maintains = maintains_input_order[1]; let left_oeq_class = left.oeq_class; let right_oeq_class = right.oeq_class; - match (left_maintains, right_maintains) { - (true, true) => { - unreachable!("Cannot maintain ordering of both sides"); - } - (true, false) => { + match maintains_input_order { + [true, false] => { // In this special case, right side ordering can be prefixed with left side ordering. if let (Some(JoinSide::Left), JoinType::Inner) = (probe_side, join_type) { let updated_right_oeq = get_updated_right_ordering_equivalence_class( @@ -1100,7 +1093,7 @@ pub fn join_schema_properties( result.add_ordering_equivalence_class(left_oeq_class); } } - (false, true) => { + [false, true] => { let updated_right_oeq = get_updated_right_ordering_equivalence_class( join_type, right_oeq_class, @@ -1122,7 +1115,9 @@ pub fn join_schema_properties( result.add_ordering_equivalence_class(updated_right_oeq); } } - (false, false) => {} + [false, false] => {} + [true, true] => unreachable!("Cannot maintain ordering of both sides"), + _ => unreachable!("Join operators can not have more than two children"), } result } diff --git a/datafusion/physical-plan/src/joins/utils.rs b/datafusion/physical-plan/src/joins/utils.rs index 0cbf8b6b20ce..c91dc92fbc7a 100644 --- a/datafusion/physical-plan/src/joins/utils.rs +++ b/datafusion/physical-plan/src/joins/utils.rs @@ -169,10 +169,6 @@ pub fn calculate_join_output_ordering( maintains_input_order: &[bool], probe_side: Option, ) -> Option { - // All joins have 2 children: - assert_eq!(maintains_input_order.len(), 2); - let left_maintains = maintains_input_order[0]; - let right_maintains = maintains_input_order[1]; let mut right_ordering = match join_type { // In the case below, right ordering should be offseted with the left // side length, since we append the right table to the left table. @@ -187,11 +183,8 @@ pub fn calculate_join_output_ordering( } _ => right_ordering.to_vec(), }; - let output_ordering = match (left_maintains, right_maintains) { - (true, true) => { - unreachable!("Cannot maintain ordering of both sides"); - } - (true, false) => { + let output_ordering = match maintains_input_order { + [true, false] => { // Special case, we can prefix ordering of right side with the ordering of left side. if join_type == JoinType::Inner && probe_side == Some(JoinSide::Left) { replace_on_columns_of_right_ordering( @@ -204,7 +197,7 @@ pub fn calculate_join_output_ordering( left_ordering.to_vec() } } - (false, true) => { + [false, true] => { // Special case, we can prefix ordering of left side with the ordering of right side. if join_type == JoinType::Inner && probe_side == Some(JoinSide::Right) { replace_on_columns_of_right_ordering( @@ -218,7 +211,9 @@ pub fn calculate_join_output_ordering( } } // Doesn't maintain ordering, output ordering is None. - (false, false) => return None, + [false, false] => return None, + [true, true] => unreachable!("Cannot maintain ordering of both sides"), + _ => unreachable!("Join operators can not have more than two children"), }; (!output_ordering.is_empty()).then_some(output_ordering) } From 3134ee4b1854d1f6c6f2b490631e6eb61fd7d90e Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Sun, 29 Oct 2023 22:02:50 +0300 Subject: [PATCH 111/122] Review Part 23 --- datafusion/physical-expr/src/equivalence.rs | 137 ++++++++++---------- 1 file changed, 70 insertions(+), 67 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 955e165a2bed..db3e065894d1 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -359,9 +359,9 @@ impl EquivalenceGroup { /// Combine equivalence groups of the given join children. pub fn join( &self, + right_equivalences: &Self, join_type: &JoinType, - right_equivalences: &Self, - left_column_count: usize, + left_size: usize, on: &[(Column, Column)], ) -> Self { match join_type { @@ -372,7 +372,7 @@ impl EquivalenceGroup { .chain(right_equivalences.iter().map(|item| { item.iter() .cloned() - .map(|expr| add_offset_to_expr(expr, left_column_count)) + .map(|expr| add_offset_to_expr(expr, left_size)) .collect() })) .collect(), @@ -381,7 +381,7 @@ impl EquivalenceGroup { // are equal in the resulting table. if join_type == &JoinType::Inner { for (lhs, rhs) in on.iter() { - let index = rhs.index() + left_column_count; + let index = rhs.index() + left_size; let new_lhs = Arc::new(lhs.clone()) as _; let new_rhs = Arc::new(Column::new(rhs.name(), index)) as _; result.add_equal_conditions(&new_lhs, &new_rhs); @@ -395,6 +395,19 @@ impl EquivalenceGroup { } } +/// This function constructs a duplicate-free `LexOrderingReq` by filtering out +/// duplicate entries that have same physical expression inside. For example, +/// `vec![a Some(Asc), a Some(Desc)]` collapses to `vec![a Some(Asc)]`. +pub fn collapse_lex_req(input: LexRequirement) -> LexRequirement { + let mut output = Vec::::new(); + for item in input { + if !output.iter().any(|req| req.expr.eq(&item.expr)) { + output.push(item); + } + } + output +} + /// An `OrderingEquivalenceClass` object keeps track of different alternative /// orderings than can describe a schema. For example, consider the following table: /// @@ -1058,61 +1071,63 @@ pub fn join_schema_properties( probe_side: Option, on: &[(Column, Column)], ) -> SchemaProperties { - let left_columns_len = left.schema.fields.len(); + let left_size = left.schema.fields.len(); let mut result = SchemaProperties::new(join_schema); result.add_equivalence_group(left.eq_group().join( + right.eq_group(), join_type, - right.eq_group(), - left_columns_len, + left_size, on, )); let left_oeq_class = left.oeq_class; - let right_oeq_class = right.oeq_class; + let mut right_oeq_class = right.oeq_class; match maintains_input_order { [true, false] => { - // In this special case, right side ordering can be prefixed with left side ordering. + // In this special case, right side ordering can be prefixed with + // the left side ordering. if let (Some(JoinSide::Left), JoinType::Inner) = (probe_side, join_type) { - let updated_right_oeq = get_updated_right_ordering_equivalence_class( + updated_right_ordering_equivalence_class( + &mut right_oeq_class, join_type, - right_oeq_class, - left_columns_len, + left_size, ); - // Right side ordering equivalence properties should be prepended with - // those of the left side while constructing output ordering equivalence - // properties since stream side is the left side. + // Right side ordering equivalence properties should be prepended + // with those of the left side while constructing output ordering + // equivalence properties since stream side is the left side. // - // If the right table ordering equivalences contain `b ASC`, and the output - // ordering of the left table is `a ASC`, then the ordering equivalence `b ASC` - // for the right table should be converted to `a ASC, b ASC` before it is added - // to the ordering equivalences of the join. - let out_oeq_class = left_oeq_class.join_suffix(&updated_right_oeq); + // For example, if the right side ordering equivalences contain + // `b ASC`, and the output ordering of the left side is `a ASC`, + // then we should add `a ASC, b ASC` to the ordering equivalences + // of the join output. + let out_oeq_class = left_oeq_class.join_suffix(&right_oeq_class); result.add_ordering_equivalence_class(out_oeq_class); } else { result.add_ordering_equivalence_class(left_oeq_class); } } [false, true] => { - let updated_right_oeq = get_updated_right_ordering_equivalence_class( + updated_right_ordering_equivalence_class( + &mut right_oeq_class, join_type, - right_oeq_class, - left_columns_len, + left_size, ); - // In this special case, left side ordering can be prefixed with right side ordering. + // In this special case, left side ordering can be prefixed with + // the right side ordering. if let (Some(JoinSide::Right), JoinType::Inner) = (probe_side, join_type) { - // Left side ordering equivalence properties should be prepended with - // those of the right side while constructing output ordering equivalence - // properties since stream side is the right side. + // Left side ordering equivalence properties should be prepended + // with those of the right side while constructing output ordering + // equivalence properties since stream side is the right side. // - // If the right table ordering equivalences contain `b ASC`, and the output - // ordering of the left table is `a ASC`, then the ordering equivalence `b ASC` - // for the right table should be converted to `a ASC, b ASC` before it is added - // to the ordering equivalences of the join. - let out_oeq_class = updated_right_oeq.join_suffix(&left_oeq_class); + // For example, if the right side ordering equivalences contain + // `b ASC`, and the output ordering of the left table is `a ASC`, + // then we should add `a ASC, b ASC` to the ordering equivalences + // of the join output. + let out_oeq_class = right_oeq_class.join_suffix(&left_oeq_class); result.add_ordering_equivalence_class(out_oeq_class); } else { - result.add_ordering_equivalence_class(updated_right_oeq); + result.add_ordering_equivalence_class(right_oeq_class); } } [false, false] => {} @@ -1122,17 +1137,24 @@ pub fn join_schema_properties( result } -/// This function constructs a duplicate-free `LexOrderingReq` by filtering out -/// duplicate entries that have same physical expression inside. For example, -/// `vec![a Some(Asc), a Some(Desc)]` collapses to `vec![a Some(Asc)]`. -pub fn collapse_lex_req(input: LexRequirement) -> LexRequirement { - let mut output = Vec::::new(); - for item in input { - if !output.iter().any(|req| req.expr.eq(&item.expr)) { - output.push(item); - } +/// In the context of a join, update the right side `OrderingEquivalenceClass` +/// so that they point to valid indices in the join output schema. +/// +/// To do so, we increment column indices by the size of the left table when +/// join schema consists of a combination of the left and right schemas. This +/// is the case for `Inner`, `Left`, `Full` and `Right` joins. For other cases, +/// indices do not change. +fn updated_right_ordering_equivalence_class( + right_oeq_class: &mut OrderingEquivalenceClass, + join_type: &JoinType, + left_size: usize, +) { + if matches!( + join_type, + JoinType::Inner | JoinType::Left | JoinType::Full | JoinType::Right + ) { + right_oeq_class.add_offset(left_size); } - output } /// Calculates the [`SortProperties`] of a given [`ExprOrdering`] node. @@ -1175,26 +1197,6 @@ fn update_ordering( } } -/// Update right table ordering `OrderingEquivalenceClass`es so that: -/// - They point to valid indices at the output of the join schema -/// -/// To do so, we increment column indices by the size of the left table when -/// join schema consists of a combination of left and right schema (Inner, -/// Left, Full, Right joins). For other cases indices don't change. -fn get_updated_right_ordering_equivalence_class( - join_type: &JoinType, - mut right_oeq_class: OrderingEquivalenceClass, - left_columns_len: usize, -) -> OrderingEquivalenceClass { - if matches!( - join_type, - JoinType::Inner | JoinType::Left | JoinType::Full | JoinType::Right - ) { - right_oeq_class.add_offset(left_columns_len); - } - right_oeq_class -} - #[cfg(test)] mod tests { use std::sync::Arc; @@ -1995,7 +1997,7 @@ mod tests { ]; let orderings = convert_to_orderings(&orderings); // Right child ordering equivalences - let right_oeq_class = OrderingEquivalenceClass::new(orderings); + let mut right_oeq_class = OrderingEquivalenceClass::new(orderings); let left_columns_len = 4; @@ -2018,12 +2020,12 @@ mod tests { join_schema_properties.add_equal_conditions(col_a, col_x); join_schema_properties.add_equal_conditions(col_d, col_w); - let result = get_updated_right_ordering_equivalence_class( + updated_right_ordering_equivalence_class( + &mut right_oeq_class, &join_type, - right_oeq_class, left_columns_len, ); - join_schema_properties.add_ordering_equivalence_class(result); + join_schema_properties.add_ordering_equivalence_class(right_oeq_class); let result = join_schema_properties.oeq_class().clone(); // [x ASC, y ASC], [z ASC, w ASC] @@ -2340,6 +2342,7 @@ mod tests { expected_normalized ); } + Ok(()) } From 174c97309438663a54b765554a893770a1a1ef9e Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Sun, 29 Oct 2023 22:56:14 +0300 Subject: [PATCH 112/122] Review Part 24 --- datafusion/physical-expr/src/equivalence.rs | 22 ++++++++++----------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index db3e065894d1..636a11ad5780 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -359,7 +359,7 @@ impl EquivalenceGroup { /// Combine equivalence groups of the given join children. pub fn join( &self, - right_equivalences: &Self, + right_equivalences: &Self, join_type: &JoinType, left_size: usize, on: &[(Column, Column)], @@ -997,7 +997,7 @@ impl SchemaProperties { self.get_lex_ordering_section(exprs) .into_iter() .map(|sort_expr| exprs.iter().position(|expr| sort_expr.expr.eq(expr))) - .collect::>>() + .collect() } /// Checks whether any permutation of `exprs` satisfies the existing @@ -1074,7 +1074,7 @@ pub fn join_schema_properties( let left_size = left.schema.fields.len(); let mut result = SchemaProperties::new(join_schema); result.add_equivalence_group(left.eq_group().join( - right.eq_group(), + right.eq_group(), join_type, left_size, on, @@ -1088,7 +1088,7 @@ pub fn join_schema_properties( // the left side ordering. if let (Some(JoinSide::Left), JoinType::Inner) = (probe_side, join_type) { updated_right_ordering_equivalence_class( - &mut right_oeq_class, + &mut right_oeq_class, join_type, left_size, ); @@ -1109,7 +1109,7 @@ pub fn join_schema_properties( } [false, true] => { updated_right_ordering_equivalence_class( - &mut right_oeq_class, + &mut right_oeq_class, join_type, left_size, ); @@ -1145,7 +1145,7 @@ pub fn join_schema_properties( /// is the case for `Inner`, `Left`, `Full` and `Right` joins. For other cases, /// indices do not change. fn updated_right_ordering_equivalence_class( - right_oeq_class: &mut OrderingEquivalenceClass, + right_oeq_class: &mut OrderingEquivalenceClass, join_type: &JoinType, left_size: usize, ) { @@ -1158,7 +1158,7 @@ fn updated_right_ordering_equivalence_class( } /// Calculates the [`SortProperties`] of a given [`ExprOrdering`] node. -/// The node is either a leaf node, or an intermediate node: +/// The node can either be a leaf node, or an intermediate node: /// - If it is a leaf node, the children states are `None`. We directly find /// the order of the node by looking at the given sort expression and equivalence /// properties if it is a `Column` leaf, or we mark it as unordered. In the case @@ -1172,7 +1172,7 @@ fn updated_right_ordering_equivalence_class( /// the order coming from the children. fn update_ordering( mut node: ExprOrdering, - ordering_equal_properties: &SchemaProperties, + eq_properties: &SchemaProperties, ) -> Result> { if let Some(children_sort_options) = &node.children_states { // We have an intermediate (non-leaf) node, account for its children: @@ -1180,9 +1180,9 @@ fn update_ordering( Ok(Transformed::Yes(node)) } else if node.expr.as_any().is::() { // We have a Column, which is one of the two possible leaf node types: - let eq_group = &ordering_equal_properties.eq_group; + let eq_group = &eq_properties.eq_group; let normalized_expr = eq_group.normalize_expr(node.expr.clone()); - let oeq_class = &ordering_equal_properties.oeq_class; + let oeq_class = &eq_properties.oeq_class; if let Some(options) = oeq_class.get_options(&normalized_expr) { node.state = Some(SortProperties::Ordered(options)); Ok(Transformed::Yes(node)) @@ -2021,7 +2021,7 @@ mod tests { join_schema_properties.add_equal_conditions(col_d, col_w); updated_right_ordering_equivalence_class( - &mut right_oeq_class, + &mut right_oeq_class, &join_type, left_columns_len, ); From 2c562c3380be75a81cbdb2a58731b6abe4261dd4 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Mon, 30 Oct 2023 12:50:32 +0300 Subject: [PATCH 113/122] Do not construct idx and sort_expr unnecessarily, Update comments, Union meet single entry --- datafusion/physical-expr/src/equivalence.rs | 179 +++++++++--------- datafusion/physical-expr/src/utils.rs | 35 ++-- .../physical-expr/src/window/built_in.rs | 17 +- .../physical-plan/src/aggregates/row_hash.rs | 4 +- datafusion/physical-plan/src/union.rs | 10 +- datafusion/physical-plan/src/windows/mod.rs | 70 +++---- 6 files changed, 162 insertions(+), 153 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 636a11ad5780..032c2719a2b9 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -35,6 +35,37 @@ use datafusion_common::{JoinSide, JoinType, Result}; use indexmap::map::Entry; use indexmap::IndexMap; +#[derive(Debug, Clone, PartialEq)] +/// Specifies partition column properties in terms of input ordering +pub enum SetOrderMode { + /// None of the columns among the partition columns is ordered. + None, + /// Some columns of the partition columns are ordered but not all + PartiallySorted(Vec), + /// All Partition columns are ordered (Also empty case) + FullySorted(Vec), +} + +impl SetOrderMode { + // Return a reference to the ordered entries inside modes. + pub fn entries_ref(&self) -> &[T] { + match self { + SetOrderMode::FullySorted(entries) + | SetOrderMode::PartiallySorted(entries) => entries, + SetOrderMode::None => &[], + } + } + + // Return the ordered entries, inside modes. + pub fn entries(self) -> Vec { + match self { + SetOrderMode::FullySorted(entries) + | SetOrderMode::PartiallySorted(entries) => entries, + SetOrderMode::None => vec![], + } + } +} + /// An `EquivalenceClass` is a set of [`Arc`]s that are known /// to have the same value for all tuples in a relation. These are generated by /// equality predicates, typically equi-join conditions and equality conditions @@ -990,47 +1021,57 @@ impl SchemaProperties { } } - /// Checks whether any permutation of `exprs` shares a prefix with an - /// existing ordering. If so, returns the indices describing the permutation. - /// Otherwise, returns `None`. - pub fn set_satisfy(&self, exprs: &[Arc]) -> Option> { - self.get_lex_ordering_section(exprs) - .into_iter() - .map(|sort_expr| exprs.iter().position(|expr| sort_expr.expr.eq(expr))) - .collect() - } - - /// Checks whether any permutation of `exprs` satisfies the existing - /// ordering. If so, returns the indices describing the permutation. - /// Otherwise, returns `None`. - pub fn set_exactly_satisfy( + /// Returns the indices that describe an ordered section inside `exprs`. + /// It can return [`SetOrderMode::FullySorted`], [`SetOrderMode::PartiallySorted`], or [`SetOrderMode::None`] results. + /// See [`SetOrderMode`] for explanation of different modes. + pub fn set_ordered_indices( &self, exprs: &[Arc], - ) -> Option> { - self.set_satisfy(exprs) - .filter(|indices| indices.len() == exprs.len()) + ) -> SetOrderMode { + let (_ordering, order_indices) = self.set_ordered_helper(exprs); + + if order_indices.len() == exprs.len() { + // All of the expressions are ordered + SetOrderMode::FullySorted(order_indices) + } else if !order_indices.is_empty() { + // A prefix defines ordering + SetOrderMode::PartiallySorted(order_indices) + } else { + // None of the expressions is among ordered section. + SetOrderMode::None + } } - /// Assuming that `exprs` contains order-defining expressions, pairs each - /// of these expressions with its sort options according to the orderings - /// within. If the premise is false, returns `None`. - pub fn get_lex_ordering( + /// Returns the `PhysicalSortExpr`s that describe an ordered section (each expression is contained inside `exprs`). + /// It can return [`SetOrderMode::FullySorted`], [`SetOrderMode::PartiallySorted`], or [`SetOrderMode::None`] results. + /// See [`SetOrderMode`] for explanation of different modes. + pub fn set_ordered_section( &self, exprs: &[Arc], - ) -> Option { - let ordered_section = self.get_lex_ordering_section(exprs); - // Make sure ordered section covers all expressions. - (ordered_section.len() == exprs.len()).then_some(ordered_section) + ) -> SetOrderMode { + let (ordered_section, _ordering_indices) = self.set_ordered_helper(exprs); + if ordered_section.len() == exprs.len() { + // All of the expressions are ordered + SetOrderMode::FullySorted(ordered_section) + } else if !ordered_section.is_empty() { + // A prefix defines ordering + SetOrderMode::PartiallySorted(ordered_section) + } else { + // None of the expressions is among ordered section. + SetOrderMode::None + } } - /// Assuming that `exprs` contains order-defining expressions, pairs each - /// of these expressions with its sort options according to the orderings - /// within. Expressions that do not define a prefix of some ordering are - /// filtered out and they are not present in the return value. - pub fn get_lex_ordering_section( + /// Returns the longest ordered section (and its expression indices) that can be built out of given `exprs` + /// that is satisfied by existing orderings. + /// As an example if existing orderings are [a ASC, b ASC], [c DESC] + /// and `exprs` are \[c,b,a,d\]. This function returns ([a ASC, b ASC, c DESC], \[2,1,0\]). This means that + /// [a ASC, b ASC, c DESC] is satisfied by exiting ordering, and \[a,b,c\] resides at indices: 2,1,0 respectively + /// inside the argument `exprs`. + fn set_ordered_helper( &self, exprs: &[Arc], - ) -> LexOrdering { + ) -> (LexOrdering, Vec) { let normalized_exprs = self.eq_group.normalize_exprs(exprs.to_vec()); // Use a map to associate expression indices with sort options: let mut ordered_exprs = IndexMap::::new(); @@ -1053,11 +1094,16 @@ impl SchemaProperties { // Construct the lexicographical ordering according to the fpermutation: ordered_exprs .into_iter() - .map(|(idx, options)| PhysicalSortExpr { - expr: exprs[idx].clone(), - options, + .map(|(idx, options)| { + ( + PhysicalSortExpr { + expr: exprs[idx].clone(), + options, + }, + idx, + ) }) - .collect() + .unzip() } } @@ -1098,7 +1144,7 @@ pub fn join_schema_properties( // equivalence properties since stream side is the left side. // // For example, if the right side ordering equivalences contain - // `b ASC`, and the output ordering of the left side is `a ASC`, + // `b ASC`, and the left table ordering equivalences contain `a ASC`, // then we should add `a ASC, b ASC` to the ordering equivalences // of the join output. let out_oeq_class = left_oeq_class.join_suffix(&right_oeq_class); @@ -1120,9 +1166,9 @@ pub fn join_schema_properties( // with those of the right side while constructing output ordering // equivalence properties since stream side is the right side. // - // For example, if the right side ordering equivalences contain - // `b ASC`, and the output ordering of the left table is `a ASC`, - // then we should add `a ASC, b ASC` to the ordering equivalences + // For example, if the left side ordering equivalences contain + // `a ASC`, and the right table ordering equivalences contain `b ASC`, + // then we should add `b ASC, a ASC` to the ordering equivalences // of the join output. let out_oeq_class = right_oeq_class.join_suffix(&left_oeq_class); result.add_ordering_equivalence_class(out_oeq_class); @@ -2450,58 +2496,7 @@ mod tests { } #[test] - fn test_get_lex_ordering() -> Result<()> { - // Schema satisfies following orderings: - // [a ASC], [d ASC, b ASC], [e DESC, f ASC, g ASC] - // and - // Column [a=c] (e.g they are aliases). - let (test_schema, schema_props) = create_test_params()?; - let col_a = &col("a", &test_schema)?; - let col_b = &col("b", &test_schema)?; - let col_c = &col("c", &test_schema)?; - let col_d = &col("d", &test_schema)?; - let col_e = &col("e", &test_schema)?; - let option_asc = SortOptions { - descending: false, - nulls_first: false, - }; - let option_desc = SortOptions { - descending: true, - nulls_first: true, - }; - // First entry is physical expression list - // Second entry is the ordered section that consists of given physical expressions. - let test_cases = vec![ - // TEST CASE 1 - (vec![col_a], Some(vec![(col_a, option_asc)])), - // TEST CASE 2 - (vec![col_c], Some(vec![(col_c, option_asc)])), - // TEST CASE 3 - ( - vec![col_d, col_e, col_b], - Some(vec![ - (col_d, option_asc), - (col_b, option_asc), - (col_e, option_desc), - ]), - ), - // TEST CASE 4 - (vec![col_b], None), - // TEST CASE 5 - (vec![col_b, col_a], None), - ]; - for (exprs, expected) in test_cases { - let exprs = exprs.into_iter().cloned().collect::>(); - let expected = expected.map(|expected| convert_to_sort_exprs(&expected)); - let actual = schema_props.get_lex_ordering(&exprs); - assert_eq!(actual, expected); - } - - Ok(()) - } - - #[test] - fn test_get_lex_ordering_section() -> Result<()> { + fn test_set_ordered_helper() -> Result<()> { // Schema satisfies following orderings: // [a ASC], [d ASC, b ASC], [e DESC, f ASC, g ASC] // and @@ -2556,7 +2551,7 @@ mod tests { for (exprs, expected) in test_cases { let exprs = exprs.into_iter().cloned().collect::>(); let expected = convert_to_sort_exprs(&expected); - let actual = eq_properties.get_lex_ordering_section(&exprs); + let (actual, _indices) = eq_properties.set_ordered_helper(&exprs); assert_eq!(actual, expected); } diff --git a/datafusion/physical-expr/src/utils.rs b/datafusion/physical-expr/src/utils.rs index 28b2465325c4..0679dfb971f8 100644 --- a/datafusion/physical-expr/src/utils.rs +++ b/datafusion/physical-expr/src/utils.rs @@ -658,13 +658,17 @@ mod tests { }, ]]); assert_eq!( - eq_properties.set_exactly_satisfy(&required_columns), - Some(vec![0, 1]) + eq_properties + .set_ordered_indices(&required_columns) + .entries(), + &[0, 1] ); assert_eq!( - eq_properties.get_lex_ordering(&required_columns), - Some(vec![ + eq_properties + .set_ordered_section(&required_columns) + .entries(), + vec![ PhysicalSortExpr { expr: col_b.clone(), options: sort_options_not @@ -673,7 +677,7 @@ mod tests { expr: col_a.clone(), options: sort_options } - ]) + ] ); let schema = Schema::new(vec![ @@ -702,13 +706,17 @@ mod tests { ], ]); assert_eq!( - eq_properties.set_exactly_satisfy(&required_columns), - Some(vec![0, 1]) + eq_properties + .set_ordered_indices(&required_columns) + .entries(), + &[0, 1] ); assert_eq!( - eq_properties.get_lex_ordering(&required_columns), - Some(vec![ + eq_properties + .set_ordered_section(&required_columns) + .entries(), + vec![ PhysicalSortExpr { expr: col_b.clone(), options: sort_options_not @@ -717,7 +725,7 @@ mod tests { expr: col_a.clone(), options: sort_options } - ]) + ] ); let required_columns = [ @@ -746,7 +754,12 @@ mod tests { options: sort_options, }, ]]); - assert_eq!(eq_properties.set_exactly_satisfy(&required_columns), None); + assert_eq!( + eq_properties + .set_ordered_indices(&required_columns) + .entries(), + &[0] + ); Ok(()) } diff --git a/datafusion/physical-expr/src/window/built_in.rs b/datafusion/physical-expr/src/window/built_in.rs index dd3bfc0558ba..697a25a0df38 100644 --- a/datafusion/physical-expr/src/window/built_in.rs +++ b/datafusion/physical-expr/src/window/built_in.rs @@ -27,6 +27,7 @@ use crate::window::window_expr::{get_orderby_values, WindowFn}; use crate::window::{PartitionBatches, PartitionWindowAggStates, WindowState}; use crate::{reverse_order_bys, PhysicalExpr, SchemaProperties}; +use crate::equivalence::SetOrderMode; use arrow::array::{new_empty_array, ArrayRef}; use arrow::compute::SortOptions; use arrow::datatypes::Field; @@ -85,19 +86,11 @@ impl BuiltInWindowExpr { // expressions and existing ordering expressions are equal (w.r.t. // set equality), we can prefix the ordering of `self.expr` with // the existing ordering. - if let Some(indices) = - eq_properties.set_exactly_satisfy(&self.partition_by) + if let SetOrderMode::FullySorted(mut ordering) = + eq_properties.set_ordered_section(&self.partition_by) { - let lex_partition_by = indices - .into_iter() - .map(|idx| self.partition_by[idx].clone()) - .collect::>(); - if let Some(mut ordering) = - eq_properties.get_lex_ordering(&lex_partition_by) - { - ordering.push(fn_res_ordering); - eq_properties.add_new_orderings([ordering]); - } + ordering.push(fn_res_ordering); + eq_properties.add_new_orderings([ordering]); } } } diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index 9c044c3a846d..b3df77248e07 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -333,11 +333,11 @@ impl GroupedHashAggregateStream { let reservation = MemoryConsumer::new(name).register(context.memory_pool()); let ordered_section = agg .schema_properties() - .get_lex_ordering_section(&agg_group_by.output_exprs()); + .set_ordered_section(&agg_group_by.output_exprs()); let group_ordering = GroupOrdering::try_new( &group_schema, &agg.partition_search_mode, - &ordered_section, + &ordered_section.entries(), )?; let group_values = new_group_values(group_schema)?; diff --git a/datafusion/physical-plan/src/union.rs b/datafusion/physical-plan/src/union.rs index de8832f92569..b788eb93a8c2 100644 --- a/datafusion/physical-plan/src/union.rs +++ b/datafusion/physical-plan/src/union.rs @@ -244,9 +244,15 @@ impl ExecutionPlan for UnionExec { // equivalence class. let mut next_meets = vec![]; for current_meet in &meets { - next_meets.extend(child_eqs.oeq_class().iter().filter_map(|ordering| { + // Find the all of the meets of `current_meet` with orderings inside `child_eqs` + let valid_meets = child_eqs.oeq_class().iter().filter_map(|ordering| { child_eqs.get_meet_ordering(ordering, current_meet) - })); + }); + // Use the longest meet among `valid_meets`, other meets are redundant + if let Some(next_meet) = valid_meets.max_by(|a, b| a.len().cmp(&b.len())) + { + next_meets.push(next_meet); + } } meets = next_meets; } diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index 3c24ca6497a2..5c020f06c22b 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -32,12 +32,12 @@ use crate::{ use arrow::datatypes::Schema; use arrow_schema::{DataType, Field, SchemaRef}; -use datafusion_common::{DataFusionError, Result, ScalarValue}; +use datafusion_common::{exec_err, DataFusionError, Result, ScalarValue}; use datafusion_expr::{ window_function::{BuiltInWindowFunction, WindowFunction}, PartitionEvaluator, WindowFrame, WindowUDF, }; -use datafusion_physical_expr::equivalence::collapse_lex_req; +use datafusion_physical_expr::equivalence::{collapse_lex_req, SetOrderMode}; use datafusion_physical_expr::{ reverse_order_bys, window::{BuiltInWindowFunctionExpr, SlidingAggregateWindowExpr}, @@ -65,6 +65,18 @@ pub enum PartitionSearchMode { Sorted, } +impl From> for PartitionSearchMode { + fn from(value: SetOrderMode) -> Self { + match value { + SetOrderMode::FullySorted(_indices) => PartitionSearchMode::Sorted, + SetOrderMode::PartiallySorted(indices) => { + PartitionSearchMode::PartiallySorted(indices) + } + SetOrderMode::None => PartitionSearchMode::Linear, + } + } +} + /// Create a physical expression for window function pub fn create_window_expr( fun: &WindowFunction, @@ -325,11 +337,10 @@ pub(crate) fn get_ordered_partition_by_indices( partition_by_exprs: &[Arc], input: &Arc, ) -> Vec { - if let Some(indices) = input.schema_properties().set_satisfy(partition_by_exprs) { - indices - } else { - vec![] - } + input + .schema_properties() + .set_ordered_indices(partition_by_exprs) + .entries() } pub(crate) fn get_partition_by_sort_exprs( @@ -343,14 +354,14 @@ pub(crate) fn get_partition_by_sort_exprs( .collect::>(); // Make sure ordered section doesn't move over the partition by expression assert!(ordered_partition_by_indices.len() <= partition_by_exprs.len()); - input + if let SetOrderMode::FullySorted(ordered_section) = input .schema_properties() - .get_lex_ordering(&ordered_partition_exprs) - .ok_or_else(|| { - DataFusionError::Execution( - "Expects partition by expression to be ordered".to_string(), - ) - }) + .set_ordered_section(&ordered_partition_exprs) + { + Ok(ordered_section) + } else { + exec_err!("Expects partition by expression to be ordered") + } } pub(crate) fn window_ordering_equivalence( @@ -466,26 +477,17 @@ pub fn get_window_mode( input: &Arc, ) -> Result> { let input_eqs = input.schema_properties(); - let mut partition_search_mode = PartitionSearchMode::Linear; let mut partition_by_reqs: Vec = vec![]; - if partitionby_exprs.is_empty() { - partition_search_mode = PartitionSearchMode::Sorted; - } else if let Some(indices) = input_eqs.set_satisfy(partitionby_exprs) { - let item = indices - .iter() - .map(|&idx| PhysicalSortRequirement { - expr: partitionby_exprs[idx].clone(), - options: None, - }) - .collect::>(); - partition_by_reqs.extend(item); - if indices.len() == partitionby_exprs.len() { - partition_search_mode = PartitionSearchMode::Sorted; - } else if !indices.is_empty() { - partition_search_mode = PartitionSearchMode::PartiallySorted(indices); - } - } - + let set_ordered_indices = input_eqs.set_ordered_indices(partitionby_exprs); + let indices = set_ordered_indices.entries_ref(); + let item = indices + .iter() + .map(|&idx| PhysicalSortRequirement { + expr: partitionby_exprs[idx].clone(), + options: None, + }) + .collect::>(); + partition_by_reqs.extend(item); // Treat partition by exprs as constant. During analysis of requirements are satisfied. let partition_by_eqs = input_eqs.add_constants(partitionby_exprs.iter().cloned()); let order_by_reqs = PhysicalSortRequirement::from_sort_exprs(orderby_keys); @@ -498,7 +500,7 @@ pub fn get_window_mode( let req = collapse_lex_req(req); if partition_by_eqs.ordering_satisfy_requirement(&req) { // Window can be run with existing ordering - return Ok(Some((should_swap, partition_search_mode))); + return Ok(Some((should_swap, set_ordered_indices.into()))); } } Ok(None) From 6ab2763d93c1a9d6be0a50cbb28a99e7b02b7205 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Mon, 30 Oct 2023 18:34:33 +0300 Subject: [PATCH 114/122] Review Part 25 --- datafusion/physical-expr/src/equivalence.rs | 92 +++---------------- datafusion/physical-expr/src/utils.rs | 34 ++----- .../physical-expr/src/window/built_in.rs | 7 +- .../physical-plan/src/aggregates/row_hash.rs | 6 +- datafusion/physical-plan/src/windows/mod.rs | 40 ++++---- 5 files changed, 43 insertions(+), 136 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 032c2719a2b9..681cd4ed526c 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -35,37 +35,6 @@ use datafusion_common::{JoinSide, JoinType, Result}; use indexmap::map::Entry; use indexmap::IndexMap; -#[derive(Debug, Clone, PartialEq)] -/// Specifies partition column properties in terms of input ordering -pub enum SetOrderMode { - /// None of the columns among the partition columns is ordered. - None, - /// Some columns of the partition columns are ordered but not all - PartiallySorted(Vec), - /// All Partition columns are ordered (Also empty case) - FullySorted(Vec), -} - -impl SetOrderMode { - // Return a reference to the ordered entries inside modes. - pub fn entries_ref(&self) -> &[T] { - match self { - SetOrderMode::FullySorted(entries) - | SetOrderMode::PartiallySorted(entries) => entries, - SetOrderMode::None => &[], - } - } - - // Return the ordered entries, inside modes. - pub fn entries(self) -> Vec { - match self { - SetOrderMode::FullySorted(entries) - | SetOrderMode::PartiallySorted(entries) => entries, - SetOrderMode::None => vec![], - } - } -} - /// An `EquivalenceClass` is a set of [`Arc`]s that are known /// to have the same value for all tuples in a relation. These are generated by /// equality predicates, typically equi-join conditions and equality conditions @@ -1021,54 +990,17 @@ impl SchemaProperties { } } - /// Returns the indices that describe an ordered section inside `exprs`. - /// It can return [`SetOrderMode::FullySorted`], [`SetOrderMode::PartiallySorted`], or [`SetOrderMode::None`] results. - /// See [`SetOrderMode`] for explanation of different modes. - pub fn set_ordered_indices( - &self, - exprs: &[Arc], - ) -> SetOrderMode { - let (_ordering, order_indices) = self.set_ordered_helper(exprs); - - if order_indices.len() == exprs.len() { - // All of the expressions are ordered - SetOrderMode::FullySorted(order_indices) - } else if !order_indices.is_empty() { - // A prefix defines ordering - SetOrderMode::PartiallySorted(order_indices) - } else { - // None of the expressions is among ordered section. - SetOrderMode::None - } - } - - /// Returns the `PhysicalSortExpr`s that describe an ordered section (each expression is contained inside `exprs`). - /// It can return [`SetOrderMode::FullySorted`], [`SetOrderMode::PartiallySorted`], or [`SetOrderMode::None`] results. - /// See [`SetOrderMode`] for explanation of different modes. - pub fn set_ordered_section( - &self, - exprs: &[Arc], - ) -> SetOrderMode { - let (ordered_section, _ordering_indices) = self.set_ordered_helper(exprs); - if ordered_section.len() == exprs.len() { - // All of the expressions are ordered - SetOrderMode::FullySorted(ordered_section) - } else if !ordered_section.is_empty() { - // A prefix defines ordering - SetOrderMode::PartiallySorted(ordered_section) - } else { - // None of the expressions is among ordered section. - SetOrderMode::None - } - } - - /// Returns the longest ordered section (and its expression indices) that can be built out of given `exprs` - /// that is satisfied by existing orderings. - /// As an example if existing orderings are [a ASC, b ASC], [c DESC] - /// and `exprs` are \[c,b,a,d\]. This function returns ([a ASC, b ASC, c DESC], \[2,1,0\]). This means that - /// [a ASC, b ASC, c DESC] is satisfied by exiting ordering, and \[a,b,c\] resides at indices: 2,1,0 respectively - /// inside the argument `exprs`. - fn set_ordered_helper( + /// Returns the longest (potentially partial) permutation satisfying the + /// existing ordering. For example, if we have the equivalent orderings + /// `[a ASC, b ASC]` and `[c DESC]`, with `exprs` containing `[c, b, a, d]`, + /// then this function returns `([a ASC, b ASC, c DESC], [2, 1, 0])`. + /// This means that the specification `[a ASC, b ASC, c DESC]` is satisfied + /// by the existing ordering, and `[a, b, c]` resides at indices: `2, 1, 0` + /// inside the argument `exprs` (respectively). For the mathematical + /// definition of "partial permutation", see: + /// + /// + pub fn find_longest_permutation( &self, exprs: &[Arc], ) -> (LexOrdering, Vec) { @@ -2551,7 +2483,7 @@ mod tests { for (exprs, expected) in test_cases { let exprs = exprs.into_iter().cloned().collect::>(); let expected = convert_to_sort_exprs(&expected); - let (actual, _indices) = eq_properties.set_ordered_helper(&exprs); + let (actual, _indices) = eq_properties.find_longest_permutation(&exprs); assert_eq!(actual, expected); } diff --git a/datafusion/physical-expr/src/utils.rs b/datafusion/physical-expr/src/utils.rs index 0679dfb971f8..4ed1b60669fa 100644 --- a/datafusion/physical-expr/src/utils.rs +++ b/datafusion/physical-expr/src/utils.rs @@ -657,17 +657,10 @@ mod tests { options: sort_options, }, ]]); + let (result, idxs) = eq_properties.find_longest_permutation(&required_columns); + assert_eq!(idxs, vec![0, 1]); assert_eq!( - eq_properties - .set_ordered_indices(&required_columns) - .entries(), - &[0, 1] - ); - - assert_eq!( - eq_properties - .set_ordered_section(&required_columns) - .entries(), + result, vec![ PhysicalSortExpr { expr: col_b.clone(), @@ -705,17 +698,10 @@ mod tests { }, ], ]); + let (result, idxs) = eq_properties.find_longest_permutation(&required_columns); + assert_eq!(idxs, vec![0, 1]); assert_eq!( - eq_properties - .set_ordered_indices(&required_columns) - .entries(), - &[0, 1] - ); - - assert_eq!( - eq_properties - .set_ordered_section(&required_columns) - .entries(), + result, vec![ PhysicalSortExpr { expr: col_b.clone(), @@ -754,12 +740,8 @@ mod tests { options: sort_options, }, ]]); - assert_eq!( - eq_properties - .set_ordered_indices(&required_columns) - .entries(), - &[0] - ); + let (_, idxs) = eq_properties.find_longest_permutation(&required_columns); + assert_eq!(idxs, vec![0]); Ok(()) } diff --git a/datafusion/physical-expr/src/window/built_in.rs b/datafusion/physical-expr/src/window/built_in.rs index 697a25a0df38..1aa99ce2b745 100644 --- a/datafusion/physical-expr/src/window/built_in.rs +++ b/datafusion/physical-expr/src/window/built_in.rs @@ -27,7 +27,6 @@ use crate::window::window_expr::{get_orderby_values, WindowFn}; use crate::window::{PartitionBatches, PartitionWindowAggStates, WindowState}; use crate::{reverse_order_bys, PhysicalExpr, SchemaProperties}; -use crate::equivalence::SetOrderMode; use arrow::array::{new_empty_array, ArrayRef}; use arrow::compute::SortOptions; use arrow::datatypes::Field; @@ -86,9 +85,9 @@ impl BuiltInWindowExpr { // expressions and existing ordering expressions are equal (w.r.t. // set equality), we can prefix the ordering of `self.expr` with // the existing ordering. - if let SetOrderMode::FullySorted(mut ordering) = - eq_properties.set_ordered_section(&self.partition_by) - { + let (mut ordering, _) = + eq_properties.find_longest_permutation(&self.partition_by); + if ordering.len() == self.partition_by.len() { ordering.push(fn_res_ordering); eq_properties.add_new_orderings([ordering]); } diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index b3df77248e07..81122ffb66d5 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -331,13 +331,13 @@ impl GroupedHashAggregateStream { let name = format!("GroupedHashAggregateStream[{partition}]"); let reservation = MemoryConsumer::new(name).register(context.memory_pool()); - let ordered_section = agg + let (ordering, _) = agg .schema_properties() - .set_ordered_section(&agg_group_by.output_exprs()); + .find_longest_permutation(&agg_group_by.output_exprs()); let group_ordering = GroupOrdering::try_new( &group_schema, &agg.partition_search_mode, - &ordered_section.entries(), + ordering.as_slice(), )?; let group_values = new_group_values(group_schema)?; diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index 5c020f06c22b..be83288d25c7 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -37,7 +37,7 @@ use datafusion_expr::{ window_function::{BuiltInWindowFunction, WindowFunction}, PartitionEvaluator, WindowFrame, WindowUDF, }; -use datafusion_physical_expr::equivalence::{collapse_lex_req, SetOrderMode}; +use datafusion_physical_expr::equivalence::collapse_lex_req; use datafusion_physical_expr::{ reverse_order_bys, window::{BuiltInWindowFunctionExpr, SlidingAggregateWindowExpr}, @@ -65,18 +65,6 @@ pub enum PartitionSearchMode { Sorted, } -impl From> for PartitionSearchMode { - fn from(value: SetOrderMode) -> Self { - match value { - SetOrderMode::FullySorted(_indices) => PartitionSearchMode::Sorted, - SetOrderMode::PartiallySorted(indices) => { - PartitionSearchMode::PartiallySorted(indices) - } - SetOrderMode::None => PartitionSearchMode::Linear, - } - } -} - /// Create a physical expression for window function pub fn create_window_expr( fun: &WindowFunction, @@ -339,8 +327,8 @@ pub(crate) fn get_ordered_partition_by_indices( ) -> Vec { input .schema_properties() - .set_ordered_indices(partition_by_exprs) - .entries() + .find_longest_permutation(partition_by_exprs) + .1 } pub(crate) fn get_partition_by_sort_exprs( @@ -354,13 +342,13 @@ pub(crate) fn get_partition_by_sort_exprs( .collect::>(); // Make sure ordered section doesn't move over the partition by expression assert!(ordered_partition_by_indices.len() <= partition_by_exprs.len()); - if let SetOrderMode::FullySorted(ordered_section) = input + let (ordering, _) = input .schema_properties() - .set_ordered_section(&ordered_partition_exprs) - { - Ok(ordered_section) + .find_longest_permutation(&ordered_partition_exprs); + if ordering.len() == ordered_partition_exprs.len() { + Ok(ordering) } else { - exec_err!("Expects partition by expression to be ordered") + exec_err!("Expects PARTITION BY expression to be ordered") } } @@ -478,8 +466,7 @@ pub fn get_window_mode( ) -> Result> { let input_eqs = input.schema_properties(); let mut partition_by_reqs: Vec = vec![]; - let set_ordered_indices = input_eqs.set_ordered_indices(partitionby_exprs); - let indices = set_ordered_indices.entries_ref(); + let (_, indices) = input_eqs.find_longest_permutation(partitionby_exprs); let item = indices .iter() .map(|&idx| PhysicalSortRequirement { @@ -500,7 +487,14 @@ pub fn get_window_mode( let req = collapse_lex_req(req); if partition_by_eqs.ordering_satisfy_requirement(&req) { // Window can be run with existing ordering - return Ok(Some((should_swap, set_ordered_indices.into()))); + let mode = if indices.len() == partitionby_exprs.len() { + PartitionSearchMode::Sorted + } else if indices.is_empty() { + PartitionSearchMode::Linear + } else { + PartitionSearchMode::PartiallySorted(indices) + }; + return Ok(Some((should_swap, mode))); } } Ok(None) From 1a8fb60bf32092fe97afaed784197048413e926a Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Mon, 30 Oct 2023 19:48:19 +0300 Subject: [PATCH 115/122] Review Part 26 --- datafusion/physical-expr/src/equivalence.rs | 6 +++--- datafusion/physical-plan/src/aggregates/mod.rs | 4 ++-- datafusion/physical-plan/src/windows/mod.rs | 18 +++++++----------- 3 files changed, 12 insertions(+), 16 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 681cd4ed526c..f5ae82d50bce 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -1076,7 +1076,7 @@ pub fn join_schema_properties( // equivalence properties since stream side is the left side. // // For example, if the right side ordering equivalences contain - // `b ASC`, and the left table ordering equivalences contain `a ASC`, + // `b ASC`, and the left side ordering equivalences contain `a ASC`, // then we should add `a ASC, b ASC` to the ordering equivalences // of the join output. let out_oeq_class = left_oeq_class.join_suffix(&right_oeq_class); @@ -1099,7 +1099,7 @@ pub fn join_schema_properties( // equivalence properties since stream side is the right side. // // For example, if the left side ordering equivalences contain - // `a ASC`, and the right table ordering equivalences contain `b ASC`, + // `a ASC`, and the right side ordering equivalences contain `b ASC`, // then we should add `b ASC, a ASC` to the ordering equivalences // of the join output. let out_oeq_class = right_oeq_class.join_suffix(&left_oeq_class); @@ -2483,7 +2483,7 @@ mod tests { for (exprs, expected) in test_cases { let exprs = exprs.into_iter().cloned().collect::>(); let expected = convert_to_sort_exprs(&expected); - let (actual, _indices) = eq_properties.find_longest_permutation(&exprs); + let (actual, _) = eq_properties.find_longest_permutation(&exprs); assert_eq!(actual, expected); } diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index 22e2ae92893e..d7f74d5f1704 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -542,10 +542,10 @@ impl AggregateExec { let required_input_ordering = (!new_requirement.is_empty()).then_some(new_requirement); - let aggregate_oeq = input + let aggregate_eqs = input .schema_properties() .project(&projection_mapping, schema.clone()); - let output_ordering = aggregate_oeq.oeq_class().output_ordering(); + let output_ordering = aggregate_eqs.oeq_class().output_ordering(); Ok(AggregateExec { mode, diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index be83288d25c7..d06c1c74813a 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -325,10 +325,10 @@ pub(crate) fn get_ordered_partition_by_indices( partition_by_exprs: &[Arc], input: &Arc, ) -> Vec { - input + let (_, indices) = input .schema_properties() - .find_longest_permutation(partition_by_exprs) - .1 + .find_longest_permutation(partition_by_exprs); + indices } pub(crate) fn get_partition_by_sort_exprs( @@ -467,14 +467,10 @@ pub fn get_window_mode( let input_eqs = input.schema_properties(); let mut partition_by_reqs: Vec = vec![]; let (_, indices) = input_eqs.find_longest_permutation(partitionby_exprs); - let item = indices - .iter() - .map(|&idx| PhysicalSortRequirement { - expr: partitionby_exprs[idx].clone(), - options: None, - }) - .collect::>(); - partition_by_reqs.extend(item); + partition_by_reqs.extend(indices.iter().map(|&idx| PhysicalSortRequirement { + expr: partitionby_exprs[idx].clone(), + options: None, + })); // Treat partition by exprs as constant. During analysis of requirements are satisfied. let partition_by_eqs = input_eqs.add_constants(partitionby_exprs.iter().cloned()); let order_by_reqs = PhysicalSortRequirement::from_sort_exprs(orderby_keys); From ed80652b06114f54f3b1bd4f62f82b968539c3f3 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Tue, 31 Oct 2023 14:07:12 +0300 Subject: [PATCH 116/122] Name Changes, comment updates --- .../datasource/physical_plan/arrow_file.rs | 6 +- .../core/src/datasource/physical_plan/avro.rs | 6 +- .../core/src/datasource/physical_plan/csv.rs | 6 +- .../core/src/datasource/physical_plan/json.rs | 6 +- .../src/datasource/physical_plan/parquet.rs | 6 +- .../enforce_distribution.rs | 24 +-- .../src/physical_optimizer/enforce_sorting.rs | 4 +- .../replace_with_order_preserving_variants.rs | 2 +- .../src/physical_optimizer/sort_pushdown.rs | 12 +- .../core/src/physical_optimizer/utils.rs | 2 +- datafusion/physical-expr/src/equivalence.rs | 174 +++++++++--------- datafusion/physical-expr/src/lib.rs | 2 +- datafusion/physical-expr/src/partitioning.rs | 20 +- datafusion/physical-expr/src/utils.rs | 18 +- .../physical-expr/src/window/built_in.rs | 4 +- .../physical-plan/src/aggregates/mod.rs | 41 ++--- .../physical-plan/src/aggregates/row_hash.rs | 2 +- .../physical-plan/src/coalesce_batches.rs | 6 +- datafusion/physical-plan/src/filter.rs | 8 +- .../physical-plan/src/joins/cross_join.rs | 12 +- .../physical-plan/src/joins/hash_join.rs | 12 +- .../src/joins/sort_merge_join.rs | 12 +- .../src/joins/symmetric_hash_join.rs | 14 +- datafusion/physical-plan/src/lib.rs | 8 +- datafusion/physical-plan/src/limit.rs | 10 +- datafusion/physical-plan/src/memory.rs | 12 +- datafusion/physical-plan/src/projection.rs | 12 +- .../physical-plan/src/repartition/mod.rs | 6 +- datafusion/physical-plan/src/sorts/sort.rs | 6 +- .../src/sorts/sort_preserving_merge.rs | 6 +- datafusion/physical-plan/src/streaming.rs | 6 +- datafusion/physical-plan/src/union.rs | 14 +- .../src/windows/bounded_window_agg_exec.rs | 12 +- datafusion/physical-plan/src/windows/mod.rs | 22 +-- .../src/windows/window_agg_exec.rs | 10 +- 35 files changed, 265 insertions(+), 258 deletions(-) diff --git a/datafusion/core/src/datasource/physical_plan/arrow_file.rs b/datafusion/core/src/datasource/physical_plan/arrow_file.rs index acec6f60bda2..30b55db28491 100644 --- a/datafusion/core/src/datasource/physical_plan/arrow_file.rs +++ b/datafusion/core/src/datasource/physical_plan/arrow_file.rs @@ -32,7 +32,7 @@ use crate::physical_plan::{ use arrow_schema::SchemaRef; use datafusion_common::Statistics; use datafusion_execution::TaskContext; -use datafusion_physical_expr::{LexOrdering, PhysicalSortExpr, SchemaProperties}; +use datafusion_physical_expr::{EquivalenceProperties, LexOrdering, PhysicalSortExpr}; use futures::StreamExt; use object_store::{GetResultPayload, ObjectStore}; @@ -103,8 +103,8 @@ impl ExecutionPlan for ArrowExec { .map(|ordering| ordering.as_slice()) } - fn schema_properties(&self) -> SchemaProperties { - SchemaProperties::new_with_orderings( + fn equivalence_properties(&self) -> EquivalenceProperties { + EquivalenceProperties::new_with_orderings( self.schema(), &self.projected_output_ordering, ) diff --git a/datafusion/core/src/datasource/physical_plan/avro.rs b/datafusion/core/src/datasource/physical_plan/avro.rs index f3778d11970c..b97f162fd2f5 100644 --- a/datafusion/core/src/datasource/physical_plan/avro.rs +++ b/datafusion/core/src/datasource/physical_plan/avro.rs @@ -31,7 +31,7 @@ use crate::physical_plan::{ use arrow::datatypes::SchemaRef; use datafusion_execution::TaskContext; -use datafusion_physical_expr::{LexOrdering, SchemaProperties}; +use datafusion_physical_expr::{EquivalenceProperties, LexOrdering}; /// Execution plan for scanning Avro data source #[derive(Debug, Clone)] @@ -99,8 +99,8 @@ impl ExecutionPlan for AvroExec { .map(|ordering| ordering.as_slice()) } - fn schema_properties(&self) -> SchemaProperties { - SchemaProperties::new_with_orderings( + fn equivalence_properties(&self) -> EquivalenceProperties { + EquivalenceProperties::new_with_orderings( self.schema(), &self.projected_output_ordering, ) diff --git a/datafusion/core/src/datasource/physical_plan/csv.rs b/datafusion/core/src/datasource/physical_plan/csv.rs index fdfefaf9aaac..75aa343ffbfc 100644 --- a/datafusion/core/src/datasource/physical_plan/csv.rs +++ b/datafusion/core/src/datasource/physical_plan/csv.rs @@ -41,7 +41,7 @@ use crate::physical_plan::{ use arrow::csv; use arrow::datatypes::SchemaRef; use datafusion_execution::TaskContext; -use datafusion_physical_expr::{LexOrdering, SchemaProperties}; +use datafusion_physical_expr::{EquivalenceProperties, LexOrdering}; use bytes::{Buf, Bytes}; use datafusion_common::config::ConfigOptions; @@ -157,8 +157,8 @@ impl ExecutionPlan for CsvExec { .map(|ordering| ordering.as_slice()) } - fn schema_properties(&self) -> SchemaProperties { - SchemaProperties::new_with_orderings( + fn equivalence_properties(&self) -> EquivalenceProperties { + EquivalenceProperties::new_with_orderings( self.schema(), &self.projected_output_ordering, ) diff --git a/datafusion/core/src/datasource/physical_plan/json.rs b/datafusion/core/src/datasource/physical_plan/json.rs index 86283148c40a..73dcb32ac81f 100644 --- a/datafusion/core/src/datasource/physical_plan/json.rs +++ b/datafusion/core/src/datasource/physical_plan/json.rs @@ -40,7 +40,7 @@ use crate::physical_plan::{ use arrow::json::ReaderBuilder; use arrow::{datatypes::SchemaRef, json}; use datafusion_execution::TaskContext; -use datafusion_physical_expr::{LexOrdering, SchemaProperties}; +use datafusion_physical_expr::{EquivalenceProperties, LexOrdering}; use bytes::{Buf, Bytes}; use futures::{ready, stream, StreamExt, TryStreamExt}; @@ -120,8 +120,8 @@ impl ExecutionPlan for NdJsonExec { .map(|ordering| ordering.as_slice()) } - fn schema_properties(&self) -> SchemaProperties { - SchemaProperties::new_with_orderings( + fn equivalence_properties(&self) -> EquivalenceProperties { + EquivalenceProperties::new_with_orderings( self.schema(), &self.projected_output_ordering, ) diff --git a/datafusion/core/src/datasource/physical_plan/parquet.rs b/datafusion/core/src/datasource/physical_plan/parquet.rs index c8fea4ad8beb..960b2ec7337d 100644 --- a/datafusion/core/src/datasource/physical_plan/parquet.rs +++ b/datafusion/core/src/datasource/physical_plan/parquet.rs @@ -45,7 +45,7 @@ use crate::{ use arrow::datatypes::{DataType, SchemaRef}; use arrow::error::ArrowError; use datafusion_physical_expr::{ - LexOrdering, PhysicalExpr, PhysicalSortExpr, SchemaProperties, + EquivalenceProperties, LexOrdering, PhysicalExpr, PhysicalSortExpr, }; use bytes::Bytes; @@ -314,8 +314,8 @@ impl ExecutionPlan for ParquetExec { .map(|ordering| ordering.as_slice()) } - fn schema_properties(&self) -> SchemaProperties { - SchemaProperties::new_with_orderings( + fn equivalence_properties(&self) -> EquivalenceProperties { + EquivalenceProperties::new_with_orderings( self.schema(), &self.projected_output_ordering, ) diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index 7bc5f74a8499..b0d142dfa504 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -51,7 +51,9 @@ use datafusion_common::tree_node::{Transformed, TreeNode, VisitRecursion}; use datafusion_expr::logical_plan::JoinType; use datafusion_physical_expr::expressions::{Column, NoOp}; use datafusion_physical_expr::utils::map_columns_before_projection; -use datafusion_physical_expr::{physical_exprs_equal, PhysicalExpr, SchemaProperties}; +use datafusion_physical_expr::{ + physical_exprs_equal, EquivalenceProperties, PhysicalExpr, +}; use datafusion_physical_plan::unbounded_output; use datafusion_physical_plan::windows::{get_best_fitting_window, BoundedWindowAggExec}; @@ -442,7 +444,7 @@ where )) = try_reorder( join_key_pairs.clone(), parent_required, - &join_plan.schema_properties(), + &join_plan.equivalence_properties(), ) { if !new_positions.is_empty() { let new_join_on = new_join_conditions(&left_keys, &right_keys); @@ -658,8 +660,8 @@ pub(crate) fn reorder_join_keys_to_inputs( join_key_pairs, Some(left.output_partitioning()), Some(right.output_partitioning()), - &left.schema_properties(), - &right.schema_properties(), + &left.equivalence_properties(), + &right.equivalence_properties(), ) { if !new_positions.is_empty() { let new_join_on = new_join_conditions(&left_keys, &right_keys); @@ -696,8 +698,8 @@ pub(crate) fn reorder_join_keys_to_inputs( join_key_pairs, Some(left.output_partitioning()), Some(right.output_partitioning()), - &left.schema_properties(), - &right.schema_properties(), + &left.equivalence_properties(), + &right.equivalence_properties(), ) { if !new_positions.is_empty() { let new_join_on = new_join_conditions(&left_keys, &right_keys); @@ -723,8 +725,8 @@ fn reorder_current_join_keys( join_keys: JoinKeyPairs, left_partition: Option, right_partition: Option, - left_equivalence_properties: &SchemaProperties, - right_equivalence_properties: &SchemaProperties, + left_equivalence_properties: &EquivalenceProperties, + right_equivalence_properties: &EquivalenceProperties, ) -> Option<(JoinKeyPairs, Vec)> { match (left_partition, right_partition.clone()) { (Some(Partitioning::Hash(left_exprs, _)), _) => { @@ -749,7 +751,7 @@ fn reorder_current_join_keys( fn try_reorder( join_keys: JoinKeyPairs, expected: &[Arc], - equivalence_properties: &SchemaProperties, + equivalence_properties: &EquivalenceProperties, ) -> Option<(JoinKeyPairs, Vec)> { let eq_groups = equivalence_properties.eq_group(); let mut normalized_expected = vec![]; @@ -984,7 +986,7 @@ fn add_hash_on_top( let satisfied = input .output_partitioning() .satisfy(Distribution::HashPartitioned(hash_exprs.clone()), || { - input.schema_properties() + input.equivalence_properties() }); // Add hash repartitioning when: // - The hash distribution requirement is not satisfied, or @@ -1301,7 +1303,7 @@ fn ensure_distribution( // - Ordering requirement cannot be satisfied by preserving ordering through repartitions, or // - using order preserving variant is not desirable. let ordering_satisfied = child - .schema_properties() + .equivalence_properties() .ordering_satisfy_requirement(required_input_ordering); if !ordering_satisfied || !order_preserving_variants_desirable { replace_order_preserving_variants(&mut child, dist_onward)?; diff --git a/datafusion/core/src/physical_optimizer/enforce_sorting.rs b/datafusion/core/src/physical_optimizer/enforce_sorting.rs index 424d3c73e452..4779ced44f1a 100644 --- a/datafusion/core/src/physical_optimizer/enforce_sorting.rs +++ b/datafusion/core/src/physical_optimizer/enforce_sorting.rs @@ -451,7 +451,7 @@ fn ensure_sorting( match (required_ordering, physical_ordering) { (Some(required_ordering), Some(_)) => { if !child - .schema_properties() + .equivalence_properties() .ordering_satisfy_requirement(&required_ordering) { // Make sure we preserve the ordering requirements: @@ -515,7 +515,7 @@ fn analyze_immediate_sort_removal( // If this sort is unnecessary, we should remove it: if sort_input - .schema_properties() + .equivalence_properties() .ordering_satisfy(sort_exec.output_ordering().unwrap_or(&[])) { // Since we know that a `SortExec` has exactly one child, diff --git a/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs b/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs index 975c71ce37db..0c2f21d11acd 100644 --- a/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs +++ b/datafusion/core/src/physical_optimizer/replace_with_order_preserving_variants.rs @@ -258,7 +258,7 @@ pub(crate) fn replace_with_order_preserving_variants( )?; // If this sort is unnecessary, we should remove it and update the plan: if updated_sort_input - .schema_properties() + .equivalence_properties() .ordering_satisfy(plan.output_ordering().unwrap_or(&[])) { return Ok(Transformed::Yes(OrderPreservationContext { diff --git a/datafusion/core/src/physical_optimizer/sort_pushdown.rs b/datafusion/core/src/physical_optimizer/sort_pushdown.rs index ef59d81bb44c..b9502d92ac12 100644 --- a/datafusion/core/src/physical_optimizer/sort_pushdown.rs +++ b/datafusion/core/src/physical_optimizer/sort_pushdown.rs @@ -127,7 +127,7 @@ pub(crate) fn pushdown_sorts( let parent_required = requirements.required_ordering.as_deref().unwrap_or(&[]); if let Some(sort_exec) = plan.as_any().downcast_ref::() { let new_plan = if !plan - .schema_properties() + .equivalence_properties() .ordering_satisfy_requirement(parent_required) { // If the current plan is a SortExec, modify it to satisfy parent requirements: @@ -159,7 +159,7 @@ pub(crate) fn pushdown_sorts( } else { // Executors other than SortExec if plan - .schema_properties() + .equivalence_properties() .ordering_satisfy_requirement(parent_required) { // Satisfies parent requirements, immediately return. @@ -263,7 +263,7 @@ fn pushdown_requirement_to_children( } else if is_sort_preserving_merge(plan) { let new_ordering = PhysicalSortRequirement::to_sort_exprs(parent_required.to_vec()); - let mut spm_eqs = plan.schema_properties(); + let mut spm_eqs = plan.equivalence_properties(); // Sort preserving merge will have new ordering, one requirement above is pushed down to its below. spm_eqs = spm_eqs.with_reorder(new_ordering); // Do not push-down through SortPreservingMergeExec when @@ -307,13 +307,13 @@ fn determine_children_requirement( child_plan: Arc, ) -> RequirementsCompatibility { if child_plan - .schema_properties() + .equivalence_properties() .requirements_compatible(request_child, parent_required) { // request child requirements are more specific, no need to push down the parent requirements RequirementsCompatibility::Satisfy } else if child_plan - .schema_properties() + .equivalence_properties() .requirements_compatible(parent_required, request_child) { // parent requirements are more specific, adjust the request child requirements and push down the new requirements @@ -350,7 +350,7 @@ fn try_pushdown_requirements_to_join( &smj.maintains_input_order(), Some(probe_side), ); - let mut smj_eqs = smj.schema_properties(); + let mut smj_eqs = smj.equivalence_properties(); // smj will have this ordering when its input changes. smj_eqs = smj_eqs.with_reorder(new_output_ordering.unwrap_or_default()); let should_pushdown = smj_eqs.ordering_satisfy_requirement(parent_required); diff --git a/datafusion/core/src/physical_optimizer/utils.rs b/datafusion/core/src/physical_optimizer/utils.rs index cd1fc6247538..530df374ca7c 100644 --- a/datafusion/core/src/physical_optimizer/utils.rs +++ b/datafusion/core/src/physical_optimizer/utils.rs @@ -104,7 +104,7 @@ pub fn add_sort_above( ) { // If the ordering requirement is already satisfied, do not add a sort. if !node - .schema_properties() + .equivalence_properties() .ordering_satisfy_requirement(sort_requirement) { let sort_expr = PhysicalSortRequirement::to_sort_exprs(sort_requirement.to_vec()); diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index fb470130daee..7c26d2264646 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -134,6 +134,7 @@ impl EquivalenceGroup { // Remove duplicate entries from each equivalence class: self.classes.retain_mut(|cls| { // Keep groups that have at least two entries: + // Single entry inside an equivalence class is meaningless. It doesn't contain any information deduplicate_physical_exprs(cls); cls.len() > 1 }); @@ -142,6 +143,9 @@ impl EquivalenceGroup { } /// This utility function unifies/bridges classes that have common expressions. + /// As an example if an ['EquivalenceClass'] in the state is \[a, b\] and another is \[b,c\] + /// Since, both classes contain b, this means that indeed \[a,b,c] are all equal to each other. + /// This utility converts \[a,b\], \[b,c\] to \[a,b,c\]. fn bridge_classes(&mut self) { let mut idx = 0; while idx < self.classes.len() { @@ -579,10 +583,11 @@ fn finer_side(lhs: LexOrderingRef, rhs: LexOrderingRef) -> Option { all_equal.then_some(lhs.len() < rhs.len()) } -/// A `SchemaProperties` object stores useful information related to a schema. +/// A `EquivalenceProperties` object stores useful information related to a schema. /// Currently, it keeps track of: -/// - Equivalent columns, e.g columns that have same value. +/// - Equivalent expressions, e.g expressions that have same value. /// - Valid sort expressions (orderings) for the schema. +/// - Constants expressions (e.g expressions that are known to have constant values). /// /// Consider table below: /// @@ -598,7 +603,7 @@ fn finer_side(lhs: LexOrderingRef, rhs: LexOrderingRef) -> Option { /// ``` /// /// where both `a ASC` and `b DESC` can describe the table ordering. With -/// `SchemaProperties`, we can keep track of these different valid sort +/// `EquivalenceProperties`, we can keep track of these different valid sort /// expressions and treat `a ASC` and `b DESC` on an equal footing. /// /// Similarly, consider the table below: @@ -620,7 +625,7 @@ fn finer_side(lhs: LexOrderingRef, rhs: LexOrderingRef) -> Option { /// `Hash(a)` and output partitioning is `Hash(b)`, then we can deduce that /// the existing partitioning satisfies the requirement. #[derive(Debug, Clone)] -pub struct SchemaProperties { +pub struct EquivalenceProperties { /// Collection of equivalence classes that store expressions with the same /// value. eq_group: EquivalenceGroup, @@ -634,8 +639,8 @@ pub struct SchemaProperties { schema: SchemaRef, } -impl SchemaProperties { - /// Creates an empty `SchemaProperties` object. +impl EquivalenceProperties { + /// Creates an empty `EquivalenceProperties` object. pub fn new(schema: SchemaRef) -> Self { Self { eq_group: EquivalenceGroup::empty(), @@ -645,7 +650,7 @@ impl SchemaProperties { } } - /// Creates a new `SchemaProperties` object with the given orderings. + /// Creates a new `EquivalenceProperties` object with the given orderings. pub fn new_with_orderings(schema: SchemaRef, orderings: &[LexOrdering]) -> Self { Self { eq_group: EquivalenceGroup::empty(), @@ -682,7 +687,7 @@ impl SchemaProperties { ) } - /// Extends this `SchemaProperties` with the `other` object. + /// Extends this `EquivalenceProperties` with the `other` object. pub fn extend(mut self, other: Self) -> Self { self.eq_group.extend(other.eq_group); self.oeq_class.extend(other.oeq_class); @@ -695,7 +700,7 @@ impl SchemaProperties { self.oeq_class.clear(); } - /// Extends this `SchemaProperties` by adding the orderings inside the + /// Extends this `EquivalenceProperties` by adding the orderings inside the /// ordering equivalence class `other`. pub fn add_ordering_equivalence_class(&mut self, other: OrderingEquivalenceClass) { self.oeq_class.extend(other); @@ -754,8 +759,11 @@ impl SchemaProperties { /// Assume that `self.eq_group` states column `a` and `b` are aliases. /// Also assume that `self.oeq_class` states orderings `d ASC` and `a ASC, c ASC` /// are equivalent (in the sense that both describe the ordering of the table). - /// If the `sort_exprs` argument were `vec![b ASC, c ASC]`, then this function - /// would return `vec![d ASC]`. + /// If the `sort_exprs` argument were `vec![b ASC, c ASC, a ASC]`, then this function + /// would return `vec![a ASC, c ASC]`. ( + /// `vec![a ASC, c ASC, a ASC]` after normalization + /// `vec![a ASC, c ASC]` after de-duplication + /// ) fn normalize_sort_exprs(&self, sort_exprs: LexOrderingRef) -> LexOrdering { // Convert sort expressions to sort requirements: let sort_reqs = PhysicalSortRequirement::from_sort_exprs(sort_exprs.iter()); @@ -774,8 +782,11 @@ impl SchemaProperties { /// Assume that `self.eq_group` states column `a` and `b` are aliases. /// Also assume that `self.oeq_class` states orderings `d ASC` and `a ASC, c ASC` /// are equivalent (in the sense that both describe the ordering of the table). - /// If the `sort_reqs` argument were `vec![b ASC, c ASC]`, then this function - /// would return `vec![d ASC]`. + /// If the `sort_reqs` argument were `vec![b ASC, c ASC, a ASC]`, then this function + /// would return `vec![a ASC, c ASC]`. ( + /// `vec![a ASC, c ASC, a ASC]` after normalization + /// `vec![a ASC, c ASC]` after de-duplication + /// ) fn normalize_sort_requirements( &self, sort_reqs: LexRequirementRef, @@ -1023,7 +1034,7 @@ impl SchemaProperties { } } } - // Construct the lexicographical ordering according to the fpermutation: + // Construct the lexicographical ordering according to the permutation: ordered_exprs .into_iter() .map(|(idx, options)| { @@ -1040,17 +1051,17 @@ impl SchemaProperties { } /// Calculate ordering equivalence properties for the given join operation. -pub fn join_schema_properties( - left: SchemaProperties, - right: SchemaProperties, +pub fn join_equivalence_properties( + left: EquivalenceProperties, + right: EquivalenceProperties, join_type: &JoinType, join_schema: SchemaRef, maintains_input_order: &[bool], probe_side: Option, on: &[(Column, Column)], -) -> SchemaProperties { +) -> EquivalenceProperties { let left_size = left.schema.fields.len(); - let mut result = SchemaProperties::new(join_schema); + let mut result = EquivalenceProperties::new(join_schema); result.add_equivalence_group(left.eq_group().join( right.eq_group(), join_type, @@ -1137,7 +1148,7 @@ fn updated_right_ordering_equivalence_class( /// Calculates the [`SortProperties`] of a given [`ExprOrdering`] node. /// The node can either be a leaf node, or an intermediate node: -/// - If it is a leaf node, the children states are `None`. We directly find +/// - If it is a leaf node, the children states are empty vector. We directly find /// the order of the node by looking at the given sort expression and equivalence /// properties if it is a `Column` leaf, or we mark it as unordered. In the case /// of a `Literal` leaf, we mark it as singleton so that it can cooperate with @@ -1150,7 +1161,7 @@ fn updated_right_ordering_equivalence_class( /// the order coming from the children. fn update_ordering( mut node: ExprOrdering, - eq_properties: &SchemaProperties, + eq_properties: &EquivalenceProperties, ) -> Result> { if !node.expr.children().is_empty() { // We have an intermediate (non-leaf) node, account for its children: @@ -1214,7 +1225,7 @@ mod tests { /// [a ASC], [d ASC, b ASC], [e DESC, f ASC, g ASC] /// and /// Column [a=c] (e.g they are aliases). - fn create_test_params() -> Result<(SchemaRef, SchemaProperties)> { + fn create_test_params() -> Result<(SchemaRef, EquivalenceProperties)> { let test_schema = create_test_schema()?; let col_a = &col("a", &test_schema)?; let col_b = &col("b", &test_schema)?; @@ -1223,7 +1234,7 @@ mod tests { let col_e = &col("e", &test_schema)?; let col_f = &col("f", &test_schema)?; let col_g = &col("g", &test_schema)?; - let mut eq_properties = SchemaProperties::new(test_schema.clone()); + let mut eq_properties = EquivalenceProperties::new(test_schema.clone()); eq_properties.add_equal_conditions(col_a, col_c); let option_asc = SortOptions { @@ -1269,7 +1280,7 @@ mod tests { /// where /// Column [a=f] (e.g they are aliases). /// Column e is constant. - fn create_random_schema(seed: u64) -> Result<(SchemaRef, SchemaProperties)> { + fn create_random_schema(seed: u64) -> Result<(SchemaRef, EquivalenceProperties)> { let test_schema = create_test_schema_2()?; let col_a = &col("a", &test_schema)?; let col_b = &col("b", &test_schema)?; @@ -1279,7 +1290,7 @@ mod tests { let col_f = &col("f", &test_schema)?; let col_exprs = [col_a, col_b, col_c, col_d, col_e, col_f]; - let mut eq_properties = SchemaProperties::new(test_schema.clone()); + let mut eq_properties = EquivalenceProperties::new(test_schema.clone()); // Define a and f are aliases eq_properties.add_equal_conditions(col_a, col_f); // Column e has constant value. @@ -1357,7 +1368,7 @@ mod tests { Field::new("y", DataType::Int64, true), ])); - let mut schema_properties = SchemaProperties::new(schema); + let mut eq_properties = EquivalenceProperties::new(schema); let col_a_expr = Arc::new(Column::new("a", 0)) as Arc; let col_b_expr = Arc::new(Column::new("b", 1)) as Arc; let col_c_expr = Arc::new(Column::new("c", 2)) as Arc; @@ -1365,36 +1376,36 @@ mod tests { let col_y_expr = Arc::new(Column::new("y", 4)) as Arc; // a and b are aliases - schema_properties.add_equal_conditions(&col_a_expr, &col_b_expr); - assert_eq!(schema_properties.eq_group().len(), 1); + eq_properties.add_equal_conditions(&col_a_expr, &col_b_expr); + assert_eq!(eq_properties.eq_group().len(), 1); // This new entry is redundant, size shouldn't increase - schema_properties.add_equal_conditions(&col_b_expr, &col_a_expr); - assert_eq!(schema_properties.eq_group().len(), 1); - let eq_groups = &schema_properties.eq_group().classes[0]; + eq_properties.add_equal_conditions(&col_b_expr, &col_a_expr); + assert_eq!(eq_properties.eq_group().len(), 1); + let eq_groups = &eq_properties.eq_group().classes[0]; assert_eq!(eq_groups.len(), 2); assert!(physical_exprs_contains(eq_groups, &col_a_expr)); assert!(physical_exprs_contains(eq_groups, &col_b_expr)); // b and c are aliases. Exising equivalence class should expand, // however there shouldn't be any new equivalence class - schema_properties.add_equal_conditions(&col_b_expr, &col_c_expr); - assert_eq!(schema_properties.eq_group().len(), 1); - let eq_groups = &schema_properties.eq_group().classes[0]; + eq_properties.add_equal_conditions(&col_b_expr, &col_c_expr); + assert_eq!(eq_properties.eq_group().len(), 1); + let eq_groups = &eq_properties.eq_group().classes[0]; assert_eq!(eq_groups.len(), 3); assert!(physical_exprs_contains(eq_groups, &col_a_expr)); assert!(physical_exprs_contains(eq_groups, &col_b_expr)); assert!(physical_exprs_contains(eq_groups, &col_c_expr)); // This is a new set of equality. Hence equivalent class count should be 2. - schema_properties.add_equal_conditions(&col_x_expr, &col_y_expr); - assert_eq!(schema_properties.eq_group().len(), 2); + eq_properties.add_equal_conditions(&col_x_expr, &col_y_expr); + assert_eq!(eq_properties.eq_group().len(), 2); // This equality bridges distinct equality sets. // Hence equivalent class count should decrease from 2 to 1. - schema_properties.add_equal_conditions(&col_x_expr, &col_a_expr); - assert_eq!(schema_properties.eq_group().len(), 1); - let eq_groups = &schema_properties.eq_group().classes[0]; + eq_properties.add_equal_conditions(&col_x_expr, &col_a_expr); + assert_eq!(eq_properties.eq_group().len(), 1); + let eq_groups = &eq_properties.eq_group().classes[0]; assert_eq!(eq_groups.len(), 5); assert!(physical_exprs_contains(eq_groups, &col_a_expr)); assert!(physical_exprs_contains(eq_groups, &col_b_expr)); @@ -1413,7 +1424,7 @@ mod tests { Field::new("c", DataType::Int64, true), ])); - let input_properties = SchemaProperties::new(input_schema.clone()); + let input_properties = EquivalenceProperties::new(input_schema.clone()); let col_a = col("a", &input_schema)?; let out_schema = Arc::new(Schema::new(vec![ @@ -1466,14 +1477,14 @@ mod tests { ]; // finer ordering satisfies, crude ordering should return true let empty_schema = &Arc::new(Schema::empty()); - let mut schema_properties = SchemaProperties::new(empty_schema.clone()); - schema_properties.oeq_class.push(finer.clone()); - assert!(schema_properties.ordering_satisfy(&crude)); + let mut eq_properties_finer = EquivalenceProperties::new(empty_schema.clone()); + eq_properties_finer.oeq_class.push(finer.clone()); + assert!(eq_properties_finer.ordering_satisfy(&crude)); // Crude ordering doesn't satisfy finer ordering. should return false - let mut schema_properties = SchemaProperties::new(empty_schema.clone()); - schema_properties.oeq_class.push(crude.clone()); - assert!(!schema_properties.ordering_satisfy(&finer)); + let mut eq_properties_crude = EquivalenceProperties::new(empty_schema.clone()); + eq_properties_crude.oeq_class.push(crude.clone()); + assert!(!eq_properties_crude.ordering_satisfy(&finer)); Ok(()) } @@ -1483,7 +1494,7 @@ mod tests { // [a ASC], [d ASC, b ASC], [e DESC, f ASC, g ASC] // and // Column [a=c] (e.g they are aliases). - let (test_schema, schema_properties) = create_test_params()?; + let (test_schema, eq_properties) = create_test_params()?; let col_a = &col("a", &test_schema)?; let col_b = &col("b", &test_schema)?; let col_c = &col("c", &test_schema)?; @@ -1500,7 +1511,7 @@ mod tests { nulls_first: true, }; let table_data_with_properties = - generate_table_for_schema_properties(&schema_properties, 625, 5)?; + generate_table_for_eq_properties(&eq_properties, 625, 5)?; // First element in the tuple stores vector of requirement, second element is the expected return value for ordering_satisfy function let requirements = vec![ @@ -1637,7 +1648,7 @@ mod tests { expected ); assert_eq!( - schema_properties.ordering_satisfy(&required), + eq_properties.ordering_satisfy(&required), expected, "{err_msg}" ); @@ -1657,13 +1668,10 @@ mod tests { for seed in 0..N_RANDOM_SCHEMA { // Create a random schema with random properties - let (test_schema, schema_properties) = create_random_schema(seed as u64)?; + let (test_schema, eq_properties) = create_random_schema(seed as u64)?; // Generate a data that satisfies properties given - let table_data_with_properties = generate_table_for_schema_properties( - &schema_properties, - N_ELEMENTS, - N_DISTINCT, - )?; + let table_data_with_properties = + generate_table_for_eq_properties(&eq_properties, N_ELEMENTS, N_DISTINCT)?; let col_exprs = vec![ col("a", &test_schema)?, col("b", &test_schema)?, @@ -1693,7 +1701,7 @@ mod tests { // Check whether ordering_satisfy API result and // experimental result matches. assert_eq!( - schema_properties.ordering_satisfy(&requirement), + eq_properties.ordering_satisfy(&requirement), expected, "{}", err_msg @@ -1719,7 +1727,7 @@ mod tests { nulls_first: false, }; // a=c (e.g they are aliases). - let mut eq_properties = SchemaProperties::new(test_schema); + let mut eq_properties = EquivalenceProperties::new(test_schema); eq_properties.add_equal_conditions(col_a, col_c); let orderings = vec![ @@ -1993,18 +2001,18 @@ mod tests { let col_z = &col("z", &schema)?; let col_w = &col("w", &schema)?; - let mut join_schema_properties = SchemaProperties::new(Arc::new(schema)); + let mut join_eq_properties = EquivalenceProperties::new(Arc::new(schema)); // a=x and d=w - join_schema_properties.add_equal_conditions(col_a, col_x); - join_schema_properties.add_equal_conditions(col_d, col_w); + join_eq_properties.add_equal_conditions(col_a, col_x); + join_eq_properties.add_equal_conditions(col_d, col_w); updated_right_ordering_equivalence_class( &mut right_oeq_class, &join_type, left_columns_len, ); - join_schema_properties.add_ordering_equivalence_class(right_oeq_class); - let result = join_schema_properties.oeq_class().clone(); + join_eq_properties.add_ordering_equivalence_class(right_oeq_class); + let result = join_eq_properties.oeq_class().clone(); // [x ASC, y ASC], [z ASC, w ASC] let orderings = vec![ @@ -2101,14 +2109,14 @@ mod tests { // Generate a table that satisfies schema properties, // in terms of ordering equivalences, equivalences, and constants. - fn generate_table_for_schema_properties( - schema_properties: &SchemaProperties, + fn generate_table_for_eq_properties( + eq_properties: &EquivalenceProperties, n_elem: usize, n_distinct: usize, ) -> Result { let mut rng = StdRng::seed_from_u64(23); - let schema = schema_properties.schema(); + let schema = eq_properties.schema(); let mut schema_vec = vec![None; schema.fields.len()]; // Utility closure to generate random array @@ -2120,7 +2128,7 @@ mod tests { }; // Fill constant columns - for constant in &schema_properties.constants { + for constant in &eq_properties.constants { let col = constant.as_any().downcast_ref::().unwrap(); let (idx, _field) = schema.column_with_name(col.name()).unwrap(); let arr = @@ -2129,7 +2137,7 @@ mod tests { } // Fill columns based on ordering equivalences - for ordering in schema_properties.oeq_class.iter() { + for ordering in eq_properties.oeq_class.iter() { let (sort_columns, indices): (Vec<_>, Vec<_>) = ordering .iter() .map(|PhysicalSortExpr { expr, options }| { @@ -2153,7 +2161,7 @@ mod tests { } // Fill columns based on equivalence groups - for eq_group in schema_properties.eq_group.iter() { + for eq_group in eq_properties.eq_group.iter() { let representative_array = get_representative_arr(eq_group, &schema_vec, schema.clone()) .unwrap_or_else(|| generate_random_array(n_elem, n_distinct)); @@ -2186,7 +2194,7 @@ mod tests { let col_b = &Column::new("b", 1); let col_c = &Column::new("c", 2); // Assume that column a and c are aliases. - let (_test_schema, schema_properties) = create_test_params()?; + let (_test_schema, eq_properties) = create_test_params()?; let col_a_expr = Arc::new(col_a.clone()) as Arc; let col_b_expr = Arc::new(col_b.clone()) as Arc; @@ -2202,10 +2210,10 @@ mod tests { // Cannot normalize column b (&col_b_expr, &col_b_expr), ]; - let eq_groups = schema_properties.eq_group(); + let eq_group = eq_properties.eq_group(); for (expr, expected_eq) in expressions { assert!( - expected_eq.eq(&eq_groups.normalize_expr(expr.clone())), + expected_eq.eq(&eq_group.normalize_expr(expr.clone())), "error in test: expr: {expr:?}" ); } @@ -2220,7 +2228,7 @@ mod tests { nulls_first: false, }; // Assume that column a and c are aliases. - let (test_schema, schema_properties) = create_test_params()?; + let (test_schema, eq_properties) = create_test_params()?; let col_a = &col("a", &test_schema)?; let col_c = &col("c", &test_schema)?; let col_d = &col("d", &test_schema)?; @@ -2239,7 +2247,7 @@ mod tests { let reqs = convert_to_sort_reqs(&reqs); let expected = convert_to_sort_reqs(&expected); - let normalized = schema_properties.normalize_sort_requirements(&reqs); + let normalized = eq_properties.normalize_sort_requirements(&reqs); assert!( expected.eq(&normalized), "error in test: reqs: {reqs:?}, expected: {expected:?}, normalized: {normalized:?}" @@ -2255,7 +2263,7 @@ mod tests { // a=c // and following orderings are valid // [a ASC], [d ASC, b ASC], [e DESC, f ASC, g ASC] - let (test_schema, schema_properties) = create_test_params()?; + let (test_schema, eq_properties) = create_test_params()?; let col_a = &col("a", &test_schema)?; let col_b = &col("b", &test_schema)?; let col_c = &col("c", &test_schema)?; @@ -2316,7 +2324,7 @@ mod tests { let expected_normalized = convert_to_sort_reqs(&expected_normalized); assert_eq!( - schema_properties.normalize_sort_requirements(&req), + eq_properties.normalize_sort_requirements(&req), expected_normalized ); } @@ -2330,7 +2338,7 @@ mod tests { let col_a = &col("a", &schema)?; let col_b = &col("b", &schema)?; let col_c = &col("c", &schema)?; - let schema_properties = SchemaProperties::new(schema); + let eq_properties = EquivalenceProperties::new(schema); let option_asc = SortOptions { descending: false, nulls_first: false, @@ -2376,7 +2384,7 @@ mod tests { let lhs = convert_to_sort_reqs(&lhs); let rhs = convert_to_sort_reqs(&rhs); let expected = expected.map(|expected| convert_to_sort_reqs(&expected)); - let finer = schema_properties.get_finer_requirement(&lhs, &rhs); + let finer = eq_properties.get_finer_requirement(&lhs, &rhs); assert_eq!(finer, expected) } @@ -2388,7 +2396,7 @@ mod tests { let schema = create_test_schema()?; let col_a = &col("a", &schema)?; let col_b = &col("b", &schema)?; - let schema_properties = SchemaProperties::new(schema); + let eq_properties = EquivalenceProperties::new(schema); let option_asc = SortOptions { descending: false, nulls_first: false, @@ -2420,7 +2428,7 @@ mod tests { let lhs = convert_to_sort_exprs(&lhs); let rhs = convert_to_sort_exprs(&rhs); let expected = expected.map(|expected| convert_to_sort_exprs(&expected)); - let finer = schema_properties.get_meet_ordering(&lhs, &rhs); + let finer = eq_properties.get_meet_ordering(&lhs, &rhs); assert_eq!(finer, expected) } @@ -2499,7 +2507,7 @@ mod tests { Field::new("d", DataType::Int32, true), ]); - let mut schema_properties = SchemaProperties::new(Arc::new(schema.clone())); + let mut eq_properties = EquivalenceProperties::new(Arc::new(schema.clone())); let col_a = &col("a", &schema)?; let col_b = &col("b", &schema)?; let col_c = &col("c", &schema)?; @@ -2509,9 +2517,9 @@ mod tests { nulls_first: false, }; // b=a (e.g they are aliases) - schema_properties.add_equal_conditions(col_b, col_a); + eq_properties.add_equal_conditions(col_b, col_a); // [b ASC], [d ASC] - schema_properties.add_new_orderings(vec![ + eq_properties.add_new_orderings(vec![ vec![PhysicalSortExpr { expr: col_b.clone(), options: option_asc, @@ -2549,7 +2557,7 @@ mod tests { for (expr, expected) in test_cases { let expr_ordering = ExprOrdering::new(expr.clone()); let expr_ordering = expr_ordering - .transform_up(&|expr| update_ordering(expr, &schema_properties))?; + .transform_up(&|expr| update_ordering(expr, &eq_properties))?; let err_msg = format!( "expr:{:?}, expected: {:?}, actual: {:?}", expr, expected, expr_ordering.state diff --git a/datafusion/physical-expr/src/lib.rs b/datafusion/physical-expr/src/lib.rs index 5d0dfec8cbab..fffa8f602d87 100644 --- a/datafusion/physical-expr/src/lib.rs +++ b/datafusion/physical-expr/src/lib.rs @@ -53,7 +53,7 @@ pub use aggregate::groups_accumulator::{ }; pub use aggregate::AggregateExpr; pub use analysis::{analyze, AnalysisContext, ExprBoundaries}; -pub use equivalence::SchemaProperties; +pub use equivalence::EquivalenceProperties; pub use partitioning::{Distribution, Partitioning}; pub use physical_expr::{ physical_exprs_bag_equal, physical_exprs_contains, physical_exprs_equal, diff --git a/datafusion/physical-expr/src/partitioning.rs b/datafusion/physical-expr/src/partitioning.rs index 164c78133998..6a8fca4a1543 100644 --- a/datafusion/physical-expr/src/partitioning.rs +++ b/datafusion/physical-expr/src/partitioning.rs @@ -20,7 +20,7 @@ use std::fmt; use std::sync::Arc; -use crate::{physical_exprs_equal, PhysicalExpr, SchemaProperties}; +use crate::{physical_exprs_equal, EquivalenceProperties, PhysicalExpr}; /// Partitioning schemes supported by operators. #[derive(Debug, Clone)] @@ -63,10 +63,10 @@ impl Partitioning { /// Returns true when the guarantees made by this [[Partitioning]] are sufficient to /// satisfy the partitioning scheme mandated by the `required` [[Distribution]] - pub fn satisfy SchemaProperties>( + pub fn satisfy EquivalenceProperties>( &self, required: Distribution, - schema_properties: F, + eq_properties: F, ) -> bool { match required { Distribution::UnspecifiedDistribution => true, @@ -79,10 +79,10 @@ impl Partitioning { Partitioning::Hash(partition_exprs, _) => { let fast_match = physical_exprs_equal(&required_exprs, partition_exprs); - // If the required exprs do not match, need to leverage the schema_properties provided by the child + // If the required exprs do not match, need to leverage the eq_properties provided by the child // and normalize both exprs based on the equivalent groups. if !fast_match { - let eq_properties = schema_properties(); + let eq_properties = eq_properties(); let eq_groups = eq_properties.eq_group(); if !eq_groups.is_empty() { let normalized_required_exprs = required_exprs @@ -195,19 +195,19 @@ mod tests { for distribution in distribution_types { let result = ( single_partition.satisfy(distribution.clone(), || { - SchemaProperties::new(schema.clone()) + EquivalenceProperties::new(schema.clone()) }), unspecified_partition.satisfy(distribution.clone(), || { - SchemaProperties::new(schema.clone()) + EquivalenceProperties::new(schema.clone()) }), round_robin_partition.satisfy(distribution.clone(), || { - SchemaProperties::new(schema.clone()) + EquivalenceProperties::new(schema.clone()) }), hash_partition1.satisfy(distribution.clone(), || { - SchemaProperties::new(schema.clone()) + EquivalenceProperties::new(schema.clone()) }), hash_partition2.satisfy(distribution.clone(), || { - SchemaProperties::new(schema.clone()) + EquivalenceProperties::new(schema.clone()) }), ); diff --git a/datafusion/physical-expr/src/utils.rs b/datafusion/physical-expr/src/utils.rs index 4ed1b60669fa..a341f5d9bc2f 100644 --- a/datafusion/physical-expr/src/utils.rs +++ b/datafusion/physical-expr/src/utils.rs @@ -366,7 +366,7 @@ mod tests { use std::sync::Arc; use super::*; - use crate::equivalence::SchemaProperties; + use crate::equivalence::EquivalenceProperties; use crate::expressions::{binary, cast, col, in_list, lit, Column, Literal}; use crate::PhysicalSortExpr; @@ -646,7 +646,7 @@ mod tests { let col_a = &col("a", &schema)?; let col_b = &col("b", &schema)?; let required_columns = [col_b.clone(), col_a.clone()]; - let mut eq_properties = SchemaProperties::new(Arc::new(schema)); + let mut eq_properties = EquivalenceProperties::new(Arc::new(schema)); eq_properties.add_new_orderings([vec![ PhysicalSortExpr { expr: Arc::new(Column::new("b", 1)), @@ -681,7 +681,7 @@ mod tests { let col_a = &col("a", &schema)?; let col_b = &col("b", &schema)?; let required_columns = [col_b.clone(), col_a.clone()]; - let mut eq_properties = SchemaProperties::new(Arc::new(schema)); + let mut eq_properties = EquivalenceProperties::new(Arc::new(schema)); eq_properties.add_new_orderings([ vec![PhysicalSortExpr { expr: Arc::new(Column::new("c", 2)), @@ -723,7 +723,7 @@ mod tests { Field::new("b", DataType::Int32, true), Field::new("c", DataType::Int32, true), ]); - let mut eq_properties = SchemaProperties::new(Arc::new(schema)); + let mut eq_properties = EquivalenceProperties::new(Arc::new(schema)); // not satisfied orders eq_properties.add_new_orderings([vec![ @@ -758,7 +758,7 @@ mod tests { let col_a_expr = col("a", &schema)?; let col_b_expr = col("b", &schema)?; let col_c_expr = col("c", &schema)?; - let mut eq_properties = SchemaProperties::new(Arc::new(schema.clone())); + let mut eq_properties = EquivalenceProperties::new(Arc::new(schema.clone())); eq_properties.add_equal_conditions(&col_a_expr, &col_c_expr); let others = vec![ @@ -773,7 +773,7 @@ mod tests { ]; eq_properties.add_new_orderings(others); - let mut expected_eqs = SchemaProperties::new(Arc::new(schema)); + let mut expected_eqs = EquivalenceProperties::new(Arc::new(schema)); expected_eqs.add_new_orderings([ vec![PhysicalSortExpr { expr: col_b_expr.clone(), @@ -799,7 +799,7 @@ mod tests { Field::new("b", DataType::Int32, true), Field::new("c", DataType::Int32, true), ]); - let mut eq_properties = SchemaProperties::new(Arc::new(schema.clone())); + let mut eq_properties = EquivalenceProperties::new(Arc::new(schema.clone())); let ordering = vec![PhysicalSortExpr { expr: Arc::new(Column::new("b", 1)), options: SortOptions::default(), @@ -838,7 +838,7 @@ mod tests { Field::new("b", DataType::Int32, true), Field::new("c", DataType::Int32, true), ]); - let schema_properties = SchemaProperties::new(Arc::new(schema)); + let eq_properties = EquivalenceProperties::new(Arc::new(schema)); let projection_mapping = vec![ ( Arc::new(Column::new("c", 2)) as _, @@ -853,7 +853,7 @@ mod tests { Field::new("c_new", DataType::Int32, true), Field::new("b_new", DataType::Int32, true), ])); - let projected = schema_properties.project(&projection_mapping, projection_schema); + let projected = eq_properties.project(&projection_mapping, projection_schema); // After projection there is no ordering. assert!(projected.oeq_class().output_ordering().is_none()); diff --git a/datafusion/physical-expr/src/window/built_in.rs b/datafusion/physical-expr/src/window/built_in.rs index 1aa99ce2b745..665ceb70d658 100644 --- a/datafusion/physical-expr/src/window/built_in.rs +++ b/datafusion/physical-expr/src/window/built_in.rs @@ -25,7 +25,7 @@ use super::{BuiltInWindowFunctionExpr, WindowExpr}; use crate::expressions::PhysicalSortExpr; use crate::window::window_expr::{get_orderby_values, WindowFn}; use crate::window::{PartitionBatches, PartitionWindowAggStates, WindowState}; -use crate::{reverse_order_bys, PhysicalExpr, SchemaProperties}; +use crate::{reverse_order_bys, EquivalenceProperties, PhysicalExpr}; use arrow::array::{new_empty_array, ArrayRef}; use arrow::compute::SortOptions; @@ -72,7 +72,7 @@ impl BuiltInWindowExpr { /// If `self.expr` doesn't have an ordering, ordering equivalence properties /// are not updated. Otherwise, ordering equivalence properties are updated /// by the ordering of `self.expr`. - pub fn add_equal_orderings(&self, eq_properties: &mut SchemaProperties) { + pub fn add_equal_orderings(&self, eq_properties: &mut EquivalenceProperties) { let schema = eq_properties.schema(); if let Some(fn_res_ordering) = self.expr.get_result_ordering(schema) { if self.partition_by.is_empty() { diff --git a/datafusion/physical-plan/src/aggregates/mod.rs b/datafusion/physical-plan/src/aggregates/mod.rs index d7f74d5f1704..3947b6e1f483 100644 --- a/datafusion/physical-plan/src/aggregates/mod.rs +++ b/datafusion/physical-plan/src/aggregates/mod.rs @@ -46,9 +46,8 @@ use datafusion_physical_expr::{ aggregate::is_order_sensitive, equivalence::collapse_lex_req, expressions::{Column, Max, Min, UnKnownColumn}, - physical_exprs_contains, reverse_order_bys, AggregateExpr, LexOrdering, - LexRequirement, PhysicalExpr, PhysicalSortExpr, PhysicalSortRequirement, - SchemaProperties, + physical_exprs_contains, reverse_order_bys, AggregateExpr, EquivalenceProperties, + LexOrdering, LexRequirement, PhysicalExpr, PhysicalSortExpr, PhysicalSortRequirement, }; use itertools::{izip, Itertools}; @@ -334,18 +333,18 @@ fn get_init_req( fn get_finest_requirement( aggr_expr: &mut [Arc], order_by_expr: &mut [Option], - schema_properties: &SchemaProperties, + eq_properties: &EquivalenceProperties, ) -> Result> { // First, we check if all the requirements are satisfied by the existing // ordering. If so, we return `None` to indicate this. let mut all_satisfied = true; for (aggr_expr, fn_req) in aggr_expr.iter_mut().zip(order_by_expr.iter_mut()) { - if schema_properties.ordering_satisfy(fn_req.as_deref().unwrap_or(&[])) { + if eq_properties.ordering_satisfy(fn_req.as_deref().unwrap_or(&[])) { continue; } if let Some(reverse) = aggr_expr.reverse_expr() { let reverse_req = fn_req.as_ref().map(|item| reverse_order_bys(item)); - if schema_properties.ordering_satisfy(reverse_req.as_deref().unwrap_or(&[])) { + if eq_properties.ordering_satisfy(reverse_req.as_deref().unwrap_or(&[])) { // We need to update `aggr_expr` with its reverse since only its // reverse requirement is compatible with the existing requirements: *aggr_expr = reverse; @@ -367,8 +366,7 @@ fn get_finest_requirement( }; if let Some(finest_req) = &mut finest_req { - if let Some(finer) = schema_properties.get_finer_ordering(finest_req, fn_req) - { + if let Some(finer) = eq_properties.get_finer_ordering(finest_req, fn_req) { *finest_req = finer; continue; } @@ -377,7 +375,7 @@ fn get_finest_requirement( if let Some(reverse) = aggr_expr.reverse_expr() { let fn_req_reverse = reverse_order_bys(fn_req); if let Some(finer) = - schema_properties.get_finer_ordering(finest_req, &fn_req_reverse) + eq_properties.get_finer_ordering(finest_req, &fn_req_reverse) { // We need to update `aggr_expr` with its reverse, since only its // reverse requirement is compatible with existing requirements: @@ -506,7 +504,7 @@ impl AggregateExec { let requirement = get_finest_requirement( &mut aggr_expr, &mut order_by_expr, - &input.schema_properties(), + &input.equivalence_properties(), )?; let mut ordering_req = requirement.unwrap_or(vec![]); let partition_search_mode = get_aggregate_search_mode( @@ -543,7 +541,7 @@ impl AggregateExec { (!new_requirement.is_empty()).then_some(new_requirement); let aggregate_eqs = input - .schema_properties() + .equivalence_properties() .project(&projection_mapping, schema.clone()); let output_ordering = aggregate_eqs.oeq_class().output_ordering(); @@ -746,14 +744,14 @@ impl ExecutionPlan for AggregateExec { // First stage aggregation will not change the output partitioning, // but needs to respect aliases (e.g. mapping in the GROUP BY // expression). - let input_schema_properties = self.input.schema_properties(); + let input_eq_properties = self.input.equivalence_properties(); // First stage Aggregation will not change the output partitioning but need to respect the Alias let input_partition = self.input.output_partitioning(); if let Partitioning::Hash(exprs, part) = input_partition { let normalized_exprs = exprs .into_iter() .map(|expr| { - input_schema_properties + input_eq_properties .project_expr(&expr, &self.projection_mapping) .unwrap_or_else(|| { Arc::new(UnKnownColumn::new(&expr.to_string())) @@ -807,9 +805,9 @@ impl ExecutionPlan for AggregateExec { vec![self.required_input_ordering.clone()] } - fn schema_properties(&self) -> SchemaProperties { + fn equivalence_properties(&self) -> EquivalenceProperties { self.input - .schema_properties() + .equivalence_properties() .project(&self.projection_mapping, self.schema()) } @@ -1170,7 +1168,7 @@ mod tests { lit, ApproxDistinct, Count, FirstValue, LastValue, Median, }; use datafusion_physical_expr::{ - AggregateExpr, PhysicalExpr, PhysicalSortExpr, SchemaProperties, + AggregateExpr, EquivalenceProperties, PhysicalExpr, PhysicalSortExpr, }; use futures::{FutureExt, Stream}; @@ -2043,9 +2041,9 @@ mod tests { let col_a = &col("a", &test_schema)?; let col_b = &col("b", &test_schema)?; let col_c = &col("c", &test_schema)?; - let mut schema_properties = SchemaProperties::new(test_schema); + let mut eq_properties = EquivalenceProperties::new(test_schema); // Columns a and b are equal. - schema_properties.add_equal_conditions(col_a, col_b); + eq_properties.add_equal_conditions(col_a, col_b); // Aggregate requirements are // [None], [a ASC], [a ASC, b ASC, c ASC], [a ASC, b ASC] respectively let mut order_by_exprs = vec![ @@ -2103,11 +2101,8 @@ mod tests { vec![], )) as _; let mut aggr_exprs = vec![aggr_expr; order_by_exprs.len()]; - let res = get_finest_requirement( - &mut aggr_exprs, - &mut order_by_exprs, - &schema_properties, - )?; + let res = + get_finest_requirement(&mut aggr_exprs, &mut order_by_exprs, &eq_properties)?; assert_eq!(res, common_requirement); Ok(()) } diff --git a/datafusion/physical-plan/src/aggregates/row_hash.rs b/datafusion/physical-plan/src/aggregates/row_hash.rs index 81122ffb66d5..d6936afbd0f2 100644 --- a/datafusion/physical-plan/src/aggregates/row_hash.rs +++ b/datafusion/physical-plan/src/aggregates/row_hash.rs @@ -332,7 +332,7 @@ impl GroupedHashAggregateStream { let name = format!("GroupedHashAggregateStream[{partition}]"); let reservation = MemoryConsumer::new(name).register(context.memory_pool()); let (ordering, _) = agg - .schema_properties() + .equivalence_properties() .find_longest_permutation(&agg_group_by.output_exprs()); let group_ordering = GroupOrdering::try_new( &group_schema, diff --git a/datafusion/physical-plan/src/coalesce_batches.rs b/datafusion/physical-plan/src/coalesce_batches.rs index db11da77aa69..09d1ea87ca37 100644 --- a/datafusion/physical-plan/src/coalesce_batches.rs +++ b/datafusion/physical-plan/src/coalesce_batches.rs @@ -36,7 +36,7 @@ use arrow::error::Result as ArrowResult; use arrow::record_batch::RecordBatch; use datafusion_common::Result; use datafusion_execution::TaskContext; -use datafusion_physical_expr::SchemaProperties; +use datafusion_physical_expr::EquivalenceProperties; use futures::stream::{Stream, StreamExt}; use log::trace; @@ -134,8 +134,8 @@ impl ExecutionPlan for CoalesceBatchesExec { vec![false] } - fn schema_properties(&self) -> SchemaProperties { - self.input.schema_properties() + fn equivalence_properties(&self) -> EquivalenceProperties { + self.input.equivalence_properties() } fn with_new_children( diff --git a/datafusion/physical-plan/src/filter.rs b/datafusion/physical-plan/src/filter.rs index 656171d2aa84..ce66d614721c 100644 --- a/datafusion/physical-plan/src/filter.rs +++ b/datafusion/physical-plan/src/filter.rs @@ -45,8 +45,8 @@ use datafusion_physical_expr::expressions::BinaryExpr; use datafusion_physical_expr::intervals::utils::check_support; use datafusion_physical_expr::utils::collect_columns; use datafusion_physical_expr::{ - analyze, split_conjunction, AnalysisContext, ExprBoundaries, PhysicalExpr, - SchemaProperties, + analyze, split_conjunction, AnalysisContext, EquivalenceProperties, ExprBoundaries, + PhysicalExpr, }; use futures::stream::{Stream, StreamExt}; @@ -144,10 +144,10 @@ impl ExecutionPlan for FilterExec { vec![true] } - fn schema_properties(&self) -> SchemaProperties { + fn equivalence_properties(&self) -> EquivalenceProperties { let stats = self.statistics().unwrap(); // Combine the equal predicates with the input equivalence properties - let mut result = self.input.schema_properties(); + let mut result = self.input.equivalence_properties(); let (equal_pairs, _) = collect_columns_from_predicate(&self.predicate); for (lhs, rhs) in equal_pairs { let lhs_expr = Arc::new(lhs.clone()) as _; diff --git a/datafusion/physical-plan/src/joins/cross_join.rs b/datafusion/physical-plan/src/joins/cross_join.rs index c2f9b3632364..102f0c42e90c 100644 --- a/datafusion/physical-plan/src/joins/cross_join.rs +++ b/datafusion/physical-plan/src/joins/cross_join.rs @@ -38,8 +38,8 @@ use datafusion_common::stats::Precision; use datafusion_common::{plan_err, DataFusionError, JoinType, Result, ScalarValue}; use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; use datafusion_execution::TaskContext; -use datafusion_physical_expr::equivalence::join_schema_properties; -use datafusion_physical_expr::SchemaProperties; +use datafusion_physical_expr::equivalence::join_equivalence_properties; +use datafusion_physical_expr::EquivalenceProperties; use async_trait::async_trait; use futures::{ready, Stream, StreamExt, TryStreamExt}; @@ -214,10 +214,10 @@ impl ExecutionPlan for CrossJoinExec { None } - fn schema_properties(&self) -> SchemaProperties { - join_schema_properties( - self.left.schema_properties(), - self.right.schema_properties(), + fn equivalence_properties(&self) -> EquivalenceProperties { + join_equivalence_properties( + self.left.equivalence_properties(), + self.right.equivalence_properties(), &JoinType::Full, self.schema(), &[false, false], diff --git a/datafusion/physical-plan/src/joins/hash_join.rs b/datafusion/physical-plan/src/joins/hash_join.rs index 60834fdfba2b..0e2d552b543b 100644 --- a/datafusion/physical-plan/src/joins/hash_join.rs +++ b/datafusion/physical-plan/src/joins/hash_join.rs @@ -67,8 +67,8 @@ use datafusion_common::{ }; use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; use datafusion_execution::TaskContext; -use datafusion_physical_expr::equivalence::join_schema_properties; -use datafusion_physical_expr::SchemaProperties; +use datafusion_physical_expr::equivalence::join_equivalence_properties; +use datafusion_physical_expr::EquivalenceProperties; use ahash::RandomState; use futures::{ready, Stream, StreamExt, TryStreamExt}; @@ -479,10 +479,10 @@ impl ExecutionPlan for HashJoinExec { Self::maintains_input_order(self.join_type) } - fn schema_properties(&self) -> SchemaProperties { - join_schema_properties( - self.left.schema_properties(), - self.right.schema_properties(), + fn equivalence_properties(&self) -> EquivalenceProperties { + join_equivalence_properties( + self.left.equivalence_properties(), + self.right.equivalence_properties(), &self.join_type, self.schema(), &self.maintains_input_order(), diff --git a/datafusion/physical-plan/src/joins/sort_merge_join.rs b/datafusion/physical-plan/src/joins/sort_merge_join.rs index d314a5b6eb03..f6fdc6d77c0c 100644 --- a/datafusion/physical-plan/src/joins/sort_merge_join.rs +++ b/datafusion/physical-plan/src/joins/sort_merge_join.rs @@ -51,8 +51,8 @@ use datafusion_common::{ }; use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; use datafusion_execution::TaskContext; -use datafusion_physical_expr::equivalence::join_schema_properties; -use datafusion_physical_expr::{PhysicalSortRequirement, SchemaProperties}; +use datafusion_physical_expr::equivalence::join_equivalence_properties; +use datafusion_physical_expr::{EquivalenceProperties, PhysicalSortRequirement}; use futures::{Stream, StreamExt}; @@ -281,10 +281,10 @@ impl ExecutionPlan for SortMergeJoinExec { Self::maintains_input_order(self.join_type) } - fn schema_properties(&self) -> SchemaProperties { - join_schema_properties( - self.left.schema_properties(), - self.right.schema_properties(), + fn equivalence_properties(&self) -> EquivalenceProperties { + join_equivalence_properties( + self.left.equivalence_properties(), + self.right.equivalence_properties(), &self.join_type, self.schema(), &self.maintains_input_order(), diff --git a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs index c7ff6c21cb80..3617893a1c61 100644 --- a/datafusion/physical-plan/src/joins/symmetric_hash_join.rs +++ b/datafusion/physical-plan/src/joins/symmetric_hash_join.rs @@ -49,8 +49,8 @@ use crate::{ expressions::{Column, PhysicalSortExpr}, joins::StreamJoinPartitionMode, metrics::{self, ExecutionPlanMetricsSet, MetricBuilder, MetricsSet}, - DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, Partitioning, - RecordBatchStream, SchemaProperties, SendableRecordBatchStream, Statistics, + DisplayAs, DisplayFormatType, Distribution, EquivalenceProperties, ExecutionPlan, + Partitioning, RecordBatchStream, SendableRecordBatchStream, Statistics, }; use arrow::array::{ArrowPrimitiveType, NativeAdapter, PrimitiveArray, PrimitiveBuilder}; @@ -63,7 +63,7 @@ use datafusion_common::{ }; use datafusion_execution::memory_pool::MemoryConsumer; use datafusion_execution::TaskContext; -use datafusion_physical_expr::equivalence::join_schema_properties; +use datafusion_physical_expr::equivalence::join_equivalence_properties; use datafusion_physical_expr::intervals::ExprIntervalGraph; use ahash::RandomState; @@ -429,10 +429,10 @@ impl ExecutionPlan for SymmetricHashJoinExec { None } - fn schema_properties(&self) -> SchemaProperties { - join_schema_properties( - self.left.schema_properties(), - self.right.schema_properties(), + fn equivalence_properties(&self) -> EquivalenceProperties { + join_equivalence_properties( + self.left.equivalence_properties(), + self.right.equivalence_properties(), &self.join_type, self.schema(), &self.maintains_input_order(), diff --git a/datafusion/physical-plan/src/lib.rs b/datafusion/physical-plan/src/lib.rs index c6c8635fd55c..9295ba8c3286 100644 --- a/datafusion/physical-plan/src/lib.rs +++ b/datafusion/physical-plan/src/lib.rs @@ -35,7 +35,7 @@ use datafusion_common::{plan_err, DataFusionError, Result}; use datafusion_execution::TaskContext; use datafusion_physical_expr::expressions::Column; use datafusion_physical_expr::{ - PhysicalSortExpr, PhysicalSortRequirement, SchemaProperties, + EquivalenceProperties, PhysicalSortExpr, PhysicalSortRequirement, }; use futures::stream::TryStreamExt; @@ -187,9 +187,9 @@ pub trait ExecutionPlan: Debug + DisplayAs + Send + Sync { .collect() } - /// Get the `SchemaProperties` within the plan - fn schema_properties(&self) -> SchemaProperties { - SchemaProperties::new(self.schema()) + /// Get the [`EquivalenceProperties`] within the plan + fn equivalence_properties(&self) -> EquivalenceProperties { + EquivalenceProperties::new(self.schema()) } /// Get a list of child execution plans that provide the input for this plan. The returned list diff --git a/datafusion/physical-plan/src/limit.rs b/datafusion/physical-plan/src/limit.rs index 753dd237ce79..945dad16b794 100644 --- a/datafusion/physical-plan/src/limit.rs +++ b/datafusion/physical-plan/src/limit.rs @@ -26,7 +26,7 @@ use super::expressions::PhysicalSortExpr; use super::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; use super::{DisplayAs, RecordBatchStream, SendableRecordBatchStream, Statistics}; use crate::{ - DisplayFormatType, Distribution, ExecutionPlan, Partitioning, SchemaProperties, + DisplayFormatType, Distribution, EquivalenceProperties, ExecutionPlan, Partitioning, }; use arrow::array::ArrayRef; @@ -133,8 +133,8 @@ impl ExecutionPlan for GlobalLimitExec { self.input.output_ordering() } - fn schema_properties(&self) -> SchemaProperties { - self.input.schema_properties() + fn equivalence_properties(&self) -> EquivalenceProperties { + self.input.equivalence_properties() } fn with_new_children( @@ -318,8 +318,8 @@ impl ExecutionPlan for LocalLimitExec { vec![true] } - fn schema_properties(&self) -> SchemaProperties { - self.input.schema_properties() + fn equivalence_properties(&self) -> EquivalenceProperties { + self.input.equivalence_properties() } fn unbounded_output(&self, _children: &[bool]) -> Result { diff --git a/datafusion/physical-plan/src/memory.rs b/datafusion/physical-plan/src/memory.rs index cda3e66ad81e..5f1660a225b9 100644 --- a/datafusion/physical-plan/src/memory.rs +++ b/datafusion/physical-plan/src/memory.rs @@ -32,7 +32,7 @@ use arrow::datatypes::SchemaRef; use arrow::record_batch::RecordBatch; use datafusion_common::{internal_err, project_schema, DataFusionError, Result}; use datafusion_execution::TaskContext; -use datafusion_physical_expr::{LexOrdering, SchemaProperties}; +use datafusion_physical_expr::{EquivalenceProperties, LexOrdering}; use futures::Stream; @@ -121,8 +121,8 @@ impl ExecutionPlan for MemoryExec { .map(|ordering| ordering.as_slice()) } - fn schema_properties(&self) -> SchemaProperties { - SchemaProperties::new_with_orderings(self.schema(), &self.sort_information) + fn equivalence_properties(&self) -> EquivalenceProperties { + EquivalenceProperties::new_with_orderings(self.schema(), &self.sort_information) } fn with_new_children( @@ -178,7 +178,7 @@ impl MemoryExec { } /// A memory table can be ordered by multiple expressions simultaneously. - /// `SchemaProperties` keeps track of expressions that describe the + /// [`EquivalenceProperties`] keeps track of expressions that describe the /// global ordering of the schema. These columns are not necessarily same; e.g. /// ```text /// ┌-------┐ @@ -191,7 +191,7 @@ impl MemoryExec { /// └---┴---┘ /// ``` /// where both `a ASC` and `b DESC` can describe the table ordering. With - /// `SchemaProperties`, we can keep track of these equivalences + /// [`EquivalenceProperties`], we can keep track of these equivalences /// and treat `a ASC` and `b DESC` as the same ordering requirement. pub fn with_sort_information(mut self, sort_information: Vec) -> Self { self.sort_information = sort_information; @@ -300,7 +300,7 @@ mod tests { .with_sort_information(sort_information); assert_eq!(mem_exec.output_ordering().unwrap(), expected_output_order); - let eq_properties = mem_exec.schema_properties(); + let eq_properties = mem_exec.equivalence_properties(); assert!(eq_properties.oeq_class().contains(&expected_order_eq)); Ok(()) } diff --git a/datafusion/physical-plan/src/projection.rs b/datafusion/physical-plan/src/projection.rs index b17721a05b17..d4242f20a8a3 100644 --- a/datafusion/physical-plan/src/projection.rs +++ b/datafusion/physical-plan/src/projection.rs @@ -40,7 +40,7 @@ use datafusion_common::stats::Precision; use datafusion_common::Result; use datafusion_execution::TaskContext; use datafusion_physical_expr::expressions::{Literal, UnKnownColumn}; -use datafusion_physical_expr::SchemaProperties; +use datafusion_physical_expr::EquivalenceProperties; use futures::stream::{Stream, StreamExt}; use log::trace; @@ -96,7 +96,7 @@ impl ProjectionExec { // construct a map from the input expressions to the output expression of the Projection let projection_mapping = calculate_projection_mapping(&expr, &input_schema)?; - let input_eqs = input.schema_properties(); + let input_eqs = input.equivalence_properties(); let project_eqs = input_eqs.project(&projection_mapping, schema.clone()); let output_ordering = project_eqs.oeq_class().output_ordering(); @@ -174,12 +174,12 @@ impl ExecutionPlan for ProjectionExec { fn output_partitioning(&self) -> Partitioning { // Output partition need to respect the alias let input_partition = self.input.output_partitioning(); - let input_schema_properties = self.input.schema_properties(); + let input_eq_properties = self.input.equivalence_properties(); if let Partitioning::Hash(exprs, part) = input_partition { let normalized_exprs = exprs .into_iter() .map(|expr| { - input_schema_properties + input_eq_properties .project_expr(&expr, &self.projection_mapping) .unwrap_or_else(|| { Arc::new(UnKnownColumn::new(&expr.to_string())) @@ -201,9 +201,9 @@ impl ExecutionPlan for ProjectionExec { vec![true] } - fn schema_properties(&self) -> SchemaProperties { + fn equivalence_properties(&self) -> EquivalenceProperties { self.input - .schema_properties() + .equivalence_properties() .project(&self.projection_mapping, self.schema()) } diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index 3e03f550d7ba..ea0b4e4c537e 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -44,7 +44,7 @@ use arrow::record_batch::RecordBatch; use datafusion_common::{not_impl_err, DataFusionError, Result}; use datafusion_execution::memory_pool::MemoryConsumer; use datafusion_execution::TaskContext; -use datafusion_physical_expr::{PhysicalExpr, SchemaProperties}; +use datafusion_physical_expr::{EquivalenceProperties, PhysicalExpr}; use futures::stream::Stream; use futures::{FutureExt, StreamExt}; @@ -464,8 +464,8 @@ impl ExecutionPlan for RepartitionExec { } } - fn schema_properties(&self) -> SchemaProperties { - let mut result = self.input.schema_properties(); + fn equivalence_properties(&self) -> EquivalenceProperties { + let mut result = self.input.equivalence_properties(); // If the ordering is lost, reset the ordering equivalence class. if !self.maintains_input_order()[0] { result.clear_orderings(); diff --git a/datafusion/physical-plan/src/sorts/sort.rs b/datafusion/physical-plan/src/sorts/sort.rs index b311535a6f32..0ea9ddad7bd9 100644 --- a/datafusion/physical-plan/src/sorts/sort.rs +++ b/datafusion/physical-plan/src/sorts/sort.rs @@ -51,7 +51,7 @@ use datafusion_execution::memory_pool::{ }; use datafusion_execution::runtime_env::RuntimeEnv; use datafusion_execution::TaskContext; -use datafusion_physical_expr::SchemaProperties; +use datafusion_physical_expr::EquivalenceProperties; use futures::{StreamExt, TryStreamExt}; use log::{debug, error, trace}; @@ -828,10 +828,10 @@ impl ExecutionPlan for SortExec { Some(&self.expr) } - fn schema_properties(&self) -> SchemaProperties { + fn equivalence_properties(&self) -> EquivalenceProperties { // Reset the ordering equivalence class with the new ordering: self.input - .schema_properties() + .equivalence_properties() .with_reorder(self.expr.to_vec()) } diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index 6e6e57023c3e..f4b57e8bfb45 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -33,7 +33,7 @@ use arrow::datatypes::SchemaRef; use datafusion_common::{internal_err, DataFusionError, Result}; use datafusion_execution::memory_pool::MemoryConsumer; use datafusion_execution::TaskContext; -use datafusion_physical_expr::{PhysicalSortRequirement, SchemaProperties}; +use datafusion_physical_expr::{EquivalenceProperties, PhysicalSortRequirement}; use log::{debug, trace}; @@ -173,8 +173,8 @@ impl ExecutionPlan for SortPreservingMergeExec { vec![true] } - fn schema_properties(&self) -> SchemaProperties { - self.input.schema_properties() + fn equivalence_properties(&self) -> EquivalenceProperties { + self.input.equivalence_properties() } fn children(&self) -> Vec> { diff --git a/datafusion/physical-plan/src/streaming.rs b/datafusion/physical-plan/src/streaming.rs index 85689ef3a0b9..363047d775f9 100644 --- a/datafusion/physical-plan/src/streaming.rs +++ b/datafusion/physical-plan/src/streaming.rs @@ -28,7 +28,7 @@ use crate::{ExecutionPlan, Partitioning, SendableRecordBatchStream}; use arrow::datatypes::SchemaRef; use datafusion_common::{internal_err, plan_err, DataFusionError, Result}; use datafusion_execution::TaskContext; -use datafusion_physical_expr::{LexOrdering, PhysicalSortExpr, SchemaProperties}; +use datafusion_physical_expr::{EquivalenceProperties, LexOrdering, PhysicalSortExpr}; use async_trait::async_trait; use futures::stream::StreamExt; @@ -156,8 +156,8 @@ impl ExecutionPlan for StreamingTableExec { self.projected_output_ordering.as_deref() } - fn schema_properties(&self) -> SchemaProperties { - let mut result = SchemaProperties::new(self.schema()); + fn equivalence_properties(&self) -> EquivalenceProperties { + let mut result = EquivalenceProperties::new(self.schema()); if let Some(ordering) = &self.projected_output_ordering { result.add_new_orderings([ordering.clone()]) } diff --git a/datafusion/physical-plan/src/union.rs b/datafusion/physical-plan/src/union.rs index b788eb93a8c2..b20e0391b78e 100644 --- a/datafusion/physical-plan/src/union.rs +++ b/datafusion/physical-plan/src/union.rs @@ -40,7 +40,7 @@ use arrow::record_batch::RecordBatch; use datafusion_common::stats::Precision; use datafusion_common::{exec_err, internal_err, DFSchemaRef, DataFusionError, Result}; use datafusion_execution::TaskContext; -use datafusion_physical_expr::SchemaProperties; +use datafusion_physical_expr::EquivalenceProperties; use futures::Stream; use itertools::Itertools; @@ -223,15 +223,15 @@ impl ExecutionPlan for UnionExec { } } - fn schema_properties(&self) -> SchemaProperties { + fn equivalence_properties(&self) -> EquivalenceProperties { // TODO: In some cases, we should be able to preserve some equivalence // classes and constants. Add support for such cases. let children_eqs = self .inputs .iter() - .map(|child| child.schema_properties()) + .map(|child| child.equivalence_properties()) .collect::>(); - let mut result = SchemaProperties::new(self.schema()); + let mut result = EquivalenceProperties::new(self.schema()); // Use the ordering equivalence class of the first child as the seed: let mut meets = children_eqs[0] .oeq_class() @@ -785,7 +785,7 @@ mod tests { } #[tokio::test] - async fn test_union_schema_properties() -> Result<()> { + async fn test_union_equivalence_properties() -> Result<()> { let schema = create_test_schema()?; let col_a = &col("a", &schema)?; let col_b = &col("b", &schema)?; @@ -866,8 +866,8 @@ mod tests { ); let union = UnionExec::new(vec![child1, child2]); - let union_schema_properties = union.schema_properties(); - let union_actual_orderings = union_schema_properties.oeq_class(); + let union_eq_properties = union.equivalence_properties(); + let union_actual_orderings = union_eq_properties.oeq_class(); let err_msg = format!( "Error in test id: {:?}, test case: {:?}", test_idx, test_cases[test_idx] diff --git a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs index 29a8613ae3c8..fb679b013863 100644 --- a/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/bounded_window_agg_exec.rs @@ -31,7 +31,7 @@ use crate::expressions::PhysicalSortExpr; use crate::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; use crate::windows::{ calc_requirements, get_ordered_partition_by_indices, get_partition_by_sort_exprs, - window_ordering_equivalence, PartitionSearchMode, + window_equivalence_properties, PartitionSearchMode, }; use crate::{ ColumnStatistics, DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, @@ -57,7 +57,9 @@ use datafusion_expr::ColumnarValue; use datafusion_physical_expr::window::{ PartitionBatches, PartitionKey, PartitionWindowAggStates, WindowState, }; -use datafusion_physical_expr::{PhysicalExpr, PhysicalSortRequirement, SchemaProperties}; +use datafusion_physical_expr::{ + EquivalenceProperties, PhysicalExpr, PhysicalSortRequirement, +}; use ahash::RandomState; use futures::stream::Stream; @@ -264,9 +266,9 @@ impl ExecutionPlan for BoundedWindowAggExec { } } - /// Get the `SchemaProperties` within the plan - fn schema_properties(&self) -> SchemaProperties { - window_ordering_equivalence(&self.schema, &self.input, &self.window_expr) + /// Get the [`EquivalenceProperties`] within the plan + fn equivalence_properties(&self) -> EquivalenceProperties { + window_equivalence_properties(&self.schema, &self.input, &self.window_expr) } fn maintains_input_order(&self) -> Vec { diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index d06c1c74813a..56b9e6d174d9 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -41,7 +41,7 @@ use datafusion_physical_expr::equivalence::collapse_lex_req; use datafusion_physical_expr::{ reverse_order_bys, window::{BuiltInWindowFunctionExpr, SlidingAggregateWindowExpr}, - AggregateExpr, LexOrdering, PhysicalSortRequirement, SchemaProperties, + AggregateExpr, EquivalenceProperties, LexOrdering, PhysicalSortRequirement, }; mod bounded_window_agg_exec; @@ -326,7 +326,7 @@ pub(crate) fn get_ordered_partition_by_indices( input: &Arc, ) -> Vec { let (_, indices) = input - .schema_properties() + .equivalence_properties() .find_longest_permutation(partition_by_exprs); indices } @@ -343,7 +343,7 @@ pub(crate) fn get_partition_by_sort_exprs( // Make sure ordered section doesn't move over the partition by expression assert!(ordered_partition_by_indices.len() <= partition_by_exprs.len()); let (ordering, _) = input - .schema_properties() + .equivalence_properties() .find_longest_permutation(&ordered_partition_exprs); if ordering.len() == ordered_partition_exprs.len() { Ok(ordering) @@ -352,24 +352,24 @@ pub(crate) fn get_partition_by_sort_exprs( } } -pub(crate) fn window_ordering_equivalence( +pub(crate) fn window_equivalence_properties( schema: &SchemaRef, input: &Arc, window_expr: &[Arc], -) -> SchemaProperties { +) -> EquivalenceProperties { // We need to update the schema, so we can not directly use - // `input.schema_properties()`. - let mut window_schema_properties = - SchemaProperties::new(schema.clone()).extend(input.schema_properties()); + // `input.equivalence_properties()`. + let mut window_eq_properties = + EquivalenceProperties::new(schema.clone()).extend(input.equivalence_properties()); for expr in window_expr { if let Some(builtin_window_expr) = expr.as_any().downcast_ref::() { - builtin_window_expr.add_equal_orderings(&mut window_schema_properties); + builtin_window_expr.add_equal_orderings(&mut window_eq_properties); } } - window_schema_properties + window_eq_properties } /// Constructs the best-fitting windowing operator (a `WindowAggExec` or a @@ -464,7 +464,7 @@ pub fn get_window_mode( orderby_keys: &[PhysicalSortExpr], input: &Arc, ) -> Result> { - let input_eqs = input.schema_properties(); + let input_eqs = input.equivalence_properties(); let mut partition_by_reqs: Vec = vec![]; let (_, indices) = input_eqs.find_longest_permutation(partitionby_exprs); partition_by_reqs.extend(indices.iter().map(|&idx| PhysicalSortRequirement { diff --git a/datafusion/physical-plan/src/windows/window_agg_exec.rs b/datafusion/physical-plan/src/windows/window_agg_exec.rs index 3fba7027d1ce..6c245f65ba4f 100644 --- a/datafusion/physical-plan/src/windows/window_agg_exec.rs +++ b/datafusion/physical-plan/src/windows/window_agg_exec.rs @@ -27,7 +27,7 @@ use crate::expressions::PhysicalSortExpr; use crate::metrics::{BaselineMetrics, ExecutionPlanMetricsSet, MetricsSet}; use crate::windows::{ calc_requirements, get_ordered_partition_by_indices, get_partition_by_sort_exprs, - window_ordering_equivalence, + window_equivalence_properties, }; use crate::{ ColumnStatistics, DisplayAs, DisplayFormatType, Distribution, ExecutionPlan, @@ -47,7 +47,7 @@ use datafusion_common::stats::Precision; use datafusion_common::utils::evaluate_partition_ranges; use datafusion_common::{internal_err, plan_err, DataFusionError, Result}; use datafusion_execution::TaskContext; -use datafusion_physical_expr::{PhysicalSortRequirement, SchemaProperties}; +use datafusion_physical_expr::{EquivalenceProperties, PhysicalSortRequirement}; use futures::stream::Stream; use futures::{ready, StreamExt}; @@ -210,9 +210,9 @@ impl ExecutionPlan for WindowAggExec { } } - /// Get the `SchemaProperties` within the plan - fn schema_properties(&self) -> SchemaProperties { - window_ordering_equivalence(&self.schema, &self.input, &self.window_expr) + /// Get the [`EquivalenceProperties`] within the plan + fn equivalence_properties(&self) -> EquivalenceProperties { + window_equivalence_properties(&self.schema, &self.input, &self.window_expr) } fn with_new_children( From 2a37ed8bcfcc471be1a2bc2377d16702f32eb7a3 Mon Sep 17 00:00:00 2001 From: Mehmet Ozan Kabak Date: Tue, 31 Oct 2023 15:13:38 +0300 Subject: [PATCH 117/122] Review Part 27 --- datafusion/physical-expr/src/equivalence.rs | 53 ++++++++++----------- datafusion/physical-plan/src/union.rs | 19 ++++---- 2 files changed, 35 insertions(+), 37 deletions(-) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 7c26d2264646..7a7b26b5d3c6 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -133,8 +133,8 @@ impl EquivalenceGroup { fn remove_redundant_entries(&mut self) { // Remove duplicate entries from each equivalence class: self.classes.retain_mut(|cls| { - // Keep groups that have at least two entries: - // Single entry inside an equivalence class is meaningless. It doesn't contain any information + // Keep groups that have at least two entries as singleton class is + // meaningless (i.e. it contains no non-trivial information): deduplicate_physical_exprs(cls); cls.len() > 1 }); @@ -143,9 +143,9 @@ impl EquivalenceGroup { } /// This utility function unifies/bridges classes that have common expressions. - /// As an example if an ['EquivalenceClass'] in the state is \[a, b\] and another is \[b,c\] - /// Since, both classes contain b, this means that indeed \[a,b,c] are all equal to each other. - /// This utility converts \[a,b\], \[b,c\] to \[a,b,c\]. + /// For example, assume that we have [`EquivalenceClass`]es `[a, b]` and `[b, c]`. + /// Since both classes contain `b`, columns `a`, `b` and `c` are actually all + /// equal and belong to one class. This utility converts merges such classes. fn bridge_classes(&mut self) { let mut idx = 0; while idx < self.classes.len() { @@ -759,11 +759,10 @@ impl EquivalenceProperties { /// Assume that `self.eq_group` states column `a` and `b` are aliases. /// Also assume that `self.oeq_class` states orderings `d ASC` and `a ASC, c ASC` /// are equivalent (in the sense that both describe the ordering of the table). - /// If the `sort_exprs` argument were `vec![b ASC, c ASC, a ASC]`, then this function - /// would return `vec![a ASC, c ASC]`. ( - /// `vec![a ASC, c ASC, a ASC]` after normalization - /// `vec![a ASC, c ASC]` after de-duplication - /// ) + /// If the `sort_exprs` argument were `vec![b ASC, c ASC, a ASC]`, then this + /// function would return `vec![a ASC, c ASC]`. Internally, it would first + /// normalize to `vec![a ASC, c ASC, a ASC]` and end up with the final result + /// after deduplication. fn normalize_sort_exprs(&self, sort_exprs: LexOrderingRef) -> LexOrdering { // Convert sort expressions to sort requirements: let sort_reqs = PhysicalSortRequirement::from_sort_exprs(sort_exprs.iter()); @@ -782,11 +781,10 @@ impl EquivalenceProperties { /// Assume that `self.eq_group` states column `a` and `b` are aliases. /// Also assume that `self.oeq_class` states orderings `d ASC` and `a ASC, c ASC` /// are equivalent (in the sense that both describe the ordering of the table). - /// If the `sort_reqs` argument were `vec![b ASC, c ASC, a ASC]`, then this function - /// would return `vec![a ASC, c ASC]`. ( - /// `vec![a ASC, c ASC, a ASC]` after normalization - /// `vec![a ASC, c ASC]` after de-duplication - /// ) + /// If the `sort_reqs` argument were `vec![b ASC, c ASC, a ASC]`, then this + /// function would return `vec![a ASC, c ASC]`. Internally, it would first + /// normalize to `vec![a ASC, c ASC, a ASC]` and end up with the final result + /// after deduplication. fn normalize_sort_requirements( &self, sort_reqs: LexRequirementRef, @@ -1148,17 +1146,16 @@ fn updated_right_ordering_equivalence_class( /// Calculates the [`SortProperties`] of a given [`ExprOrdering`] node. /// The node can either be a leaf node, or an intermediate node: -/// - If it is a leaf node, the children states are empty vector. We directly find -/// the order of the node by looking at the given sort expression and equivalence -/// properties if it is a `Column` leaf, or we mark it as unordered. In the case -/// of a `Literal` leaf, we mark it as singleton so that it can cooperate with -/// some ordered columns at the upper steps. +/// - If it is a leaf node, we directly find the order of the node by looking +/// at the given sort expression and equivalence properties if it is a `Column` +/// leaf, or we mark it as unordered. In the case of a `Literal` leaf, we mark +/// it as singleton so that it can cooperate with all ordered columns. /// - If it is an intermediate node, the children states matter. Each `PhysicalExpr` -/// and operator has its own rules about how to propagate the children orderings. -/// However, before the children order propagation, it is checked that whether -/// the intermediate node can be directly matched with the sort expression. If there -/// is a match, the sort expression emerges at that node immediately, discarding -/// the order coming from the children. +/// and operator has its own rules on how to propagate the children orderings. +/// However, before we engage in recursion, we check whether this intermediate +/// node directly matches with the sort expression. If there is a match, the +/// sort expression emerges at that node immediately, discarding the recursive +/// result coming from its children. fn update_ordering( mut node: ExprOrdering, eq_properties: &EquivalenceProperties, @@ -1198,8 +1195,8 @@ mod tests { use arrow_array::{ArrayRef, RecordBatch, UInt32Array, UInt64Array}; use arrow_schema::{Fields, SortOptions}; use datafusion_common::Result; - use datafusion_expr::Operator; + use itertools::{izip, Itertools}; use rand::rngs::StdRng; use rand::seq::SliceRandom; @@ -2107,8 +2104,8 @@ mod tests { None } - // Generate a table that satisfies schema properties, - // in terms of ordering equivalences, equivalences, and constants. + // Generate a table that satisfies the given equivalence properties; i.e. + // equivalences, ordering equivalences, and constants. fn generate_table_for_eq_properties( eq_properties: &EquivalenceProperties, n_elem: usize, diff --git a/datafusion/physical-plan/src/union.rs b/datafusion/physical-plan/src/union.rs index b20e0391b78e..9700605ce406 100644 --- a/datafusion/physical-plan/src/union.rs +++ b/datafusion/physical-plan/src/union.rs @@ -242,19 +242,20 @@ impl ExecutionPlan for UnionExec { for child_eqs in &children_eqs[1..] { // Compute meet orderings of the current meets and the new ordering // equivalence class. - let mut next_meets = vec![]; - for current_meet in &meets { - // Find the all of the meets of `current_meet` with orderings inside `child_eqs` + let mut idx = 0; + while idx < meets.len() { + // Find all the meets of `current_meet` with this child's orderings: let valid_meets = child_eqs.oeq_class().iter().filter_map(|ordering| { - child_eqs.get_meet_ordering(ordering, current_meet) + child_eqs.get_meet_ordering(ordering, &meets[idx]) }); - // Use the longest meet among `valid_meets`, other meets are redundant - if let Some(next_meet) = valid_meets.max_by(|a, b| a.len().cmp(&b.len())) - { - next_meets.push(next_meet); + // Use the longest of these meets as others are redundant: + if let Some(next_meet) = valid_meets.max_by_key(|m| m.len()) { + meets[idx] = next_meet; + idx += 1; + } else { + meets.swap_remove(idx); } } - meets = next_meets; } // We know have all the valid orderings after union, remove redundant // entries (implicitly) and return: From 95531680a851c6572b4c6f74a79e9f024b89ed31 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Thu, 2 Nov 2023 11:20:24 +0300 Subject: [PATCH 118/122] Add issue links --- datafusion/physical-expr/src/equivalence.rs | 1 + datafusion/physical-expr/src/physical_expr.rs | 1 + 2 files changed, 2 insertions(+) diff --git a/datafusion/physical-expr/src/equivalence.rs b/datafusion/physical-expr/src/equivalence.rs index 7a7b26b5d3c6..04b0f2eedcdb 100644 --- a/datafusion/physical-expr/src/equivalence.rs +++ b/datafusion/physical-expr/src/equivalence.rs @@ -327,6 +327,7 @@ impl EquivalenceGroup { }); // TODO: Convert the algorithm below to a version that uses `HashMap`. // once `Arc` can be stored in `HashMap`. + // See issue: https://github.com/apache/arrow-datafusion/issues/8027 let mut new_classes = vec![]; for (source, target) in mapping { if new_classes.is_empty() { diff --git a/datafusion/physical-expr/src/physical_expr.rs b/datafusion/physical-expr/src/physical_expr.rs index 7f70de80e4df..79cbe6828b64 100644 --- a/datafusion/physical-expr/src/physical_expr.rs +++ b/datafusion/physical-expr/src/physical_expr.rs @@ -272,6 +272,7 @@ pub fn physical_exprs_bag_equal( pub fn deduplicate_physical_exprs(exprs: &mut Vec>) { // TODO: Once we can use `HashSet`s with `Arc`, this // function should use a `HashSet` to reduce computational complexity. + // See issue: https://github.com/apache/arrow-datafusion/issues/8027 let mut idx = 0; while idx < exprs.len() { let mut rest_idx = idx + 1; From 25f698d661e92bc4a978e786ad96792480fefd83 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 3 Nov 2023 09:08:41 +0300 Subject: [PATCH 119/122] Address reviews --- .../core/src/datasource/file_format/arrow.rs | 2 +- .../physical_optimizer/enforce_distribution.rs | 3 ++- .../physical-plan/src/coalesce_partitions.rs | 8 ++++++++ .../physical-plan/src/joins/nested_loop_join.rs | 16 +++++++++++++++- 4 files changed, 26 insertions(+), 3 deletions(-) diff --git a/datafusion/core/src/datasource/file_format/arrow.rs b/datafusion/core/src/datasource/file_format/arrow.rs index 900aad95747e..3dff6b6e04a1 100644 --- a/datafusion/core/src/datasource/file_format/arrow.rs +++ b/datafusion/core/src/datasource/file_format/arrow.rs @@ -106,7 +106,7 @@ const ARROW_MAGIC: [u8; 6] = [b'A', b'R', b'R', b'O', b'W', b'1']; const CONTINUATION_MARKER: [u8; 4] = [0xff; 4]; /// Custom implementation of inferring schema. Should eventually be moved upstream to arrow-rs. -/// See https://github.com/apache/arrow-rs/issues/5021 +/// See `` async fn infer_schema_from_file_stream( mut stream: BoxStream<'static, object_store::Result>, ) -> Result { diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index b0d142dfa504..11166f5fe7f2 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -4399,8 +4399,9 @@ mod tests { assert_optimized!(expected, physical_plan.clone(), true); let expected = &[ - "SortExec: expr=[c@2 ASC]", + // After CoalescePartitionsExec c is still constant. Hence c@2 ASC ordering is already satisfied. "CoalescePartitionsExec", + // Since after this stage c is constant. c@2 ASC ordering is already satisfied. "FilterExec: c@2 = 0", "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", diff --git a/datafusion/physical-plan/src/coalesce_partitions.rs b/datafusion/physical-plan/src/coalesce_partitions.rs index 28b487edf0c8..bfcff2853538 100644 --- a/datafusion/physical-plan/src/coalesce_partitions.rs +++ b/datafusion/physical-plan/src/coalesce_partitions.rs @@ -31,6 +31,7 @@ use crate::{DisplayFormatType, ExecutionPlan, Partitioning}; use arrow::datatypes::SchemaRef; use datafusion_common::{internal_err, DataFusionError, Result}; use datafusion_execution::TaskContext; +use datafusion_physical_expr::EquivalenceProperties; /// Merge execution plan executes partitions in parallel and combines them into a single /// partition. No guarantees are made about the order of the resulting partition. @@ -100,6 +101,13 @@ impl ExecutionPlan for CoalescePartitionsExec { None } + fn equivalence_properties(&self) -> EquivalenceProperties { + let mut output_eq = self.input.equivalence_properties(); + // Coalesce partitions loses existing orderings. + output_eq.clear_orderings(); + output_eq + } + fn benefits_from_input_partitioning(&self) -> Vec { vec![false] } diff --git a/datafusion/physical-plan/src/joins/nested_loop_join.rs b/datafusion/physical-plan/src/joins/nested_loop_join.rs index 4b1937763fe6..5a77ed6e2907 100644 --- a/datafusion/physical-plan/src/joins/nested_loop_join.rs +++ b/datafusion/physical-plan/src/joins/nested_loop_join.rs @@ -48,8 +48,9 @@ use datafusion_common::{exec_err, DataFusionError, JoinSide, Result, Statistics} use datafusion_execution::memory_pool::{MemoryConsumer, MemoryReservation}; use datafusion_execution::TaskContext; use datafusion_expr::JoinType; -use datafusion_physical_expr::PhysicalSortExpr; +use datafusion_physical_expr::{EquivalenceProperties, PhysicalSortExpr}; +use datafusion_physical_expr::equivalence::join_equivalence_properties; use futures::{ready, Stream, StreamExt, TryStreamExt}; /// Data of the inner table side @@ -190,6 +191,19 @@ impl ExecutionPlan for NestedLoopJoinExec { distribution_from_join_type(&self.join_type) } + fn equivalence_properties(&self) -> EquivalenceProperties { + join_equivalence_properties( + self.left.equivalence_properties(), + self.right.equivalence_properties(), + &self.join_type, + self.schema(), + &self.maintains_input_order(), + None, + // No on columns in nested loop join + &[], + ) + } + fn children(&self) -> Vec> { vec![self.left.clone(), self.right.clone()] } From 4de554070a1a99164fc00c5702c65883ef72f702 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 3 Nov 2023 10:07:42 +0300 Subject: [PATCH 120/122] Fix failing test --- .../src/physical_optimizer/enforce_distribution.rs | 12 ++---------- 1 file changed, 2 insertions(+), 10 deletions(-) diff --git a/datafusion/core/src/physical_optimizer/enforce_distribution.rs b/datafusion/core/src/physical_optimizer/enforce_distribution.rs index 11166f5fe7f2..ee6e11bd271a 100644 --- a/datafusion/core/src/physical_optimizer/enforce_distribution.rs +++ b/datafusion/core/src/physical_optimizer/enforce_distribution.rs @@ -4388,16 +4388,6 @@ mod tests { let input = parquet_exec_multiple_sorted(vec![sort_key.clone()]); let physical_plan = sort_preserving_merge_exec(sort_key, filter_exec(input)); - let expected = &[ - "SortPreservingMergeExec: [c@2 ASC]", - // Since after this stage c is constant. c@2 ASC ordering is already satisfied. - "FilterExec: c@2 = 0", - "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", - "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", - ]; - - assert_optimized!(expected, physical_plan.clone(), true); - let expected = &[ // After CoalescePartitionsExec c is still constant. Hence c@2 ASC ordering is already satisfied. "CoalescePartitionsExec", @@ -4406,6 +4396,8 @@ mod tests { "RepartitionExec: partitioning=RoundRobinBatch(10), input_partitions=2", "ParquetExec: file_groups={2 groups: [[x], [y]]}, projection=[a, b, c, d, e], output_ordering=[c@2 ASC]", ]; + + assert_optimized!(expected, physical_plan.clone(), true); assert_optimized!(expected, physical_plan, false); Ok(()) From b582cda06160d1d910dcf84ddefefd9146713ba9 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 3 Nov 2023 13:03:19 +0300 Subject: [PATCH 121/122] Update comments --- datafusion/physical-plan/src/windows/mod.rs | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) diff --git a/datafusion/physical-plan/src/windows/mod.rs b/datafusion/physical-plan/src/windows/mod.rs index 56b9e6d174d9..26dddc4ddde4 100644 --- a/datafusion/physical-plan/src/windows/mod.rs +++ b/datafusion/physical-plan/src/windows/mod.rs @@ -55,13 +55,19 @@ pub use datafusion_physical_expr::window::{ }; #[derive(Debug, Clone, PartialEq)] -/// Specifies partition column properties in terms of input ordering +/// Specifies partition expression properties in terms of existing ordering(s). +/// As an example if existing ordering is [a ASC, b ASC, c ASC], +/// `PARTITION BY b` will have `PartitionSearchMode::Linear`. +/// `PARTITION BY a, c` and `PARTITION BY c, a` will have `PartitionSearchMode::PartiallySorted(0)`, `PartitionSearchMode::PartiallySorted(1)` +/// respectively (subset `a` defines an ordered section. Indices points to index of `a` among partition by expressions). +/// `PARTITION BY a, b` and `PARTITION BY b, a` will have `PartitionSearchMode::Sorted` mode. pub enum PartitionSearchMode { - /// None of the columns among the partition columns is ordered. + /// None of the partition expressions is ordered. Linear, - /// Some columns of the partition columns are ordered but not all + /// A non-empty subset of the the partition expressions are ordered. + /// Indices stored constructs ordered subset, that is satisfied by existing ordering(s). PartiallySorted(Vec), - /// All Partition columns are ordered (Also empty case) + /// All Partition expressions are ordered (Also empty case) Sorted, } From c7f020630bb1afc8d3c27aae4e3c3a1e66d678c0 Mon Sep 17 00:00:00 2001 From: Mustafa Akur Date: Fri, 3 Nov 2023 17:47:12 +0300 Subject: [PATCH 122/122] SortPreservingMerge, SortPreservingRepartition only preserves given expression ordering among input ordering equivalences --- datafusion/physical-plan/src/repartition/mod.rs | 3 +++ datafusion/physical-plan/src/sorts/sort_preserving_merge.rs | 3 ++- 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/datafusion/physical-plan/src/repartition/mod.rs b/datafusion/physical-plan/src/repartition/mod.rs index ea0b4e4c537e..66f7037e5c2d 100644 --- a/datafusion/physical-plan/src/repartition/mod.rs +++ b/datafusion/physical-plan/src/repartition/mod.rs @@ -470,6 +470,9 @@ impl ExecutionPlan for RepartitionExec { if !self.maintains_input_order()[0] { result.clear_orderings(); } + if self.preserve_order { + result = result.with_reorder(self.sort_exprs().unwrap_or_default().to_vec()) + } result } diff --git a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs index f4b57e8bfb45..65cd8e41480e 100644 --- a/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs +++ b/datafusion/physical-plan/src/sorts/sort_preserving_merge.rs @@ -174,7 +174,8 @@ impl ExecutionPlan for SortPreservingMergeExec { } fn equivalence_properties(&self) -> EquivalenceProperties { - self.input.equivalence_properties() + let output_oeq = self.input.equivalence_properties(); + output_oeq.with_reorder(self.expr.to_vec()) } fn children(&self) -> Vec> {